DL-4: Repackage the source under apache namespace
diff --git a/distributedlog-benchmark/conf/log4j.properties b/distributedlog-benchmark/conf/log4j.properties
index 73b4cfa..930db8d 100644
--- a/distributedlog-benchmark/conf/log4j.properties
+++ b/distributedlog-benchmark/conf/log4j.properties
@@ -30,11 +30,11 @@
 log4j.logger.org.apache.bookkeeper=INFO
 
 # redirect executor output to executors.log since slow op warnings can be quite verbose
-log4j.logger.com.twitter.distributedlog.util.MonitoredFuturePool=INFO, Executors
-log4j.logger.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=INFO, Executors
+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.com.twitter.distributedlog.util.MonitoredFuturePool=false
-log4j.additivity.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=false
+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/com/twitter/distributedlog/benchmark/Benchmarker.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Benchmarker.java
deleted file mode 100644
index 5b04a05..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Benchmarker.java
+++ /dev/null
@@ -1,468 +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 com.twitter.distributedlog.benchmark;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.benchmark.utils.ShiftableRateLimiter;
-import com.twitter.finagle.stats.OstrichStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.NullStatsProvider;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.stats.StatsProvider;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.cli.BasicParser;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The launcher for benchmarks.
- */
-public class Benchmarker {
-
-    private static final Logger logger = LoggerFactory.getLogger(Benchmarker.class);
-
-    static final String USAGE = "Benchmarker [-u <uri>] [-c <conf>] [-s serverset] [-m (read|write|dlwrite)]";
-
-    final String[] args;
-    final Options options = new Options();
-
-    int rate = 100;
-    int maxRate = 1000;
-    int changeRate = 100;
-    int changeRateSeconds = 1800;
-    int concurrency = 10;
-    String streamPrefix = "dlog-loadtest";
-    int shardId = -1;
-    int numStreams = 10;
-    List<String> serversetPaths = new ArrayList<String>();
-    List<String> finagleNames = new ArrayList<String>();
-    int msgSize = 256;
-    String mode = null;
-    int durationMins = 60;
-    URI dlUri = null;
-    int batchSize = 0;
-    int readersPerStream = 1;
-    Integer maxStreamId = null;
-    int truncationInterval = 3600;
-    Integer startStreamId = null;
-    Integer endStreamId = null;
-    int hostConnectionCoreSize = 10;
-    int hostConnectionLimit = 10;
-    boolean thriftmux = false;
-    boolean handshakeWithClientInfo = false;
-    boolean readFromHead = false;
-    int sendBufferSize = 1024 * 1024;
-    int recvBufferSize = 1024 * 1024;
-    boolean enableBatching = false;
-    int batchBufferSize = 256 * 1024;
-    int batchFlushIntervalMicros = 2000;
-    String routingServiceFinagleNameString;
-
-    final DistributedLogConfiguration conf = new DistributedLogConfiguration();
-    final StatsReceiver statsReceiver = new OstrichStatsReceiver();
-    StatsProvider statsProvider = null;
-
-    Benchmarker(String[] args) {
-        this.args = args;
-        // prepare options
-        options.addOption("s", "serverset", true, "Proxy Server Set (separated by ',')");
-        options.addOption("fn", "finagle-name", true, "Write proxy finagle name (separated by ',')");
-        options.addOption("c", "conf", true, "DistributedLog Configuration File");
-        options.addOption("u", "uri", true, "DistributedLog URI");
-        options.addOption("i", "shard", true, "Shard Id");
-        options.addOption("p", "provider", true, "DistributedLog Stats Provider");
-        options.addOption("d", "duration", true, "Duration (minutes)");
-        options.addOption("sp", "streamprefix", true, "Stream Prefix");
-        options.addOption("sc", "streamcount", true, "Number of Streams");
-        options.addOption("ms", "messagesize", true, "Message Size (bytes)");
-        options.addOption("bs", "batchsize", true, "Batch Size");
-        options.addOption("r", "rate", true, "Rate limit (requests/second)");
-        options.addOption("mr", "max-rate", true, "Maximum Rate limit (requests/second)");
-        options.addOption("cr", "change-rate", true, "Rate to increase each change period (requests/second)");
-        options.addOption("ci", "change-interval", true, "Rate to increase period, seconds");
-        options.addOption("t", "concurrency", true, "Concurrency (number of threads)");
-        options.addOption("m", "mode", true, "Benchmark mode (read/write)");
-        options.addOption("rps", "readers-per-stream", true, "Number readers per stream");
-        options.addOption("msid", "max-stream-id", true, "Max Stream ID");
-        options.addOption("ti", "truncation-interval", true, "Truncation interval in seconds");
-        options.addOption("ssid", "start-stream-id", true, "Start Stream ID");
-        options.addOption("esid", "end-stream-id", true, "Start Stream ID");
-        options.addOption("hccs", "host-connection-core-size", true, "Finagle hostConnectionCoreSize");
-        options.addOption("hcl", "host-connection-limit", true, "Finagle hostConnectionLimit");
-        options.addOption("mx", "thriftmux", false, "Enable thriftmux (write mode only)");
-        options.addOption("hsci", "handshake-with-client-info", false, "Enable handshaking with client info");
-        options.addOption("rfh", "read-from-head", false, "Read from head of the stream");
-        options.addOption("sb", "send-buffer", true, "Channel send buffer size, in bytes");
-        options.addOption("rb", "recv-buffer", true, "Channel recv buffer size, in bytes");
-        options.addOption("bt", "enable-batch", false, "Enable batching on writers");
-        options.addOption("bbs", "batch-buffer-size", true, "The batch buffer size in bytes");
-        options.addOption("bfi", "batch-flush-interval", true, "The batch buffer flush interval in micros");
-        options.addOption("rs", "routing-service", true, "The routing service finagle name for server-side routing");
-        options.addOption("h", "help", false, "Print usage.");
-    }
-
-    void printUsage() {
-        HelpFormatter helpFormatter = new HelpFormatter();
-        helpFormatter.printHelp(USAGE, options);
-    }
-
-    void run() throws Exception {
-        logger.info("Running benchmark.");
-
-        BasicParser parser = new BasicParser();
-        CommandLine cmdline = parser.parse(options, args);
-        if (cmdline.hasOption("h")) {
-            printUsage();
-            System.exit(0);
-        }
-        if (cmdline.hasOption("s")) {
-            String serversetPathStr = cmdline.getOptionValue("s");
-            serversetPaths = Arrays.asList(StringUtils.split(serversetPathStr, ','));
-        }
-        if (cmdline.hasOption("fn")) {
-            String finagleNameStr = cmdline.getOptionValue("fn");
-            finagleNames = Arrays.asList(StringUtils.split(finagleNameStr, ','));
-        }
-        if (cmdline.hasOption("i")) {
-            shardId = Integer.parseInt(cmdline.getOptionValue("i"));
-        }
-        if (cmdline.hasOption("d")) {
-            durationMins = Integer.parseInt(cmdline.getOptionValue("d"));
-        }
-        if (cmdline.hasOption("sp")) {
-            streamPrefix = cmdline.getOptionValue("sp");
-        }
-        if (cmdline.hasOption("sc")) {
-            numStreams = Integer.parseInt(cmdline.getOptionValue("sc"));
-        }
-        if (cmdline.hasOption("ms")) {
-            msgSize = Integer.parseInt(cmdline.getOptionValue("ms"));
-        }
-        if (cmdline.hasOption("r")) {
-            rate = Integer.parseInt(cmdline.getOptionValue("r"));
-        }
-        if (cmdline.hasOption("mr")) {
-            maxRate = Integer.parseInt(cmdline.getOptionValue("mr"));
-        }
-        if (cmdline.hasOption("cr")) {
-            changeRate = Integer.parseInt(cmdline.getOptionValue("cr"));
-        }
-        if (cmdline.hasOption("ci")) {
-            changeRateSeconds = Integer.parseInt(cmdline.getOptionValue("ci"));
-        }
-        if (cmdline.hasOption("t")) {
-            concurrency = Integer.parseInt(cmdline.getOptionValue("t"));
-        }
-        if (cmdline.hasOption("m")) {
-            mode = cmdline.getOptionValue("m");
-        }
-        if (cmdline.hasOption("u")) {
-            dlUri = URI.create(cmdline.getOptionValue("u"));
-        }
-        if (cmdline.hasOption("bs")) {
-            batchSize = Integer.parseInt(cmdline.getOptionValue("bs"));
-            checkArgument("write" != mode, "batchSize supported only for mode=write");
-        }
-        if (cmdline.hasOption("c")) {
-            String configFile = cmdline.getOptionValue("c");
-            conf.loadConf(new File(configFile).toURI().toURL());
-        }
-        if (cmdline.hasOption("rps")) {
-            readersPerStream = Integer.parseInt(cmdline.getOptionValue("rps"));
-        }
-        if (cmdline.hasOption("msid")) {
-            maxStreamId = Integer.parseInt(cmdline.getOptionValue("msid"));
-        }
-        if (cmdline.hasOption("ti")) {
-            truncationInterval = Integer.parseInt(cmdline.getOptionValue("ti"));
-        }
-        if (cmdline.hasOption("ssid")) {
-            startStreamId = Integer.parseInt(cmdline.getOptionValue("ssid"));
-        }
-        if (cmdline.hasOption("esid")) {
-            endStreamId = Integer.parseInt(cmdline.getOptionValue("esid"));
-        }
-        if (cmdline.hasOption("hccs")) {
-            hostConnectionCoreSize = Integer.parseInt(cmdline.getOptionValue("hccs"));
-        }
-        if (cmdline.hasOption("hcl")) {
-            hostConnectionLimit = Integer.parseInt(cmdline.getOptionValue("hcl"));
-        }
-        if (cmdline.hasOption("sb")) {
-            sendBufferSize = Integer.parseInt(cmdline.getOptionValue("sb"));
-        }
-        if (cmdline.hasOption("rb")) {
-            recvBufferSize = Integer.parseInt(cmdline.getOptionValue("rb"));
-        }
-        if (cmdline.hasOption("rs")) {
-            routingServiceFinagleNameString = cmdline.getOptionValue("rs");
-        }
-        thriftmux = cmdline.hasOption("mx");
-        handshakeWithClientInfo = cmdline.hasOption("hsci");
-        readFromHead = cmdline.hasOption("rfh");
-        enableBatching = cmdline.hasOption("bt");
-        if (cmdline.hasOption("bbs")) {
-            batchBufferSize = Integer.parseInt(cmdline.getOptionValue("bbs"));
-        }
-        if (cmdline.hasOption("bfi")) {
-            batchFlushIntervalMicros = Integer.parseInt(cmdline.getOptionValue("bfi"));
-        }
-
-        checkArgument(shardId >= 0, "shardId must be >= 0");
-        checkArgument(numStreams > 0, "numStreams must be > 0");
-        checkArgument(durationMins > 0, "durationMins must be > 0");
-        checkArgument(streamPrefix != null, "streamPrefix must be defined");
-        checkArgument(hostConnectionCoreSize > 0, "host connection core size must be > 0");
-        checkArgument(hostConnectionLimit > 0, "host connection limit must be > 0");
-
-        if (cmdline.hasOption("p")) {
-            statsProvider = ReflectionUtils.newInstance(cmdline.getOptionValue("p"), StatsProvider.class);
-        } else {
-            statsProvider = new NullStatsProvider();
-        }
-
-        logger.info("Starting stats provider : {}.", statsProvider.getClass());
-        statsProvider.start(conf);
-
-        Worker w = null;
-        if (mode.startsWith("read")) {
-            w = runReader();
-        } else if (mode.startsWith("write")) {
-            w = runWriter();
-        } else if (mode.startsWith("dlwrite")) {
-            w = runDLWriter();
-        } else if (mode.startsWith("dlread")) {
-            w = runDLReader();
-        }
-
-        if (w == null) {
-            throw new IOException("Unknown mode " + mode + " to run the benchmark.");
-        }
-
-        Thread workerThread = new Thread(w, mode + "-benchmark-thread");
-        workerThread.start();
-
-        TimeUnit.MINUTES.sleep(durationMins);
-
-        logger.info("{} minutes passed, exiting...", durationMins);
-        w.close();
-
-        if (null != statsProvider) {
-            statsProvider.stop();
-        }
-
-        Runtime.getRuntime().exit(0);
-    }
-
-    Worker runWriter() {
-        checkArgument(!finagleNames.isEmpty() || !serversetPaths.isEmpty() || null != dlUri,
-                "either serverset paths, finagle-names or uri required");
-        checkArgument(msgSize > 0, "messagesize must be greater than 0");
-        checkArgument(rate > 0, "rate must be greater than 0");
-        checkArgument(maxRate >= rate, "max rate must be greater than rate");
-        checkArgument(changeRate >= 0, "change rate must be positive");
-        checkArgument(changeRateSeconds >= 0, "change rate must be positive");
-        checkArgument(concurrency > 0, "concurrency must be greater than 0");
-
-        ShiftableRateLimiter rateLimiter =
-                new ShiftableRateLimiter(rate, maxRate, changeRate, changeRateSeconds, TimeUnit.SECONDS);
-        return createWriteWorker(
-                streamPrefix,
-                dlUri,
-                null == startStreamId ? shardId * numStreams : startStreamId,
-                null == endStreamId ? (shardId + 1) * numStreams : endStreamId,
-                rateLimiter,
-                concurrency,
-                msgSize,
-                batchSize,
-                hostConnectionCoreSize,
-                hostConnectionLimit,
-                serversetPaths,
-                finagleNames,
-                statsReceiver.scope("write_client"),
-                statsProvider.getStatsLogger("write"),
-                thriftmux,
-                handshakeWithClientInfo,
-                sendBufferSize,
-                recvBufferSize,
-                enableBatching,
-                batchBufferSize,
-                batchFlushIntervalMicros,
-                routingServiceFinagleNameString);
-    }
-
-    protected WriterWorker createWriteWorker(
-            String streamPrefix,
-            URI uri,
-            int startStreamId,
-            int endStreamId,
-            ShiftableRateLimiter rateLimiter,
-            int writeConcurrency,
-            int messageSizeBytes,
-            int batchSize,
-            int hostConnectionCoreSize,
-            int hostConnectionLimit,
-            List<String> serverSetPaths,
-            List<String> finagleNames,
-            StatsReceiver statsReceiver,
-            StatsLogger statsLogger,
-            boolean thriftmux,
-            boolean handshakeWithClientInfo,
-            int sendBufferSize,
-            int recvBufferSize,
-            boolean enableBatching,
-            int batchBufferSize,
-            int batchFlushIntervalMicros,
-            String routingServiceFinagleNameString) {
-        return new WriterWorker(
-                streamPrefix,
-                uri,
-                startStreamId,
-                endStreamId,
-                rateLimiter,
-                writeConcurrency,
-                messageSizeBytes,
-                batchSize,
-                hostConnectionCoreSize,
-                hostConnectionLimit,
-                serverSetPaths,
-                finagleNames,
-                statsReceiver,
-                statsLogger,
-                thriftmux,
-                handshakeWithClientInfo,
-                sendBufferSize,
-                recvBufferSize,
-                enableBatching,
-                batchBufferSize,
-                batchFlushIntervalMicros,
-                routingServiceFinagleNameString);
-    }
-
-    Worker runDLWriter() throws IOException {
-        checkNotNull(dlUri, "dlUri must be defined");
-        checkArgument(rate > 0, "rate must be greater than 0");
-        checkArgument(maxRate >= rate, "max rate must be greater than rate");
-        checkArgument(changeRate >= 0, "change rate must be positive");
-        checkArgument(changeRateSeconds >= 0, "change rate must be positive");
-        checkArgument(concurrency > 0, "concurrency must be greater than 0");
-
-        ShiftableRateLimiter rateLimiter =
-                new ShiftableRateLimiter(rate, maxRate, changeRate, changeRateSeconds, TimeUnit.SECONDS);
-
-        return new DLWriterWorker(conf,
-                dlUri,
-                streamPrefix,
-                shardId * numStreams,
-                (shardId + 1) * numStreams,
-                rateLimiter,
-                concurrency,
-                msgSize,
-                statsProvider.getStatsLogger("dlwrite"));
-    }
-
-    Worker runReader() throws IOException {
-        checkArgument(!finagleNames.isEmpty() || !serversetPaths.isEmpty() || null != dlUri,
-                "either serverset paths, finagle-names or dlUri required");
-        checkArgument(concurrency > 0, "concurrency must be greater than 0");
-        checkArgument(truncationInterval > 0, "truncation interval should be greater than 0");
-        return runReaderInternal(serversetPaths, finagleNames, truncationInterval);
-    }
-
-    Worker runDLReader() throws IOException {
-        return runReaderInternal(new ArrayList<String>(), new ArrayList<String>(), 0);
-    }
-
-    private Worker runReaderInternal(List<String> serversetPaths,
-                                     List<String> finagleNames,
-                                     int truncationInterval) throws IOException {
-        checkNotNull(dlUri);
-
-        int ssid = null == startStreamId ? shardId * numStreams : startStreamId;
-        int esid = null == endStreamId ? (shardId + readersPerStream) * numStreams : endStreamId;
-        if (null != maxStreamId) {
-            esid = Math.min(esid, maxStreamId);
-        }
-
-        return createReaderWorker(
-                conf,
-                dlUri,
-                streamPrefix,
-                ssid,
-                esid,
-                concurrency,
-                serversetPaths,
-                finagleNames,
-                truncationInterval,
-                readFromHead,
-                statsReceiver,
-                statsProvider.getStatsLogger("dlreader"));
-    }
-
-    protected ReaderWorker createReaderWorker(
-            DistributedLogConfiguration conf,
-            URI uri,
-            String streamPrefix,
-            int startStreamId,
-            int endStreamId,
-            int readThreadPoolSize,
-            List<String> serverSetPaths,
-            List<String> finagleNames,
-            int truncationIntervalInSeconds,
-            boolean readFromHead, /* read from the earliest data of log */
-            StatsReceiver statsReceiver,
-            StatsLogger statsLogger) throws IOException {
-        return new ReaderWorker(
-                conf,
-                uri,
-                streamPrefix,
-                startStreamId,
-                endStreamId,
-                readThreadPoolSize,
-                serverSetPaths,
-                finagleNames,
-                truncationIntervalInSeconds,
-                readFromHead,
-                statsReceiver,
-                statsLogger);
-    }
-
-    public static void main(String[] args) {
-        Benchmarker benchmarker = new Benchmarker(args);
-        try {
-            benchmarker.run();
-        } catch (Exception e) {
-            logger.info("Benchmark quit due to : ", e);
-        }
-    }
-
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/DLWriterWorker.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/DLWriterWorker.java
deleted file mode 100644
index 152cd32..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/DLWriterWorker.java
+++ /dev/null
@@ -1,245 +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 com.twitter.distributedlog.benchmark;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.benchmark.utils.ShiftableRateLimiter;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.util.FutureEventListener;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The benchmark for core library writer.
- */
-public class DLWriterWorker implements Worker {
-
-    private static final Logger LOG = LoggerFactory.getLogger(DLWriterWorker.class);
-
-    static final int BACKOFF_MS = 200;
-
-    final String streamPrefix;
-    final int startStreamId;
-    final int endStreamId;
-    final int writeConcurrency;
-    final int messageSizeBytes;
-    final ExecutorService executorService;
-    final ScheduledExecutorService rescueService;
-    final ShiftableRateLimiter rateLimiter;
-    final Random random;
-    final DistributedLogNamespace namespace;
-    final List<DistributedLogManager> dlms;
-    final List<AsyncLogWriter> streamWriters;
-    final int numStreams;
-
-    volatile boolean running = true;
-
-    final StatsLogger statsLogger;
-    final OpStatsLogger requestStat;
-
-    public DLWriterWorker(DistributedLogConfiguration conf,
-                          URI uri,
-                          String streamPrefix,
-                          int startStreamId,
-                          int endStreamId,
-                          ShiftableRateLimiter rateLimiter,
-                          int writeConcurrency,
-                          int messageSizeBytes,
-                          StatsLogger statsLogger) throws IOException {
-        checkArgument(startStreamId <= endStreamId);
-        this.streamPrefix = streamPrefix;
-        this.startStreamId = startStreamId;
-        this.endStreamId = endStreamId;
-        this.rateLimiter = rateLimiter;
-        this.writeConcurrency = writeConcurrency;
-        this.messageSizeBytes = messageSizeBytes;
-        this.statsLogger = statsLogger;
-        this.requestStat = this.statsLogger.getOpStatsLogger("requests");
-        this.executorService = Executors.newCachedThreadPool();
-        this.rescueService = Executors.newSingleThreadScheduledExecutor();
-        this.random = new Random(System.currentTimeMillis());
-
-        this.namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(uri)
-                .statsLogger(statsLogger.scope("dl"))
-                .build();
-        this.numStreams = endStreamId - startStreamId;
-        dlms = new ArrayList<DistributedLogManager>(numStreams);
-        streamWriters = new ArrayList<AsyncLogWriter>(numStreams);
-        final ConcurrentMap<String, AsyncLogWriter> writers = new ConcurrentHashMap<String, AsyncLogWriter>();
-        final CountDownLatch latch = new CountDownLatch(this.numStreams);
-        for (int i = startStreamId; i < endStreamId; i++) {
-            final String streamName = String.format("%s_%d", streamPrefix, i);
-            final DistributedLogManager dlm = namespace.openLog(streamName);
-            executorService.submit(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-                        if (null != writers.putIfAbsent(streamName, writer)) {
-                            FutureUtils.result(writer.asyncClose());
-                        }
-                        latch.countDown();
-                    } catch (IOException e) {
-                        LOG.error("Failed to intialize writer for stream : {}", streamName, e);
-                    }
-
-                }
-            });
-            dlms.add(dlm);
-        }
-        try {
-            latch.await();
-        } catch (InterruptedException e) {
-            throw new IOException("Interrupted on initializing writers for streams.", e);
-        }
-        for (int i = startStreamId; i < endStreamId; i++) {
-            final String streamName = String.format("%s_%d", streamPrefix, i);
-            AsyncLogWriter writer = writers.get(streamName);
-            if (null == writer) {
-                throw new IOException("Writer for " + streamName + " never initialized.");
-            }
-            streamWriters.add(writer);
-        }
-        LOG.info("Writing to {} streams.", numStreams);
-    }
-
-    void rescueWriter(int idx, AsyncLogWriter writer) {
-        if (streamWriters.get(idx) == writer) {
-            try {
-                FutureUtils.result(writer.asyncClose());
-            } catch (IOException e) {
-                LOG.error("Failed to close writer for stream {}.", idx);
-            }
-            AsyncLogWriter newWriter = null;
-            try {
-                newWriter = dlms.get(idx).startAsyncLogSegmentNonPartitioned();
-            } catch (IOException e) {
-                LOG.error("Failed to create new writer for stream {}, backoff for {} ms.",
-                          idx, BACKOFF_MS);
-                scheduleRescue(idx, writer, BACKOFF_MS);
-            }
-            streamWriters.set(idx, newWriter);
-        } else {
-            LOG.warn("AsyncLogWriter for stream {} was already rescued.", idx);
-        }
-    }
-
-    void scheduleRescue(final int idx, final AsyncLogWriter writer, int delayMs) {
-        Runnable r = new Runnable() {
-            @Override
-            public void run() {
-                rescueWriter(idx, writer);
-            }
-        };
-        if (delayMs > 0) {
-            rescueService.schedule(r, delayMs, TimeUnit.MILLISECONDS);
-        } else {
-            rescueService.submit(r);
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        this.running = false;
-        SchedulerUtils.shutdownScheduler(this.executorService, 2, TimeUnit.MINUTES);
-        SchedulerUtils.shutdownScheduler(this.rescueService, 2, TimeUnit.MINUTES);
-        for (AsyncLogWriter writer : streamWriters) {
-            FutureUtils.result(writer.asyncClose());
-        }
-        for (DistributedLogManager dlm : dlms) {
-            dlm.close();
-        }
-        namespace.close();
-    }
-
-    @Override
-    public void run() {
-        LOG.info("Starting dlwriter (concurrency = {}, prefix = {}, numStreams = {})",
-                 new Object[] { writeConcurrency, streamPrefix, numStreams });
-        for (int i = 0; i < writeConcurrency; i++) {
-            executorService.submit(new Writer(i));
-        }
-    }
-
-    class Writer implements Runnable {
-
-        final int idx;
-
-        Writer(int idx) {
-            this.idx = idx;
-        }
-
-        @Override
-        public void run() {
-            LOG.info("Started writer {}.", idx);
-            while (running) {
-                final int streamIdx = random.nextInt(numStreams);
-                final AsyncLogWriter writer = streamWriters.get(streamIdx);
-                rateLimiter.getLimiter().acquire();
-                final long requestMillis = System.currentTimeMillis();
-                final byte[] data;
-                try {
-                    data = Utils.generateMessage(requestMillis, messageSizeBytes);
-                } catch (TException e) {
-                    LOG.error("Error on generating message : ", e);
-                    break;
-                }
-                writer.write(new LogRecord(requestMillis, data)).addEventListener(new FutureEventListener<DLSN>() {
-                    @Override
-                    public void onSuccess(DLSN value) {
-                        requestStat.registerSuccessfulEvent(System.currentTimeMillis() - requestMillis);
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        requestStat.registerFailedEvent(System.currentTimeMillis() - requestMillis);
-                        LOG.error("Failed to publish, rescue it : ", cause);
-                        scheduleRescue(streamIdx, writer, 0);
-                    }
-                });
-            }
-        }
-    }
-
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/ReaderWorker.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/ReaderWorker.java
deleted file mode 100644
index adbdeda..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/ReaderWorker.java
+++ /dev/null
@@ -1,468 +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 com.twitter.distributedlog.benchmark;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.common.base.Stopwatch;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.AsyncLogReader;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogRecordSet;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.benchmark.thrift.Message;
-import com.twitter.distributedlog.client.serverset.DLZkServerSet;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Duration$;
-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.net.URI;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The benchmark for core library reader.
- */
-public class ReaderWorker implements Worker {
-
-    private static final Logger LOG = LoggerFactory.getLogger(ReaderWorker.class);
-
-    static final int BACKOFF_MS = 200;
-
-    final String streamPrefix;
-    final int startStreamId;
-    final int endStreamId;
-    final ScheduledExecutorService executorService;
-    final ExecutorService callbackExecutor;
-    final DistributedLogNamespace namespace;
-    final DistributedLogManager[] dlms;
-    final AsyncLogReader[] logReaders;
-    final StreamReader[] streamReaders;
-    final int numStreams;
-    final boolean readFromHead;
-
-    final int truncationIntervalInSeconds;
-    // DL Client Related Variables
-    final DLZkServerSet[] serverSets;
-    final List<String> finagleNames;
-    final DistributedLogClient dlc;
-
-    volatile boolean running = true;
-
-    final StatsReceiver statsReceiver;
-    final StatsLogger statsLogger;
-    final OpStatsLogger e2eStat;
-    final OpStatsLogger deliveryStat;
-    final OpStatsLogger negativeE2EStat;
-    final OpStatsLogger negativeDeliveryStat;
-    final OpStatsLogger truncationStat;
-    final Counter invalidRecordsCounter;
-    final Counter outOfOrderSequenceIdCounter;
-
-    class StreamReader implements FutureEventListener<List<LogRecordWithDLSN>>, Runnable, Gauge<Number> {
-
-        final int streamIdx;
-        final String streamName;
-        DLSN prevDLSN = null;
-        long prevSequenceId = Long.MIN_VALUE;
-        private static final String gaugeLabel = "sequence_id";
-
-        StreamReader(int idx, StatsLogger statsLogger) {
-            this.streamIdx = idx;
-            int streamId = startStreamId + streamIdx;
-            streamName = String.format("%s_%d", streamPrefix, streamId);
-            statsLogger.scope(streamName).registerGauge(gaugeLabel, this);
-        }
-
-        @Override
-        public void onSuccess(final List<LogRecordWithDLSN> records) {
-            for (final LogRecordWithDLSN record : records) {
-                if (record.isRecordSet()) {
-                    try {
-                        processRecordSet(record);
-                    } catch (IOException e) {
-                        onFailure(e);
-                    }
-                } else {
-                    processRecord(record);
-                }
-            }
-            readLoop();
-        }
-
-        public void processRecordSet(final LogRecordWithDLSN record) throws IOException {
-            LogRecordSet.Reader reader = LogRecordSet.of(record);
-            LogRecordWithDLSN nextRecord = reader.nextRecord();
-            while (null != nextRecord) {
-                processRecord(nextRecord);
-                nextRecord = reader.nextRecord();
-            }
-        }
-
-        public void processRecord(final LogRecordWithDLSN record) {
-            Message msg;
-            try {
-                msg = Utils.parseMessage(record.getPayload());
-            } catch (TException e) {
-                invalidRecordsCounter.inc();
-                LOG.warn("Failed to parse record {} for stream {} : size = {} , ",
-                         new Object[] { record, streamIdx, record.getPayload().length, e });
-                return;
-            }
-            long curTimeMillis = System.currentTimeMillis();
-            long e2eLatency = curTimeMillis - msg.getPublishTime();
-            long deliveryLatency = curTimeMillis - record.getTransactionId();
-            if (e2eLatency >= 0) {
-                e2eStat.registerSuccessfulEvent(e2eLatency);
-            } else {
-                negativeE2EStat.registerSuccessfulEvent(-e2eLatency);
-            }
-            if (deliveryLatency >= 0) {
-                deliveryStat.registerSuccessfulEvent(deliveryLatency);
-            } else {
-                negativeDeliveryStat.registerSuccessfulEvent(-deliveryLatency);
-            }
-
-            prevDLSN = record.getDlsn();
-        }
-
-        @Override
-        public void onFailure(Throwable cause) {
-            scheduleReinitStream(streamIdx).map(new Function<Void, Void>() {
-                @Override
-                public Void apply(Void value) {
-                    prevDLSN = null;
-                    prevSequenceId = Long.MIN_VALUE;
-                    readLoop();
-                    return null;
-                }
-            });
-        }
-
-        void readLoop() {
-            if (!running) {
-                return;
-            }
-            logReaders[streamIdx].readBulk(10).addEventListener(this);
-        }
-
-        @Override
-        public void run() {
-            final DLSN dlsnToTruncate = prevDLSN;
-            if (null == dlsnToTruncate) {
-                return;
-            }
-            final Stopwatch stopwatch = Stopwatch.createStarted();
-            dlc.truncate(streamName, dlsnToTruncate).addEventListener(
-                    new FutureEventListener<Boolean>() {
-                        @Override
-                        public void onSuccess(Boolean value) {
-                            truncationStat.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MILLISECONDS));
-                        }
-
-                        @Override
-                        public void onFailure(Throwable cause) {
-                            truncationStat.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MILLISECONDS));
-                            LOG.error("Failed to truncate stream {} to {} : ",
-                                    new Object[]{streamName, dlsnToTruncate, cause});
-                        }
-                    });
-        }
-
-        @Override
-        public Number getDefaultValue() {
-            return Long.MIN_VALUE;
-        }
-
-        @Override
-        public synchronized Number getSample() {
-            return prevSequenceId;
-        }
-
-        void unregisterGauge() {
-            statsLogger.scope(streamName).unregisterGauge(gaugeLabel, this);
-        }
-    }
-
-    public ReaderWorker(DistributedLogConfiguration conf,
-                        URI uri,
-                        String streamPrefix,
-                        int startStreamId,
-                        int endStreamId,
-                        int readThreadPoolSize,
-                        List<String> serverSetPaths,
-                        List<String> finagleNames,
-                        int truncationIntervalInSeconds,
-                        boolean readFromHead, /* read from the earliest data of log */
-                        StatsReceiver statsReceiver,
-                        StatsLogger statsLogger) throws IOException {
-        checkArgument(startStreamId <= endStreamId);
-        this.streamPrefix = streamPrefix;
-        this.startStreamId = startStreamId;
-        this.endStreamId = endStreamId;
-        this.truncationIntervalInSeconds = truncationIntervalInSeconds;
-        this.readFromHead = readFromHead;
-        this.statsReceiver = statsReceiver;
-        this.statsLogger = statsLogger;
-        this.e2eStat = this.statsLogger.getOpStatsLogger("e2e");
-        this.negativeE2EStat = this.statsLogger.getOpStatsLogger("e2eNegative");
-        this.deliveryStat = this.statsLogger.getOpStatsLogger("delivery");
-        this.negativeDeliveryStat = this.statsLogger.getOpStatsLogger("deliveryNegative");
-        this.truncationStat = this.statsLogger.getOpStatsLogger("truncation");
-        this.invalidRecordsCounter = this.statsLogger.getCounter("invalid_records");
-        this.outOfOrderSequenceIdCounter = this.statsLogger.getCounter("out_of_order_seq_id");
-        this.executorService = Executors.newScheduledThreadPool(
-                readThreadPoolSize, new ThreadFactoryBuilder().setNameFormat("benchmark.reader-%d").build());
-        this.callbackExecutor = Executors.newFixedThreadPool(
-                Runtime.getRuntime().availableProcessors(),
-                new ThreadFactoryBuilder().setNameFormat("benchmark.reader-callback-%d").build());
-        this.finagleNames = finagleNames;
-        this.serverSets = createServerSets(serverSetPaths);
-
-        conf.setDeserializeRecordSetOnReads(false);
-
-        if (truncationIntervalInSeconds > 0 && (!finagleNames.isEmpty() || !serverSetPaths.isEmpty())) {
-            // Construct client for truncation
-            DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
-                    .clientId(ClientId$.MODULE$.apply("dlog_loadtest_reader"))
-                    .clientBuilder(ClientBuilder.get()
-                        .hostConnectionLimit(10)
-                        .hostConnectionCoresize(10)
-                        .tcpConnectTimeout(Duration$.MODULE$.fromSeconds(1))
-                        .requestTimeout(Duration$.MODULE$.fromSeconds(2)))
-                    .redirectBackoffStartMs(100)
-                    .redirectBackoffMaxMs(500)
-                    .requestTimeoutMs(2000)
-                    .statsReceiver(statsReceiver)
-                    .thriftmux(true)
-                    .name("reader");
-
-            if (serverSetPaths.isEmpty()) {
-                // Prepare finagle names
-                String local = finagleNames.get(0);
-                String[] remotes = new String[finagleNames.size() - 1];
-                finagleNames.subList(1, finagleNames.size()).toArray(remotes);
-
-                builder = builder.finagleNameStrs(local, remotes);
-                LOG.info("Initialized distributedlog client for truncation @ {}.", finagleNames);
-            } else if (serverSets.length != 0){
-                ServerSet local = this.serverSets[0].getServerSet();
-                ServerSet[] remotes = new ServerSet[this.serverSets.length - 1];
-                for (int i = 1; i < serverSets.length; i++) {
-                    remotes[i - 1] = serverSets[i].getServerSet();
-                }
-
-                builder = builder.serverSets(local, remotes);
-                LOG.info("Initialized distributedlog client for truncation @ {}.", serverSetPaths);
-            } else {
-                builder = builder.uri(uri);
-                LOG.info("Initialized distributedlog client for namespace {}", uri);
-            }
-            dlc = builder.build();
-        } else {
-            dlc = null;
-        }
-
-        // construct the factory
-        this.namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(uri)
-                .statsLogger(statsLogger.scope("dl"))
-                .build();
-        this.numStreams = endStreamId - startStreamId;
-        this.dlms = new DistributedLogManager[numStreams];
-        this.logReaders = new AsyncLogReader[numStreams];
-        final CountDownLatch latch = new CountDownLatch(numStreams);
-        for (int i = 0; i < numStreams; i++) {
-            final int idx = i;
-            executorService.submit(new Runnable() {
-                @Override
-                public void run() {
-                    reinitStream(idx).map(new Function<Void, Void>() {
-                        @Override
-                        public Void apply(Void value) {
-                            LOG.info("Initialized stream reader {}.", idx);
-                            latch.countDown();
-                            return null;
-                        }
-                    });
-                }
-            });
-        }
-        try {
-            latch.await();
-        } catch (InterruptedException e) {
-            throw new DLInterruptedException("Failed to intialize benchmark readers : ", e);
-        }
-        this.streamReaders = new StreamReader[numStreams];
-        for (int i = 0; i < numStreams; i++) {
-            streamReaders[i] = new StreamReader(i, statsLogger.scope("perstream"));
-            if (truncationIntervalInSeconds > 0) {
-                executorService.scheduleWithFixedDelay(streamReaders[i],
-                        truncationIntervalInSeconds, truncationIntervalInSeconds, TimeUnit.SECONDS);
-            }
-        }
-        LOG.info("Initialized benchmark reader on {} streams {} : [{} - {})",
-                 new Object[] { numStreams, streamPrefix, startStreamId, endStreamId });
-    }
-
-    protected DLZkServerSet[] createServerSets(List<String> serverSetPaths) {
-        DLZkServerSet[] serverSets = new DLZkServerSet[serverSetPaths.size()];
-        for (int i = 0; i < serverSets.length; i++) {
-            String serverSetPath = serverSetPaths.get(i);
-            serverSets[i] = DLZkServerSet.of(URI.create(serverSetPath), 60000);
-        }
-        return serverSets;
-    }
-
-    private Future<Void> reinitStream(int idx) {
-        Promise<Void> promise = new Promise<Void>();
-        reinitStream(idx, promise);
-        return promise;
-    }
-
-    private void reinitStream(int idx, Promise<Void> promise) {
-        int streamId = startStreamId + idx;
-        String streamName = String.format("%s_%d", streamPrefix, streamId);
-
-        if (logReaders[idx] != null) {
-            try {
-                FutureUtils.result(logReaders[idx].asyncClose());
-            } catch (IOException e) {
-                LOG.warn("Failed on closing stream reader {} : ", streamName, e);
-            }
-            logReaders[idx] = null;
-        }
-        if (dlms[idx] != null) {
-            try {
-                dlms[idx].close();
-            } catch (IOException e) {
-                LOG.warn("Failed on closing dlm {} : ", streamName, e);
-            }
-            dlms[idx] = null;
-        }
-
-        try {
-            dlms[idx] = namespace.openLog(streamName);
-        } catch (IOException ioe) {
-            LOG.error("Failed on creating dlm {} : ", streamName, ioe);
-            scheduleReinitStream(idx, promise);
-            return;
-        }
-        DLSN lastDLSN;
-        if (readFromHead) {
-            lastDLSN = DLSN.InitialDLSN;
-        } else {
-            try {
-                lastDLSN = dlms[idx].getLastDLSN();
-            } catch (IOException ioe) {
-                LOG.error("Failed on getting last dlsn from stream {} : ", streamName, ioe);
-                scheduleReinitStream(idx, promise);
-                return;
-            }
-        }
-        try {
-            logReaders[idx] = dlms[idx].getAsyncLogReader(lastDLSN);
-        } catch (IOException ioe) {
-            LOG.error("Failed on opening reader for stream {} starting from {} : ",
-                      new Object[] { streamName, lastDLSN, ioe });
-            scheduleReinitStream(idx, promise);
-            return;
-        }
-        LOG.info("Opened reader for stream {}, starting from {}.", streamName, lastDLSN);
-        promise.setValue(null);
-    }
-
-    Future<Void> scheduleReinitStream(int idx) {
-        Promise<Void> promise = new Promise<Void>();
-        scheduleReinitStream(idx, promise);
-        return promise;
-    }
-
-    void scheduleReinitStream(final int idx, final Promise<Void> promise) {
-        executorService.schedule(new Runnable() {
-            @Override
-            public void run() {
-                reinitStream(idx, promise);
-            }
-        }, BACKOFF_MS, TimeUnit.MILLISECONDS);
-    }
-
-    @Override
-    public void close() throws IOException {
-        this.running = false;
-        for (AsyncLogReader reader : logReaders) {
-            if (null != reader) {
-                FutureUtils.result(reader.asyncClose());
-            }
-        }
-        for (DistributedLogManager dlm : dlms) {
-            if (null != dlm) {
-                dlm.close();
-            }
-        }
-        namespace.close();
-        SchedulerUtils.shutdownScheduler(executorService, 2, TimeUnit.MINUTES);
-        SchedulerUtils.shutdownScheduler(callbackExecutor, 2, TimeUnit.MINUTES);
-        if (this.dlc != null) {
-            this.dlc.close();
-        }
-        for (DLZkServerSet serverSet: serverSets) {
-            serverSet.close();
-        }
-        // Unregister gauges to prevent GC spirals
-        for (StreamReader sr : streamReaders) {
-            sr.unregisterGauge();
-        }
-    }
-
-    @Override
-    public void run() {
-        LOG.info("Starting reader (prefix = {}, numStreams = {}).",
-                 streamPrefix, numStreams);
-        for (StreamReader sr : streamReaders) {
-            sr.readLoop();
-        }
-    }
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Utils.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Utils.java
deleted file mode 100644
index f5c32db..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Utils.java
+++ /dev/null
@@ -1,57 +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 com.twitter.distributedlog.benchmark;
-
-import com.twitter.distributedlog.benchmark.thrift.Message;
-import java.nio.ByteBuffer;
-import java.util.Random;
-import org.apache.thrift.TException;
-import org.apache.thrift.TSerializer;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.apache.thrift.transport.TMemoryInputTransport;
-
-/**
- * Utils for generating and parsing messages.
- */
-public class Utils {
-
-    static final Random RAND = new Random(System.currentTimeMillis());
-    static final ThreadLocal<TSerializer> MSG_SERIALIZER =
-            new ThreadLocal<TSerializer>() {
-                @Override
-                public TSerializer initialValue() {
-                    return new TSerializer(new TBinaryProtocol.Factory());
-                }
-            };
-
-    public static byte[] generateMessage(long requestMillis, int payLoadSize) throws TException {
-        byte[] payload = new byte[payLoadSize];
-        RAND.nextBytes(payload);
-        Message msg = new Message(requestMillis, ByteBuffer.wrap(payload));
-        return MSG_SERIALIZER.get().serialize(msg);
-    }
-
-    public static Message parseMessage(byte[] data) throws TException {
-        Message msg = new Message();
-        TMemoryInputTransport transport = new TMemoryInputTransport(data);
-        TBinaryProtocol protocol = new TBinaryProtocol(transport);
-        msg.read(protocol);
-        return msg;
-    }
-
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Worker.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Worker.java
deleted file mode 100644
index 6c60034..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Worker.java
+++ /dev/null
@@ -1,26 +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 com.twitter.distributedlog.benchmark;
-
-import java.io.Closeable;
-
-/**
- * Worker to run benchmark.
- */
-public interface Worker extends Closeable, Runnable {
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/WriterWorker.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/WriterWorker.java
deleted file mode 100644
index dc5a6e2..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/WriterWorker.java
+++ /dev/null
@@ -1,387 +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 com.twitter.distributedlog.benchmark;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.benchmark.utils.ShiftableRateLimiter;
-import com.twitter.distributedlog.client.DistributedLogMultiStreamWriter;
-import com.twitter.distributedlog.client.serverset.DLZkServerSet;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.finagle.thrift.ClientId;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Duration$;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import java.io.IOException;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Random;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.thrift.TException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Benchmark for distributedlog proxy client.
- */
-public class WriterWorker implements Worker {
-
-    static final Logger LOG = LoggerFactory.getLogger(WriterWorker.class);
-
-    final String streamPrefix;
-    final int startStreamId;
-    final int endStreamId;
-    final int writeConcurrency;
-    final int messageSizeBytes;
-    final int hostConnectionCoreSize;
-    final int hostConnectionLimit;
-    final ExecutorService executorService;
-    final ShiftableRateLimiter rateLimiter;
-    final URI dlUri;
-    final DLZkServerSet[] serverSets;
-    final List<String> finagleNames;
-    final Random random;
-    final List<String> streamNames;
-    final int numStreams;
-    final int batchSize;
-    final boolean thriftmux;
-    final boolean handshakeWithClientInfo;
-    final int sendBufferSize;
-    final int recvBufferSize;
-    final boolean enableBatching;
-    final int batchBufferSize;
-    final int batchFlushIntervalMicros;
-    private final String routingServiceFinagleName;
-
-    volatile boolean running = true;
-
-    final StatsReceiver statsReceiver;
-    final StatsLogger statsLogger;
-    final OpStatsLogger requestStat;
-    final StatsLogger exceptionsLogger;
-    final StatsLogger dlErrorCodeLogger;
-
-    // callback thread
-    final ExecutorService executor;
-
-    public WriterWorker(String streamPrefix,
-                        URI uri,
-                        int startStreamId,
-                        int endStreamId,
-                        ShiftableRateLimiter rateLimiter,
-                        int writeConcurrency,
-                        int messageSizeBytes,
-                        int batchSize,
-                        int hostConnectionCoreSize,
-                        int hostConnectionLimit,
-                        List<String> serverSetPaths,
-                        List<String> finagleNames,
-                        StatsReceiver statsReceiver,
-                        StatsLogger statsLogger,
-                        boolean thriftmux,
-                        boolean handshakeWithClientInfo,
-                        int sendBufferSize,
-                        int recvBufferSize,
-                        boolean enableBatching,
-                        int batchBufferSize,
-                        int batchFlushIntervalMicros,
-                        String routingServiceFinagleName) {
-        checkArgument(startStreamId <= endStreamId);
-        checkArgument(!finagleNames.isEmpty() || !serverSetPaths.isEmpty());
-        this.streamPrefix = streamPrefix;
-        this.dlUri = uri;
-        this.startStreamId = startStreamId;
-        this.endStreamId = endStreamId;
-        this.rateLimiter = rateLimiter;
-        this.writeConcurrency = writeConcurrency;
-        this.messageSizeBytes = messageSizeBytes;
-        this.statsReceiver = statsReceiver;
-        this.statsLogger = statsLogger;
-        this.requestStat = this.statsLogger.getOpStatsLogger("requests");
-        this.exceptionsLogger = statsLogger.scope("exceptions");
-        this.dlErrorCodeLogger = statsLogger.scope("dl_error_code");
-        this.executorService = Executors.newCachedThreadPool();
-        this.random = new Random(System.currentTimeMillis());
-        this.batchSize = batchSize;
-        this.hostConnectionCoreSize = hostConnectionCoreSize;
-        this.hostConnectionLimit = hostConnectionLimit;
-        this.thriftmux = thriftmux;
-        this.handshakeWithClientInfo = handshakeWithClientInfo;
-        this.sendBufferSize = sendBufferSize;
-        this.recvBufferSize = recvBufferSize;
-        this.enableBatching = enableBatching;
-        this.batchBufferSize = batchBufferSize;
-        this.batchFlushIntervalMicros = batchFlushIntervalMicros;
-        this.finagleNames = finagleNames;
-        this.serverSets = createServerSets(serverSetPaths);
-        this.executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
-        this.routingServiceFinagleName = routingServiceFinagleName;
-
-        // Streams
-        streamNames = new ArrayList<String>(endStreamId - startStreamId);
-        for (int i = startStreamId; i < endStreamId; i++) {
-            streamNames.add(String.format("%s_%d", streamPrefix, i));
-        }
-        numStreams = streamNames.size();
-        LOG.info("Writing to {} streams : {}", numStreams, streamNames);
-    }
-
-    protected DLZkServerSet[] createServerSets(List<String> serverSetPaths) {
-        DLZkServerSet[] serverSets = new DLZkServerSet[serverSetPaths.size()];
-        for (int i = 0; i < serverSets.length; i++) {
-            String serverSetPath = serverSetPaths.get(i);
-            serverSets[i] = DLZkServerSet.of(URI.create(serverSetPath), 60000);
-        }
-        return serverSets;
-    }
-
-    @Override
-    public void close() throws IOException {
-        this.running = false;
-        SchedulerUtils.shutdownScheduler(this.executorService, 2, TimeUnit.MINUTES);
-        for (DLZkServerSet serverSet: serverSets) {
-            serverSet.close();
-        }
-    }
-
-    private DistributedLogClient buildDlogClient() {
-        ClientBuilder clientBuilder = ClientBuilder.get()
-            .hostConnectionLimit(hostConnectionLimit)
-            .hostConnectionCoresize(hostConnectionCoreSize)
-            .tcpConnectTimeout(Duration$.MODULE$.fromMilliseconds(200))
-            .connectTimeout(Duration$.MODULE$.fromMilliseconds(200))
-            .requestTimeout(Duration$.MODULE$.fromSeconds(10))
-            .sendBufferSize(sendBufferSize)
-            .recvBufferSize(recvBufferSize);
-
-        ClientId clientId = ClientId$.MODULE$.apply("dlog_loadtest_writer");
-
-        DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
-            .clientId(clientId)
-            .clientBuilder(clientBuilder)
-            .thriftmux(thriftmux)
-            .redirectBackoffStartMs(100)
-            .redirectBackoffMaxMs(500)
-            .requestTimeoutMs(10000)
-            .statsReceiver(statsReceiver)
-            .streamNameRegex("^" + streamPrefix + "_[0-9]+$")
-            .handshakeWithClientInfo(handshakeWithClientInfo)
-            .periodicHandshakeIntervalMs(TimeUnit.SECONDS.toMillis(30))
-            .periodicOwnershipSyncIntervalMs(TimeUnit.MINUTES.toMillis(5))
-            .periodicDumpOwnershipCache(true)
-            .handshakeTracing(true)
-            .serverRoutingServiceFinagleNameStr(routingServiceFinagleName)
-            .name("writer");
-
-        if (!finagleNames.isEmpty()) {
-            String local = finagleNames.get(0);
-            String[] remotes = new String[finagleNames.size() - 1];
-            finagleNames.subList(1, finagleNames.size()).toArray(remotes);
-
-            builder = builder.finagleNameStrs(local, remotes);
-        } else if (serverSets.length != 0){
-            ServerSet local = serverSets[0].getServerSet();
-            ServerSet[] remotes = new ServerSet[serverSets.length - 1];
-            for (int i = 1; i < serverSets.length; i++) {
-                remotes[i - 1] = serverSets[i].getServerSet();
-            }
-            builder = builder.serverSets(local, remotes);
-        } else {
-            builder = builder.uri(dlUri);
-        }
-
-        return builder.build();
-    }
-
-    ByteBuffer buildBuffer(long requestMillis, int messageSizeBytes) {
-        ByteBuffer data;
-        try {
-            data = ByteBuffer.wrap(Utils.generateMessage(requestMillis, messageSizeBytes));
-            return data;
-        } catch (TException e) {
-            LOG.error("Error generating message : ", e);
-            return null;
-        }
-    }
-
-    List<ByteBuffer> buildBufferList(int batchSize, long requestMillis, int messageSizeBytes) {
-        ArrayList<ByteBuffer> bufferList = new ArrayList<ByteBuffer>(batchSize);
-        for (int i = 0; i < batchSize; i++) {
-            ByteBuffer buf = buildBuffer(requestMillis, messageSizeBytes);
-            if (null == buf) {
-                return null;
-            }
-            bufferList.add(buf);
-        }
-        return bufferList;
-    }
-
-    class TimedRequestHandler implements FutureEventListener<DLSN>, Runnable {
-        final String streamName;
-        final long requestMillis;
-        DLSN dlsn = null;
-        Throwable cause = null;
-
-        TimedRequestHandler(String streamName,
-                            long requestMillis) {
-            this.streamName = streamName;
-            this.requestMillis = requestMillis;
-        }
-        @Override
-        public void onSuccess(DLSN value) {
-            dlsn = value;
-            executor.submit(this);
-        }
-        @Override
-        public void onFailure(Throwable cause) {
-            this.cause = cause;
-            executor.submit(this);
-        }
-
-        @Override
-        public void run() {
-            if (null != dlsn) {
-                requestStat.registerSuccessfulEvent(System.currentTimeMillis() - requestMillis);
-            } else {
-                LOG.error("Failed to publish to {} : ", streamName, cause);
-                requestStat.registerFailedEvent(System.currentTimeMillis() - requestMillis);
-                exceptionsLogger.getCounter(cause.getClass().getName()).inc();
-                if (cause instanceof DLException) {
-                    DLException dle = (DLException) cause;
-                    dlErrorCodeLogger.getCounter(dle.getCode().toString()).inc();
-                }
-            }
-        }
-    }
-
-    class Writer implements Runnable {
-
-        final int idx;
-        final DistributedLogClient dlc;
-        DistributedLogMultiStreamWriter writer = null;
-        final ShiftableRateLimiter limiter;
-
-        Writer(int idx) {
-            this.idx = idx;
-            this.dlc = buildDlogClient();
-            if (enableBatching) {
-                writer = DistributedLogMultiStreamWriter.newBuilder()
-                        .client(this.dlc)
-                        .streams(streamNames)
-                        .compressionCodec(CompressionCodec.Type.NONE)
-                        .flushIntervalMicros(batchFlushIntervalMicros)
-                        .bufferSize(batchBufferSize)
-                        .firstSpeculativeTimeoutMs(9000)
-                        .maxSpeculativeTimeoutMs(9000)
-                        .requestTimeoutMs(10000)
-                        .speculativeBackoffMultiplier(2)
-                        .build();
-            }
-            this.limiter = rateLimiter.duplicate();
-        }
-
-        @Override
-        public void run() {
-            LOG.info("Started writer {}.", idx);
-            while (running) {
-                this.limiter.getLimiter().acquire();
-                final String streamName = streamNames.get(random.nextInt(numStreams));
-                final long requestMillis = System.currentTimeMillis();
-                final ByteBuffer data = buildBuffer(requestMillis, messageSizeBytes);
-                if (null == data) {
-                    break;
-                }
-                if (null != writer) {
-                    writer.write(data).addEventListener(
-                            new TimedRequestHandler(streamName, requestMillis));
-                } else {
-                    dlc.write(streamName, data).addEventListener(
-                            new TimedRequestHandler(streamName, requestMillis));
-                }
-            }
-            if (null != writer) {
-                writer.close();
-            }
-            dlc.close();
-        }
-    }
-
-    class BulkWriter implements Runnable {
-
-        final int idx;
-        final DistributedLogClient dlc;
-
-        BulkWriter(int idx) {
-            this.idx = idx;
-            this.dlc = buildDlogClient();
-        }
-
-        @Override
-        public void run() {
-            LOG.info("Started writer {}.", idx);
-            while (running) {
-                rateLimiter.getLimiter().acquire(batchSize);
-                String streamName = streamNames.get(random.nextInt(numStreams));
-                final long requestMillis = System.currentTimeMillis();
-                final List<ByteBuffer> data = buildBufferList(batchSize, requestMillis, messageSizeBytes);
-                if (null == data) {
-                    break;
-                }
-                List<Future<DLSN>> results = dlc.writeBulk(streamName, data);
-                for (Future<DLSN> result : results) {
-                    result.addEventListener(new TimedRequestHandler(streamName, requestMillis));
-                }
-            }
-            dlc.close();
-        }
-    }
-
-    @Override
-    public void run() {
-        LOG.info("Starting writer (concurrency = {}, prefix = {}, batchSize = {})",
-                 new Object[] { writeConcurrency, streamPrefix, batchSize });
-        try {
-            for (int i = 0; i < writeConcurrency; i++) {
-                Runnable writer = null;
-                if (batchSize > 0) {
-                    writer = new BulkWriter(i);
-                } else {
-                    writer = new Writer(i);
-                }
-                executorService.submit(writer);
-            }
-        } catch (Throwable t) {
-            LOG.error("Unhandled exception caught", t);
-        }
-    }
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/package-info.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/package-info.java
deleted file mode 100644
index 052a661..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Benchmarks for distributedlog.
- */
-package com.twitter.distributedlog.benchmark;
\ No newline at end of file
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AbstractReaderBenchmark.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AbstractReaderBenchmark.java
deleted file mode 100644
index 4d436ee..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AbstractReaderBenchmark.java
+++ /dev/null
@@ -1,70 +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 com.twitter.distributedlog.benchmark.stream;
-
-import com.twitter.distributedlog.DistributedLogConstants;
-import org.apache.commons.cli.CommandLine;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-abstract class AbstractReaderBenchmark extends StreamBenchmark {
-
-    private static final Logger logger = LoggerFactory.getLogger(SyncReaderBenchmark.class);
-
-    protected ReadMode readMode = ReadMode.LATEST;
-    protected long fromTxId = DistributedLogConstants.INVALID_TXID;
-    protected long rewindMs = 0L;
-    protected int batchSize = 1;
-
-    protected AbstractReaderBenchmark() {
-        options.addOption("t", "tx-id", true,
-            "Transaction ID to start read from when reading in mode 'position'");
-        options.addOption("r", "rewind", true,
-            "Time to rewind back to read from when reading in mode 'rewind' (in milliseconds)");
-        options.addOption("m", "mode", true,
-            "Read Mode : [oldest, latest, rewind, position]");
-        options.addOption("b", "batch-size", true, "Read batch size");
-    }
-
-    @Override
-    protected void parseCommandLine(CommandLine cmdline) {
-        if (cmdline.hasOption("m")) {
-            String mode = cmdline.getOptionValue("m");
-            try {
-                readMode = ReadMode.valueOf(mode.toUpperCase());
-            } catch (IllegalArgumentException iae) {
-                logger.error("Invalid read mode {}.", mode);
-                printUsage();
-                System.exit(0);
-            }
-        } else {
-            printUsage();
-            System.exit(0);
-        }
-        if (cmdline.hasOption("t")) {
-            fromTxId = Long.parseLong(cmdline.getOptionValue("t"));
-        }
-        if (cmdline.hasOption("r")) {
-            rewindMs = Long.parseLong(cmdline.getOptionValue("r"));
-        }
-        if (cmdline.hasOption("b")) {
-            batchSize = Integer.parseInt(cmdline.getOptionValue("b"));
-        }
-        logger.info("Start reading from transaction id {}, rewind {} ms.", fromTxId, rewindMs);
-    }
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AsyncReaderBenchmark.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
deleted file mode 100644
index 86acdb6..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
+++ /dev/null
@@ -1,158 +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 com.twitter.distributedlog.benchmark.stream;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.AsyncLogReader;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.util.FutureUtils;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-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;
-
-/**
- * Benchmark on {@link com.twitter.distributedlog.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) {
-        DistributedLogManager dlm = null;
-        while (null == dlm) {
-            try {
-                dlm = namespace.openLog(streamName);
-            } catch (IOException ioe) {
-                logger.warn("Failed to create dlm for stream {} : ", streamName, ioe);
-            }
-            if (null == dlm) {
-                try {
-                    TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
-                } catch (InterruptedException e) {
-                    logger.warn("Interrupted from sleep while creating dlm for stream {} : ",
-                        streamName, e);
-                }
-            }
-        }
-        logger.info("Created dlm for stream {}.", streamName);
-
-        // Stats
-        OpStatsLogger openReaderStats = statsLogger.getOpStatsLogger("open_reader");
-        OpStatsLogger blockingReadStats = statsLogger.getOpStatsLogger("blocking_read");
-        Counter readCounter = statsLogger.getCounter("reads");
-
-        AsyncLogReader reader = null;
-        DLSN lastDLSN = null;
-        Long lastTxId = null;
-        while (null == reader) {
-            // initialize the last txid
-            if (null == lastTxId) {
-                switch (readMode) {
-                    case OLDEST:
-                        lastTxId = 0L;
-                        lastDLSN = DLSN.InitialDLSN;
-                        break;
-                    case LATEST:
-                        lastTxId = Long.MAX_VALUE;
-                        try {
-                            lastDLSN = dlm.getLastDLSN();
-                        } catch (IOException ioe) {
-                            continue;
-                        }
-                        break;
-                    case REWIND:
-                        lastTxId = System.currentTimeMillis() - rewindMs;
-                        lastDLSN = null;
-                        break;
-                    case POSITION:
-                        lastTxId = fromTxId;
-                        lastDLSN = null;
-                        break;
-                    default:
-                        logger.warn("Unsupported mode {}", readMode);
-                        printUsage();
-                        System.exit(0);
-                        break;
-                }
-                logger.info("Reading from transaction id = {}, dlsn = {}", lastTxId, lastDLSN);
-            }
-            // Open the reader
-            Stopwatch stopwatch = Stopwatch.createStarted();
-            try {
-                if (null == lastDLSN) {
-                    reader = FutureUtils.result(dlm.openAsyncLogReader(lastTxId));
-                } else {
-                    reader = FutureUtils.result(dlm.openAsyncLogReader(lastDLSN));
-                }
-                long elapsedMs = stopwatch.elapsed(TimeUnit.MICROSECONDS);
-                openReaderStats.registerSuccessfulEvent(elapsedMs);
-                logger.info("It took {} ms to position the reader to transaction id = {}, dlsn = {}",
-                        lastTxId, lastDLSN);
-            } catch (IOException 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 });
-            }
-            if (null == reader) {
-                try {
-                    TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
-                } catch (InterruptedException e) {
-                    logger.warn("Interrupted from sleep after reader was reassigned null for stream {} : ",
-                        streamName, e);
-                }
-                continue;
-            }
-            List<LogRecordWithDLSN> records;
-            stopwatch = Stopwatch.createUnstarted();
-            while (true) {
-                try {
-                    stopwatch.start();
-                    records = FutureUtils.result(reader.readBulk(batchSize));
-                    long elapsedMicros = stopwatch.stop().elapsed(TimeUnit.MICROSECONDS);
-                    blockingReadStats.registerSuccessfulEvent(elapsedMicros);
-                    if (!records.isEmpty()) {
-                        readCounter.add(records.size());
-                        LogRecordWithDLSN lastRecord = records.get(records.size() - 1);
-                        lastTxId = lastRecord.getTransactionId();
-                        lastDLSN = lastRecord.getDlsn();
-                    }
-                    stopwatch.reset();
-                } catch (IOException e) {
-                    logger.warn("Encountered reading record from stream {} : ", streamName, e);
-                    reader = null;
-                    break;
-                }
-            }
-            try {
-                TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
-            } catch (InterruptedException e) {
-                logger.warn("Interrupted from sleep while creating reader for stream {} : ",
-                    streamName, e);
-            }
-        }
-    }
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerBatchReader.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerBatchReader.java
deleted file mode 100644
index 6a11469..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerBatchReader.java
+++ /dev/null
@@ -1,82 +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 com.twitter.distributedlog.benchmark.stream;
-
-import java.util.Enumeration;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Read ledgers in batches.
- */
-public class LedgerBatchReader implements Runnable {
-
-    private static final Logger logger = LoggerFactory.getLogger(LedgerBatchReader.class);
-
-    private final LedgerHandle lh;
-    private final ReadEntryListener readEntryListener;
-    private final int batchSize;
-
-    public LedgerBatchReader(LedgerHandle lh,
-                             ReadEntryListener readEntryListener,
-                             int batchSize) {
-        this.lh = lh;
-        this.batchSize = batchSize;
-        this.readEntryListener = readEntryListener;
-    }
-
-    @Override
-    public void run() {
-        long lac = lh.getLastAddConfirmed();
-
-        long entryId = 0L;
-
-        while (entryId <= lac) {
-            long startEntryId = entryId;
-            long endEntryId = Math.min(startEntryId + batchSize - 1, lac);
-
-            Enumeration<LedgerEntry> entries = null;
-            while (null == entries) {
-                try {
-                    entries = lh.readEntries(startEntryId, endEntryId);
-                } catch (BKException bke) {
-                    logger.error("Encountered exceptions on reading [ {} - {} ] ",
-                            new Object[] { startEntryId, endEntryId, bke });
-                } catch (InterruptedException ie) {
-                    Thread.currentThread().interrupt();
-                    break;
-                }
-            }
-            if (null == entries) {
-                break;
-            }
-
-            while (entries.hasMoreElements()) {
-                LedgerEntry entry = entries.nextElement();
-                readEntryListener.onEntryComplete(BKException.Code.OK, lh, entry, null);
-            }
-
-            entryId = endEntryId + 1;
-        }
-
-    }
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerReadBenchmark.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerReadBenchmark.java
deleted file mode 100644
index 072c3ef..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerReadBenchmark.java
+++ /dev/null
@@ -1,151 +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 com.twitter.distributedlog.benchmark.stream;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.BookKeeperClientBuilder;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Benchmark ledger reading.
- */
-public class LedgerReadBenchmark extends AbstractReaderBenchmark {
-
-    private static final Logger logger = LoggerFactory.getLogger(AsyncReaderBenchmark.class);
-
-    @Override
-    protected void benchmark(DistributedLogNamespace namespace, String logName, StatsLogger statsLogger) {
-        DistributedLogManager dlm = null;
-        while (null == dlm) {
-            try {
-                dlm = namespace.openLog(streamName);
-            } catch (IOException ioe) {
-                logger.warn("Failed to create dlm for stream {} : ", streamName, ioe);
-            }
-            if (null == dlm) {
-                try {
-                    TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
-                } catch (InterruptedException e) {
-                    logger.warn("Interrupted from sleep while creating dlm for stream {} : ",
-                        streamName, e);
-                }
-            }
-        }
-        logger.info("Created dlm for stream {}.", streamName);
-
-        List<LogSegmentMetadata> segments = null;
-        while (null == segments) {
-            try {
-                segments = dlm.getLogSegments();
-            } catch (IOException ioe) {
-                logger.warn("Failed to get log segments for stream {} : ", streamName, ioe);
-            }
-            if (null == segments) {
-                try {
-                    TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
-                } catch (InterruptedException e) {
-                    logger.warn("Interrupted from sleep while geting log segments for stream {} : ",
-                        streamName, e);
-                }
-            }
-        }
-
-        final Counter readCounter = statsLogger.getCounter("reads");
-
-        logger.info("Reading from log segments : {}", segments);
-
-        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
-                .uri(uri)
-                .name("benchmark-zkc")
-                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                .zkAclId(null)
-                .build();
-        BKDLConfig bkdlConfig;
-        try {
-            bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
-        } catch (IOException e) {
-            return;
-        }
-
-        BookKeeper bk;
-        try {
-            bk = BookKeeperClientBuilder.newBuilder()
-                    .name("benchmark-bkc")
-                    .dlConfig(conf)
-                    .zkServers(bkdlConfig.getBkZkServersForReader())
-                    .ledgersPath(bkdlConfig.getBkLedgersPath())
-                    .build()
-                    .get();
-        } catch (IOException e) {
-            return;
-        }
-
-        final int readConcurrency = conf.getInt("ledger_read_concurrency", 1000);
-        boolean streamRead = conf.getBoolean("ledger_stream_read", true);
-        try {
-            for (LogSegmentMetadata segment : segments) {
-                Stopwatch stopwatch = Stopwatch.createStarted();
-                long lid = segment.getLogSegmentId();
-                LedgerHandle lh = bk.openLedgerNoRecovery(
-                        lid, BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
-                logger.info("It took {} ms to open log segment {}",
-                    new Object[] { stopwatch.elapsed(TimeUnit.MILLISECONDS), (lh.getLastAddConfirmed() + 1), segment });
-                stopwatch.reset().start();
-                Runnable reader;
-                if (streamRead) {
-                    reader = new LedgerStreamReader(lh, new BookkeeperInternalCallbacks.ReadEntryListener() {
-                        @Override
-                        public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object ctx) {
-                            readCounter.inc();
-                        }
-                    }, readConcurrency);
-                } else {
-                    reader = new LedgerStreamReader(lh, new BookkeeperInternalCallbacks.ReadEntryListener() {
-                        @Override
-                        public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object ctx) {
-                            readCounter.inc();
-                        }
-                    }, readConcurrency);
-                }
-                reader.run();
-                logger.info("It took {} ms to complete reading {} entries from log segment {}",
-                    new Object[] { stopwatch.elapsed(TimeUnit.MILLISECONDS), (lh.getLastAddConfirmed() + 1), segment });
-            }
-        } catch (Exception e) {
-            logger.error("Error on reading bk ", e);
-        }
-    }
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerStreamReader.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerStreamReader.java
deleted file mode 100644
index e542af7..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerStreamReader.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 com.twitter.distributedlog.benchmark.stream;
-
-import java.util.Enumeration;
-import java.util.concurrent.ConcurrentLinkedQueue;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicLong;
-import org.apache.bookkeeper.client.AsyncCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Reading ledger in a streaming way.
- */
-public class LedgerStreamReader implements Runnable {
-
-    private static final Logger logger = LoggerFactory.getLogger(LedgerStreamReader.class);
-
-    class PendingReadRequest implements AsyncCallback.ReadCallback {
-
-        final long entryId;
-        boolean isDone = false;
-        int rc;
-        LedgerEntry entry = null;
-
-        PendingReadRequest(long entryId) {
-            this.entryId = entryId;
-        }
-
-        void read() {
-            lh.asyncReadEntries(entryId, entryId, this, null);
-        }
-
-        void complete(ReadEntryListener listener) {
-            listener.onEntryComplete(rc, lh, entry, null);
-        }
-
-        @Override
-        public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> enumeration, Object ctx) {
-            this.rc = rc;
-            if (BKException.Code.OK == rc && enumeration.hasMoreElements()) {
-                entry = enumeration.nextElement();
-            } else {
-                entry = null;
-            }
-            isDone = true;
-            // construct a new read request
-            long nextEntry = nextReadEntry.getAndIncrement();
-            if (nextEntry <= lac) {
-                PendingReadRequest nextRead =
-                        new PendingReadRequest(nextEntry);
-                pendingReads.add(nextRead);
-                nextRead.read();
-            }
-            triggerCallbacks();
-        }
-    }
-
-    private final LedgerHandle lh;
-    private final long lac;
-    private final ReadEntryListener readEntryListener;
-    private final int concurrency;
-    private final AtomicLong nextReadEntry = new AtomicLong(0);
-    private final CountDownLatch done = new CountDownLatch(1);
-    private final ConcurrentLinkedQueue<PendingReadRequest> pendingReads =
-            new ConcurrentLinkedQueue<PendingReadRequest>();
-
-    public LedgerStreamReader(LedgerHandle lh,
-                              ReadEntryListener readEntryListener,
-                              int concurrency) {
-        this.lh = lh;
-        this.lac = lh.getLastAddConfirmed();
-        this.readEntryListener = readEntryListener;
-        this.concurrency = concurrency;
-        for (int i = 0; i < concurrency; i++) {
-            long entryId = nextReadEntry.getAndIncrement();
-            if (entryId > lac) {
-                break;
-            }
-            PendingReadRequest request = new PendingReadRequest(entryId);
-            pendingReads.add(request);
-            request.read();
-        }
-        if (pendingReads.isEmpty()) {
-            done.countDown();
-        }
-    }
-
-    synchronized void triggerCallbacks() {
-        PendingReadRequest request;
-        while ((request = pendingReads.peek()) != null) {
-            if (!request.isDone) {
-                break;
-            }
-            pendingReads.remove();
-            request.complete(readEntryListener);
-        }
-        if (pendingReads.isEmpty()) {
-            done.countDown();
-        }
-    }
-
-    @Override
-    public void run() {
-        try {
-            done.await();
-        } catch (InterruptedException e) {
-            logger.info("Interrupted on stream reading ledger {} : ", lh.getId(), e);
-        }
-    }
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/ReadMode.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/ReadMode.java
deleted file mode 100644
index 280c9db..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/ReadMode.java
+++ /dev/null
@@ -1,28 +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 com.twitter.distributedlog.benchmark.stream;
-
-/**
- * The read mode for streaming read benchmark.
- */
-public enum ReadMode {
-    OLDEST,
-    LATEST,
-    REWIND,
-    POSITION
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/StreamBenchmark.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/StreamBenchmark.java
deleted file mode 100644
index 1eff65a..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/StreamBenchmark.java
+++ /dev/null
@@ -1,138 +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 com.twitter.distributedlog.benchmark.stream;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import java.io.File;
-import java.net.URI;
-import org.apache.bookkeeper.stats.NullStatsProvider;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.stats.StatsProvider;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.cli.BasicParser;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Benchmark Streams.
- */
-public abstract class StreamBenchmark {
-
-    private static final Logger logger = LoggerFactory.getLogger(StreamBenchmark.class);
-
-    private static final String USAGE = "StreamBenchmark <benchmark-class> [options]";
-
-    protected final Options options = new Options();
-    protected URI uri;
-    protected DistributedLogConfiguration conf;
-    protected StatsProvider statsProvider;
-    protected String streamName;
-
-    protected StreamBenchmark() {
-        options.addOption("c", "conf", true, "Configuration File");
-        options.addOption("u", "uri", true, "DistributedLog URI");
-        options.addOption("p", "stats-provider", true, "Stats Provider");
-        options.addOption("s", "stream", true, "Stream Name");
-        options.addOption("h", "help", false, "Print usage.");
-    }
-
-    protected Options getOptions() {
-        return options;
-    }
-
-    protected void printUsage() {
-        HelpFormatter hf = new HelpFormatter();
-        hf.printHelp(USAGE, options);
-    }
-
-    protected void parseCommandLine(String[] args)
-            throws Exception {
-        BasicParser parser = new BasicParser();
-        CommandLine cmdline = parser.parse(options, args);
-        if (cmdline.hasOption("h")) {
-            printUsage();
-            System.exit(0);
-        }
-        if (cmdline.hasOption("u")) {
-            this.uri = URI.create(cmdline.getOptionValue("u"));
-        } else {
-            printUsage();
-            System.exit(0);
-        }
-        this.conf = new DistributedLogConfiguration();
-        if (cmdline.hasOption("c")) {
-            String configFile = cmdline.getOptionValue("c");
-            this.conf.loadConf(new File(configFile).toURI().toURL());
-        }
-        if (cmdline.hasOption("p")) {
-            statsProvider = ReflectionUtils.newInstance(cmdline.getOptionValue("p"), StatsProvider.class);
-        } else {
-            statsProvider = new NullStatsProvider();
-        }
-        if (cmdline.hasOption("s")) {
-            this.streamName = cmdline.getOptionValue("s");
-        } else {
-            printUsage();
-            System.exit(0);
-        }
-        parseCommandLine(cmdline);
-    }
-
-    protected abstract void parseCommandLine(CommandLine cmdline);
-
-    protected void run(String[] args) throws Exception {
-        logger.info("Parsing arguments for benchmark : {}", args);
-        // parse command line
-        parseCommandLine(args);
-        statsProvider.start(conf);
-        // run the benchmark
-        StatsLogger statsLogger = statsProvider.getStatsLogger("dl");
-        DistributedLogNamespace namespace =
-                DistributedLogNamespaceBuilder.newBuilder()
-                        .conf(conf)
-                        .uri(uri)
-                        .statsLogger(statsLogger)
-                        .build();
-        try {
-            benchmark(namespace, streamName, statsProvider.getStatsLogger("benchmark"));
-        } finally {
-            namespace.close();
-            statsProvider.stop();
-        }
-    }
-
-    protected abstract void benchmark(DistributedLogNamespace namespace,
-                                      String logName,
-                                      StatsLogger statsLogger);
-
-    public static void main(String[] args) throws Exception {
-        if (args.length <= 0) {
-            System.err.println(USAGE);
-            return;
-        }
-        String benchmarkClassName = args[0];
-        StreamBenchmark benchmark = ReflectionUtils.newInstance(
-                benchmarkClassName, StreamBenchmark.class);
-        benchmark.run(args);
-    }
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/SyncReaderBenchmark.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/SyncReaderBenchmark.java
deleted file mode 100644
index 122c8ef..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/SyncReaderBenchmark.java
+++ /dev/null
@@ -1,164 +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 com.twitter.distributedlog.benchmark.stream;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogReader;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-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;
-
-/**
- * Benchmark on {@link com.twitter.distributedlog.LogReader} reading from a stream.
- */
-public class SyncReaderBenchmark extends AbstractReaderBenchmark {
-
-    private static final Logger logger = LoggerFactory.getLogger(SyncReaderBenchmark.class);
-
-    public SyncReaderBenchmark() {}
-
-    @Override
-    protected void benchmark(DistributedLogNamespace namespace, String streamName, StatsLogger statsLogger) {
-        DistributedLogManager dlm = null;
-        while (null == dlm) {
-            try {
-                dlm = namespace.openLog(streamName);
-            } catch (IOException ioe) {
-                logger.warn("Failed to create dlm for stream {} : ", streamName, ioe);
-            }
-            if (null == dlm) {
-                try {
-                    TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
-                } catch (InterruptedException e) {
-                    logger.warn("Interrupted from sleep while creating dlm for stream {} : ",
-                        streamName, e);
-                }
-            }
-        }
-        OpStatsLogger openReaderStats = statsLogger.getOpStatsLogger("open_reader");
-        OpStatsLogger nonBlockingReadStats = statsLogger.getOpStatsLogger("non_blocking_read");
-        OpStatsLogger blockingReadStats = statsLogger.getOpStatsLogger("blocking_read");
-        Counter nullReadCounter = statsLogger.getCounter("null_read");
-
-        logger.info("Created dlm for stream {}.", streamName);
-        LogReader reader = null;
-        Long lastTxId = null;
-        while (null == reader) {
-            // initialize the last txid
-            if (null == lastTxId) {
-                switch (readMode) {
-                    case OLDEST:
-                        lastTxId = 0L;
-                        break;
-                    case LATEST:
-                        try {
-                            lastTxId = dlm.getLastTxId();
-                        } catch (IOException ioe) {
-                            continue;
-                        }
-                        break;
-                    case REWIND:
-                        lastTxId = System.currentTimeMillis() - rewindMs;
-                        break;
-                    case POSITION:
-                        lastTxId = fromTxId;
-                        break;
-                    default:
-                        logger.warn("Unsupported mode {}", readMode);
-                        printUsage();
-                        System.exit(0);
-                        break;
-                }
-                logger.info("Reading from transaction id {}", lastTxId);
-            }
-            // Open the reader
-            Stopwatch stopwatch = Stopwatch.createStarted();
-            try {
-                reader = dlm.getInputStream(lastTxId);
-                long elapsedMs = stopwatch.elapsed(TimeUnit.MICROSECONDS);
-                openReaderStats.registerSuccessfulEvent(elapsedMs);
-                logger.info("It took {} ms to position the reader to transaction id {}", lastTxId);
-            } catch (IOException ioe) {
-                openReaderStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-                logger.warn("Failed to create reader for stream {} reading from {}.", streamName, lastTxId);
-            }
-            if (null == reader) {
-                try {
-                    TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
-                } catch (InterruptedException e) {
-                    logger.warn("Interrupted from sleep after reader was reassigned null for stream {} : ",
-                        streamName, e);
-                }
-                continue;
-            }
-
-            // read loop
-
-            LogRecord record;
-            boolean nonBlocking = false;
-            stopwatch = Stopwatch.createUnstarted();
-            long numCatchupReads = 0L;
-            long numCatchupBytes = 0L;
-            Stopwatch catchupStopwatch = Stopwatch.createStarted();
-            while (true) {
-                try {
-                    stopwatch.start();
-                    record = reader.readNext(nonBlocking);
-                    if (null != record) {
-                        long elapsedMicros = stopwatch.stop().elapsed(TimeUnit.MICROSECONDS);
-                        if (nonBlocking) {
-                            nonBlockingReadStats.registerSuccessfulEvent(elapsedMicros);
-                        } else {
-                            numCatchupBytes += record.getPayload().length;
-                            ++numCatchupReads;
-                            blockingReadStats.registerSuccessfulEvent(elapsedMicros);
-                        }
-                        lastTxId = record.getTransactionId();
-                    } else {
-                        nullReadCounter.inc();
-                    }
-                    if (null == record && !nonBlocking) {
-                        nonBlocking = true;
-                        catchupStopwatch.stop();
-                        logger.info("Catchup {} records (total {} bytes) in {} milliseconds",
-                                new Object[] { numCatchupReads, numCatchupBytes,
-                                    stopwatch.elapsed(TimeUnit.MILLISECONDS) });
-                    }
-                    stopwatch.reset();
-                } catch (IOException e) {
-                    logger.warn("Encountered reading record from stream {} : ", streamName, e);
-                    reader = null;
-                    break;
-                }
-            }
-            try {
-                TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
-            } catch (InterruptedException e) {
-                logger.warn("Interrupted from sleep while creating reader for stream {} : ",
-                    streamName, e);
-            }
-        }
-    }
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/package-info.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/package-info.java
deleted file mode 100644
index d8e198c..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Stream level benchmarks.
- */
-package com.twitter.distributedlog.benchmark.stream;
\ No newline at end of file
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/ShiftableRateLimiter.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/ShiftableRateLimiter.java
deleted file mode 100644
index def0346..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/ShiftableRateLimiter.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 com.twitter.distributedlog.benchmark.utils;
-
-import com.google.common.util.concurrent.RateLimiter;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-/**
- * A wrapper over rate limiter.
- */
-public class ShiftableRateLimiter implements Runnable {
-
-    private final RateLimiter rateLimiter;
-    private final ScheduledExecutorService executor;
-    private final double initialRate, maxRate, changeRate;
-    private final long changeInterval;
-    private final TimeUnit changeIntervalUnit;
-    private double nextRate;
-
-    public ShiftableRateLimiter(double initialRate,
-                                double maxRate,
-                                double changeRate,
-                                long changeInterval,
-                                TimeUnit changeIntervalUnit) {
-        this.initialRate = initialRate;
-        this.maxRate = maxRate;
-        this.changeRate = changeRate;
-        this.nextRate = initialRate;
-        this.changeInterval = changeInterval;
-        this.changeIntervalUnit = changeIntervalUnit;
-        this.rateLimiter = RateLimiter.create(initialRate);
-        this.executor = Executors.newSingleThreadScheduledExecutor();
-        this.executor.scheduleAtFixedRate(this, changeInterval, changeInterval, changeIntervalUnit);
-    }
-
-    public ShiftableRateLimiter duplicate() {
-        return new ShiftableRateLimiter(
-                initialRate,
-                maxRate,
-                changeRate,
-                changeInterval,
-                changeIntervalUnit);
-    }
-
-    @Override
-    public void run() {
-        this.nextRate = Math.min(nextRate + changeRate, maxRate);
-        this.rateLimiter.setRate(nextRate);
-    }
-
-    public RateLimiter getLimiter() {
-        return this.rateLimiter;
-    }
-}
diff --git a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/package-info.java b/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/package-info.java
deleted file mode 100644
index 369b979..0000000
--- a/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Utils for benchmarking.
- */
-package com.twitter.distributedlog.benchmark.utils;
\ No newline at end of file
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Benchmarker.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Benchmarker.java
new file mode 100644
index 0000000..f724102
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Benchmarker.java
@@ -0,0 +1,468 @@
+/**
+ * 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.benchmark;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.benchmark.utils.ShiftableRateLimiter;
+import com.twitter.finagle.stats.OstrichStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.NullStatsProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.cli.BasicParser;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The launcher for benchmarks.
+ */
+public class Benchmarker {
+
+    private static final Logger logger = LoggerFactory.getLogger(Benchmarker.class);
+
+    static final String USAGE = "Benchmarker [-u <uri>] [-c <conf>] [-s serverset] [-m (read|write|dlwrite)]";
+
+    final String[] args;
+    final Options options = new Options();
+
+    int rate = 100;
+    int maxRate = 1000;
+    int changeRate = 100;
+    int changeRateSeconds = 1800;
+    int concurrency = 10;
+    String streamPrefix = "dlog-loadtest";
+    int shardId = -1;
+    int numStreams = 10;
+    List<String> serversetPaths = new ArrayList<String>();
+    List<String> finagleNames = new ArrayList<String>();
+    int msgSize = 256;
+    String mode = null;
+    int durationMins = 60;
+    URI dlUri = null;
+    int batchSize = 0;
+    int readersPerStream = 1;
+    Integer maxStreamId = null;
+    int truncationInterval = 3600;
+    Integer startStreamId = null;
+    Integer endStreamId = null;
+    int hostConnectionCoreSize = 10;
+    int hostConnectionLimit = 10;
+    boolean thriftmux = false;
+    boolean handshakeWithClientInfo = false;
+    boolean readFromHead = false;
+    int sendBufferSize = 1024 * 1024;
+    int recvBufferSize = 1024 * 1024;
+    boolean enableBatching = false;
+    int batchBufferSize = 256 * 1024;
+    int batchFlushIntervalMicros = 2000;
+    String routingServiceFinagleNameString;
+
+    final DistributedLogConfiguration conf = new DistributedLogConfiguration();
+    final StatsReceiver statsReceiver = new OstrichStatsReceiver();
+    StatsProvider statsProvider = null;
+
+    Benchmarker(String[] args) {
+        this.args = args;
+        // prepare options
+        options.addOption("s", "serverset", true, "Proxy Server Set (separated by ',')");
+        options.addOption("fn", "finagle-name", true, "Write proxy finagle name (separated by ',')");
+        options.addOption("c", "conf", true, "DistributedLog Configuration File");
+        options.addOption("u", "uri", true, "DistributedLog URI");
+        options.addOption("i", "shard", true, "Shard Id");
+        options.addOption("p", "provider", true, "DistributedLog Stats Provider");
+        options.addOption("d", "duration", true, "Duration (minutes)");
+        options.addOption("sp", "streamprefix", true, "Stream Prefix");
+        options.addOption("sc", "streamcount", true, "Number of Streams");
+        options.addOption("ms", "messagesize", true, "Message Size (bytes)");
+        options.addOption("bs", "batchsize", true, "Batch Size");
+        options.addOption("r", "rate", true, "Rate limit (requests/second)");
+        options.addOption("mr", "max-rate", true, "Maximum Rate limit (requests/second)");
+        options.addOption("cr", "change-rate", true, "Rate to increase each change period (requests/second)");
+        options.addOption("ci", "change-interval", true, "Rate to increase period, seconds");
+        options.addOption("t", "concurrency", true, "Concurrency (number of threads)");
+        options.addOption("m", "mode", true, "Benchmark mode (read/write)");
+        options.addOption("rps", "readers-per-stream", true, "Number readers per stream");
+        options.addOption("msid", "max-stream-id", true, "Max Stream ID");
+        options.addOption("ti", "truncation-interval", true, "Truncation interval in seconds");
+        options.addOption("ssid", "start-stream-id", true, "Start Stream ID");
+        options.addOption("esid", "end-stream-id", true, "Start Stream ID");
+        options.addOption("hccs", "host-connection-core-size", true, "Finagle hostConnectionCoreSize");
+        options.addOption("hcl", "host-connection-limit", true, "Finagle hostConnectionLimit");
+        options.addOption("mx", "thriftmux", false, "Enable thriftmux (write mode only)");
+        options.addOption("hsci", "handshake-with-client-info", false, "Enable handshaking with client info");
+        options.addOption("rfh", "read-from-head", false, "Read from head of the stream");
+        options.addOption("sb", "send-buffer", true, "Channel send buffer size, in bytes");
+        options.addOption("rb", "recv-buffer", true, "Channel recv buffer size, in bytes");
+        options.addOption("bt", "enable-batch", false, "Enable batching on writers");
+        options.addOption("bbs", "batch-buffer-size", true, "The batch buffer size in bytes");
+        options.addOption("bfi", "batch-flush-interval", true, "The batch buffer flush interval in micros");
+        options.addOption("rs", "routing-service", true, "The routing service finagle name for server-side routing");
+        options.addOption("h", "help", false, "Print usage.");
+    }
+
+    void printUsage() {
+        HelpFormatter helpFormatter = new HelpFormatter();
+        helpFormatter.printHelp(USAGE, options);
+    }
+
+    void run() throws Exception {
+        logger.info("Running benchmark.");
+
+        BasicParser parser = new BasicParser();
+        CommandLine cmdline = parser.parse(options, args);
+        if (cmdline.hasOption("h")) {
+            printUsage();
+            System.exit(0);
+        }
+        if (cmdline.hasOption("s")) {
+            String serversetPathStr = cmdline.getOptionValue("s");
+            serversetPaths = Arrays.asList(StringUtils.split(serversetPathStr, ','));
+        }
+        if (cmdline.hasOption("fn")) {
+            String finagleNameStr = cmdline.getOptionValue("fn");
+            finagleNames = Arrays.asList(StringUtils.split(finagleNameStr, ','));
+        }
+        if (cmdline.hasOption("i")) {
+            shardId = Integer.parseInt(cmdline.getOptionValue("i"));
+        }
+        if (cmdline.hasOption("d")) {
+            durationMins = Integer.parseInt(cmdline.getOptionValue("d"));
+        }
+        if (cmdline.hasOption("sp")) {
+            streamPrefix = cmdline.getOptionValue("sp");
+        }
+        if (cmdline.hasOption("sc")) {
+            numStreams = Integer.parseInt(cmdline.getOptionValue("sc"));
+        }
+        if (cmdline.hasOption("ms")) {
+            msgSize = Integer.parseInt(cmdline.getOptionValue("ms"));
+        }
+        if (cmdline.hasOption("r")) {
+            rate = Integer.parseInt(cmdline.getOptionValue("r"));
+        }
+        if (cmdline.hasOption("mr")) {
+            maxRate = Integer.parseInt(cmdline.getOptionValue("mr"));
+        }
+        if (cmdline.hasOption("cr")) {
+            changeRate = Integer.parseInt(cmdline.getOptionValue("cr"));
+        }
+        if (cmdline.hasOption("ci")) {
+            changeRateSeconds = Integer.parseInt(cmdline.getOptionValue("ci"));
+        }
+        if (cmdline.hasOption("t")) {
+            concurrency = Integer.parseInt(cmdline.getOptionValue("t"));
+        }
+        if (cmdline.hasOption("m")) {
+            mode = cmdline.getOptionValue("m");
+        }
+        if (cmdline.hasOption("u")) {
+            dlUri = URI.create(cmdline.getOptionValue("u"));
+        }
+        if (cmdline.hasOption("bs")) {
+            batchSize = Integer.parseInt(cmdline.getOptionValue("bs"));
+            checkArgument("write" != mode, "batchSize supported only for mode=write");
+        }
+        if (cmdline.hasOption("c")) {
+            String configFile = cmdline.getOptionValue("c");
+            conf.loadConf(new File(configFile).toURI().toURL());
+        }
+        if (cmdline.hasOption("rps")) {
+            readersPerStream = Integer.parseInt(cmdline.getOptionValue("rps"));
+        }
+        if (cmdline.hasOption("msid")) {
+            maxStreamId = Integer.parseInt(cmdline.getOptionValue("msid"));
+        }
+        if (cmdline.hasOption("ti")) {
+            truncationInterval = Integer.parseInt(cmdline.getOptionValue("ti"));
+        }
+        if (cmdline.hasOption("ssid")) {
+            startStreamId = Integer.parseInt(cmdline.getOptionValue("ssid"));
+        }
+        if (cmdline.hasOption("esid")) {
+            endStreamId = Integer.parseInt(cmdline.getOptionValue("esid"));
+        }
+        if (cmdline.hasOption("hccs")) {
+            hostConnectionCoreSize = Integer.parseInt(cmdline.getOptionValue("hccs"));
+        }
+        if (cmdline.hasOption("hcl")) {
+            hostConnectionLimit = Integer.parseInt(cmdline.getOptionValue("hcl"));
+        }
+        if (cmdline.hasOption("sb")) {
+            sendBufferSize = Integer.parseInt(cmdline.getOptionValue("sb"));
+        }
+        if (cmdline.hasOption("rb")) {
+            recvBufferSize = Integer.parseInt(cmdline.getOptionValue("rb"));
+        }
+        if (cmdline.hasOption("rs")) {
+            routingServiceFinagleNameString = cmdline.getOptionValue("rs");
+        }
+        thriftmux = cmdline.hasOption("mx");
+        handshakeWithClientInfo = cmdline.hasOption("hsci");
+        readFromHead = cmdline.hasOption("rfh");
+        enableBatching = cmdline.hasOption("bt");
+        if (cmdline.hasOption("bbs")) {
+            batchBufferSize = Integer.parseInt(cmdline.getOptionValue("bbs"));
+        }
+        if (cmdline.hasOption("bfi")) {
+            batchFlushIntervalMicros = Integer.parseInt(cmdline.getOptionValue("bfi"));
+        }
+
+        checkArgument(shardId >= 0, "shardId must be >= 0");
+        checkArgument(numStreams > 0, "numStreams must be > 0");
+        checkArgument(durationMins > 0, "durationMins must be > 0");
+        checkArgument(streamPrefix != null, "streamPrefix must be defined");
+        checkArgument(hostConnectionCoreSize > 0, "host connection core size must be > 0");
+        checkArgument(hostConnectionLimit > 0, "host connection limit must be > 0");
+
+        if (cmdline.hasOption("p")) {
+            statsProvider = ReflectionUtils.newInstance(cmdline.getOptionValue("p"), StatsProvider.class);
+        } else {
+            statsProvider = new NullStatsProvider();
+        }
+
+        logger.info("Starting stats provider : {}.", statsProvider.getClass());
+        statsProvider.start(conf);
+
+        Worker w = null;
+        if (mode.startsWith("read")) {
+            w = runReader();
+        } else if (mode.startsWith("write")) {
+            w = runWriter();
+        } else if (mode.startsWith("dlwrite")) {
+            w = runDLWriter();
+        } else if (mode.startsWith("dlread")) {
+            w = runDLReader();
+        }
+
+        if (w == null) {
+            throw new IOException("Unknown mode " + mode + " to run the benchmark.");
+        }
+
+        Thread workerThread = new Thread(w, mode + "-benchmark-thread");
+        workerThread.start();
+
+        TimeUnit.MINUTES.sleep(durationMins);
+
+        logger.info("{} minutes passed, exiting...", durationMins);
+        w.close();
+
+        if (null != statsProvider) {
+            statsProvider.stop();
+        }
+
+        Runtime.getRuntime().exit(0);
+    }
+
+    Worker runWriter() {
+        checkArgument(!finagleNames.isEmpty() || !serversetPaths.isEmpty() || null != dlUri,
+                "either serverset paths, finagle-names or uri required");
+        checkArgument(msgSize > 0, "messagesize must be greater than 0");
+        checkArgument(rate > 0, "rate must be greater than 0");
+        checkArgument(maxRate >= rate, "max rate must be greater than rate");
+        checkArgument(changeRate >= 0, "change rate must be positive");
+        checkArgument(changeRateSeconds >= 0, "change rate must be positive");
+        checkArgument(concurrency > 0, "concurrency must be greater than 0");
+
+        ShiftableRateLimiter rateLimiter =
+                new ShiftableRateLimiter(rate, maxRate, changeRate, changeRateSeconds, TimeUnit.SECONDS);
+        return createWriteWorker(
+                streamPrefix,
+                dlUri,
+                null == startStreamId ? shardId * numStreams : startStreamId,
+                null == endStreamId ? (shardId + 1) * numStreams : endStreamId,
+                rateLimiter,
+                concurrency,
+                msgSize,
+                batchSize,
+                hostConnectionCoreSize,
+                hostConnectionLimit,
+                serversetPaths,
+                finagleNames,
+                statsReceiver.scope("write_client"),
+                statsProvider.getStatsLogger("write"),
+                thriftmux,
+                handshakeWithClientInfo,
+                sendBufferSize,
+                recvBufferSize,
+                enableBatching,
+                batchBufferSize,
+                batchFlushIntervalMicros,
+                routingServiceFinagleNameString);
+    }
+
+    protected WriterWorker createWriteWorker(
+            String streamPrefix,
+            URI uri,
+            int startStreamId,
+            int endStreamId,
+            ShiftableRateLimiter rateLimiter,
+            int writeConcurrency,
+            int messageSizeBytes,
+            int batchSize,
+            int hostConnectionCoreSize,
+            int hostConnectionLimit,
+            List<String> serverSetPaths,
+            List<String> finagleNames,
+            StatsReceiver statsReceiver,
+            StatsLogger statsLogger,
+            boolean thriftmux,
+            boolean handshakeWithClientInfo,
+            int sendBufferSize,
+            int recvBufferSize,
+            boolean enableBatching,
+            int batchBufferSize,
+            int batchFlushIntervalMicros,
+            String routingServiceFinagleNameString) {
+        return new WriterWorker(
+                streamPrefix,
+                uri,
+                startStreamId,
+                endStreamId,
+                rateLimiter,
+                writeConcurrency,
+                messageSizeBytes,
+                batchSize,
+                hostConnectionCoreSize,
+                hostConnectionLimit,
+                serverSetPaths,
+                finagleNames,
+                statsReceiver,
+                statsLogger,
+                thriftmux,
+                handshakeWithClientInfo,
+                sendBufferSize,
+                recvBufferSize,
+                enableBatching,
+                batchBufferSize,
+                batchFlushIntervalMicros,
+                routingServiceFinagleNameString);
+    }
+
+    Worker runDLWriter() throws IOException {
+        checkNotNull(dlUri, "dlUri must be defined");
+        checkArgument(rate > 0, "rate must be greater than 0");
+        checkArgument(maxRate >= rate, "max rate must be greater than rate");
+        checkArgument(changeRate >= 0, "change rate must be positive");
+        checkArgument(changeRateSeconds >= 0, "change rate must be positive");
+        checkArgument(concurrency > 0, "concurrency must be greater than 0");
+
+        ShiftableRateLimiter rateLimiter =
+                new ShiftableRateLimiter(rate, maxRate, changeRate, changeRateSeconds, TimeUnit.SECONDS);
+
+        return new DLWriterWorker(conf,
+                dlUri,
+                streamPrefix,
+                shardId * numStreams,
+                (shardId + 1) * numStreams,
+                rateLimiter,
+                concurrency,
+                msgSize,
+                statsProvider.getStatsLogger("dlwrite"));
+    }
+
+    Worker runReader() throws IOException {
+        checkArgument(!finagleNames.isEmpty() || !serversetPaths.isEmpty() || null != dlUri,
+                "either serverset paths, finagle-names or dlUri required");
+        checkArgument(concurrency > 0, "concurrency must be greater than 0");
+        checkArgument(truncationInterval > 0, "truncation interval should be greater than 0");
+        return runReaderInternal(serversetPaths, finagleNames, truncationInterval);
+    }
+
+    Worker runDLReader() throws IOException {
+        return runReaderInternal(new ArrayList<String>(), new ArrayList<String>(), 0);
+    }
+
+    private Worker runReaderInternal(List<String> serversetPaths,
+                                     List<String> finagleNames,
+                                     int truncationInterval) throws IOException {
+        checkNotNull(dlUri);
+
+        int ssid = null == startStreamId ? shardId * numStreams : startStreamId;
+        int esid = null == endStreamId ? (shardId + readersPerStream) * numStreams : endStreamId;
+        if (null != maxStreamId) {
+            esid = Math.min(esid, maxStreamId);
+        }
+
+        return createReaderWorker(
+                conf,
+                dlUri,
+                streamPrefix,
+                ssid,
+                esid,
+                concurrency,
+                serversetPaths,
+                finagleNames,
+                truncationInterval,
+                readFromHead,
+                statsReceiver,
+                statsProvider.getStatsLogger("dlreader"));
+    }
+
+    protected ReaderWorker createReaderWorker(
+            DistributedLogConfiguration conf,
+            URI uri,
+            String streamPrefix,
+            int startStreamId,
+            int endStreamId,
+            int readThreadPoolSize,
+            List<String> serverSetPaths,
+            List<String> finagleNames,
+            int truncationIntervalInSeconds,
+            boolean readFromHead, /* read from the earliest data of log */
+            StatsReceiver statsReceiver,
+            StatsLogger statsLogger) throws IOException {
+        return new ReaderWorker(
+                conf,
+                uri,
+                streamPrefix,
+                startStreamId,
+                endStreamId,
+                readThreadPoolSize,
+                serverSetPaths,
+                finagleNames,
+                truncationIntervalInSeconds,
+                readFromHead,
+                statsReceiver,
+                statsLogger);
+    }
+
+    public static void main(String[] args) {
+        Benchmarker benchmarker = new Benchmarker(args);
+        try {
+            benchmarker.run();
+        } catch (Exception e) {
+            logger.info("Benchmark quit due to : ", e);
+        }
+    }
+
+}
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
new file mode 100644
index 0000000..a5e7a0a
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/DLWriterWorker.java
@@ -0,0 +1,245 @@
+/**
+ * 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.benchmark;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogRecord;
+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 java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The benchmark for core library writer.
+ */
+public class DLWriterWorker implements Worker {
+
+    private static final Logger LOG = LoggerFactory.getLogger(DLWriterWorker.class);
+
+    static final int BACKOFF_MS = 200;
+
+    final String streamPrefix;
+    final int startStreamId;
+    final int endStreamId;
+    final int writeConcurrency;
+    final int messageSizeBytes;
+    final ExecutorService executorService;
+    final ScheduledExecutorService rescueService;
+    final ShiftableRateLimiter rateLimiter;
+    final Random random;
+    final DistributedLogNamespace namespace;
+    final List<DistributedLogManager> dlms;
+    final List<AsyncLogWriter> streamWriters;
+    final int numStreams;
+
+    volatile boolean running = true;
+
+    final StatsLogger statsLogger;
+    final OpStatsLogger requestStat;
+
+    public DLWriterWorker(DistributedLogConfiguration conf,
+                          URI uri,
+                          String streamPrefix,
+                          int startStreamId,
+                          int endStreamId,
+                          ShiftableRateLimiter rateLimiter,
+                          int writeConcurrency,
+                          int messageSizeBytes,
+                          StatsLogger statsLogger) throws IOException {
+        checkArgument(startStreamId <= endStreamId);
+        this.streamPrefix = streamPrefix;
+        this.startStreamId = startStreamId;
+        this.endStreamId = endStreamId;
+        this.rateLimiter = rateLimiter;
+        this.writeConcurrency = writeConcurrency;
+        this.messageSizeBytes = messageSizeBytes;
+        this.statsLogger = statsLogger;
+        this.requestStat = this.statsLogger.getOpStatsLogger("requests");
+        this.executorService = Executors.newCachedThreadPool();
+        this.rescueService = Executors.newSingleThreadScheduledExecutor();
+        this.random = new Random(System.currentTimeMillis());
+
+        this.namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .statsLogger(statsLogger.scope("dl"))
+                .build();
+        this.numStreams = endStreamId - startStreamId;
+        dlms = new ArrayList<DistributedLogManager>(numStreams);
+        streamWriters = new ArrayList<AsyncLogWriter>(numStreams);
+        final ConcurrentMap<String, AsyncLogWriter> writers = new ConcurrentHashMap<String, AsyncLogWriter>();
+        final CountDownLatch latch = new CountDownLatch(this.numStreams);
+        for (int i = startStreamId; i < endStreamId; i++) {
+            final String streamName = String.format("%s_%d", streamPrefix, i);
+            final DistributedLogManager dlm = namespace.openLog(streamName);
+            executorService.submit(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+                        if (null != writers.putIfAbsent(streamName, writer)) {
+                            FutureUtils.result(writer.asyncClose());
+                        }
+                        latch.countDown();
+                    } catch (IOException e) {
+                        LOG.error("Failed to intialize writer for stream : {}", streamName, e);
+                    }
+
+                }
+            });
+            dlms.add(dlm);
+        }
+        try {
+            latch.await();
+        } catch (InterruptedException e) {
+            throw new IOException("Interrupted on initializing writers for streams.", e);
+        }
+        for (int i = startStreamId; i < endStreamId; i++) {
+            final String streamName = String.format("%s_%d", streamPrefix, i);
+            AsyncLogWriter writer = writers.get(streamName);
+            if (null == writer) {
+                throw new IOException("Writer for " + streamName + " never initialized.");
+            }
+            streamWriters.add(writer);
+        }
+        LOG.info("Writing to {} streams.", numStreams);
+    }
+
+    void rescueWriter(int idx, AsyncLogWriter writer) {
+        if (streamWriters.get(idx) == writer) {
+            try {
+                FutureUtils.result(writer.asyncClose());
+            } catch (IOException e) {
+                LOG.error("Failed to close writer for stream {}.", idx);
+            }
+            AsyncLogWriter newWriter = null;
+            try {
+                newWriter = dlms.get(idx).startAsyncLogSegmentNonPartitioned();
+            } catch (IOException e) {
+                LOG.error("Failed to create new writer for stream {}, backoff for {} ms.",
+                          idx, BACKOFF_MS);
+                scheduleRescue(idx, writer, BACKOFF_MS);
+            }
+            streamWriters.set(idx, newWriter);
+        } else {
+            LOG.warn("AsyncLogWriter for stream {} was already rescued.", idx);
+        }
+    }
+
+    void scheduleRescue(final int idx, final AsyncLogWriter writer, int delayMs) {
+        Runnable r = new Runnable() {
+            @Override
+            public void run() {
+                rescueWriter(idx, writer);
+            }
+        };
+        if (delayMs > 0) {
+            rescueService.schedule(r, delayMs, TimeUnit.MILLISECONDS);
+        } else {
+            rescueService.submit(r);
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.running = false;
+        SchedulerUtils.shutdownScheduler(this.executorService, 2, TimeUnit.MINUTES);
+        SchedulerUtils.shutdownScheduler(this.rescueService, 2, TimeUnit.MINUTES);
+        for (AsyncLogWriter writer : streamWriters) {
+            FutureUtils.result(writer.asyncClose());
+        }
+        for (DistributedLogManager dlm : dlms) {
+            dlm.close();
+        }
+        namespace.close();
+    }
+
+    @Override
+    public void run() {
+        LOG.info("Starting dlwriter (concurrency = {}, prefix = {}, numStreams = {})",
+                 new Object[] { writeConcurrency, streamPrefix, numStreams });
+        for (int i = 0; i < writeConcurrency; i++) {
+            executorService.submit(new Writer(i));
+        }
+    }
+
+    class Writer implements Runnable {
+
+        final int idx;
+
+        Writer(int idx) {
+            this.idx = idx;
+        }
+
+        @Override
+        public void run() {
+            LOG.info("Started writer {}.", idx);
+            while (running) {
+                final int streamIdx = random.nextInt(numStreams);
+                final AsyncLogWriter writer = streamWriters.get(streamIdx);
+                rateLimiter.getLimiter().acquire();
+                final long requestMillis = System.currentTimeMillis();
+                final byte[] data;
+                try {
+                    data = Utils.generateMessage(requestMillis, messageSizeBytes);
+                } catch (TException e) {
+                    LOG.error("Error on generating message : ", e);
+                    break;
+                }
+                writer.write(new LogRecord(requestMillis, data)).addEventListener(new FutureEventListener<DLSN>() {
+                    @Override
+                    public void onSuccess(DLSN value) {
+                        requestStat.registerSuccessfulEvent(System.currentTimeMillis() - requestMillis);
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        requestStat.registerFailedEvent(System.currentTimeMillis() - requestMillis);
+                        LOG.error("Failed to publish, rescue it : ", cause);
+                        scheduleRescue(streamIdx, writer, 0);
+                    }
+                });
+            }
+        }
+    }
+
+}
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
new file mode 100644
index 0000000..11cba6f
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/ReaderWorker.java
@@ -0,0 +1,468 @@
+/**
+ * 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.benchmark;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+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.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogRecordSet;
+import org.apache.distributedlog.LogRecordWithDLSN;
+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.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.SchedulerUtils;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Duration$;
+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.net.URI;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The benchmark for core library reader.
+ */
+public class ReaderWorker implements Worker {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ReaderWorker.class);
+
+    static final int BACKOFF_MS = 200;
+
+    final String streamPrefix;
+    final int startStreamId;
+    final int endStreamId;
+    final ScheduledExecutorService executorService;
+    final ExecutorService callbackExecutor;
+    final DistributedLogNamespace namespace;
+    final DistributedLogManager[] dlms;
+    final AsyncLogReader[] logReaders;
+    final StreamReader[] streamReaders;
+    final int numStreams;
+    final boolean readFromHead;
+
+    final int truncationIntervalInSeconds;
+    // DL Client Related Variables
+    final DLZkServerSet[] serverSets;
+    final List<String> finagleNames;
+    final DistributedLogClient dlc;
+
+    volatile boolean running = true;
+
+    final StatsReceiver statsReceiver;
+    final StatsLogger statsLogger;
+    final OpStatsLogger e2eStat;
+    final OpStatsLogger deliveryStat;
+    final OpStatsLogger negativeE2EStat;
+    final OpStatsLogger negativeDeliveryStat;
+    final OpStatsLogger truncationStat;
+    final Counter invalidRecordsCounter;
+    final Counter outOfOrderSequenceIdCounter;
+
+    class StreamReader implements FutureEventListener<List<LogRecordWithDLSN>>, Runnable, Gauge<Number> {
+
+        final int streamIdx;
+        final String streamName;
+        DLSN prevDLSN = null;
+        long prevSequenceId = Long.MIN_VALUE;
+        private static final String gaugeLabel = "sequence_id";
+
+        StreamReader(int idx, StatsLogger statsLogger) {
+            this.streamIdx = idx;
+            int streamId = startStreamId + streamIdx;
+            streamName = String.format("%s_%d", streamPrefix, streamId);
+            statsLogger.scope(streamName).registerGauge(gaugeLabel, this);
+        }
+
+        @Override
+        public void onSuccess(final List<LogRecordWithDLSN> records) {
+            for (final LogRecordWithDLSN record : records) {
+                if (record.isRecordSet()) {
+                    try {
+                        processRecordSet(record);
+                    } catch (IOException e) {
+                        onFailure(e);
+                    }
+                } else {
+                    processRecord(record);
+                }
+            }
+            readLoop();
+        }
+
+        public void processRecordSet(final LogRecordWithDLSN record) throws IOException {
+            LogRecordSet.Reader reader = LogRecordSet.of(record);
+            LogRecordWithDLSN nextRecord = reader.nextRecord();
+            while (null != nextRecord) {
+                processRecord(nextRecord);
+                nextRecord = reader.nextRecord();
+            }
+        }
+
+        public void processRecord(final LogRecordWithDLSN record) {
+            Message msg;
+            try {
+                msg = Utils.parseMessage(record.getPayload());
+            } catch (TException e) {
+                invalidRecordsCounter.inc();
+                LOG.warn("Failed to parse record {} for stream {} : size = {} , ",
+                         new Object[] { record, streamIdx, record.getPayload().length, e });
+                return;
+            }
+            long curTimeMillis = System.currentTimeMillis();
+            long e2eLatency = curTimeMillis - msg.getPublishTime();
+            long deliveryLatency = curTimeMillis - record.getTransactionId();
+            if (e2eLatency >= 0) {
+                e2eStat.registerSuccessfulEvent(e2eLatency);
+            } else {
+                negativeE2EStat.registerSuccessfulEvent(-e2eLatency);
+            }
+            if (deliveryLatency >= 0) {
+                deliveryStat.registerSuccessfulEvent(deliveryLatency);
+            } else {
+                negativeDeliveryStat.registerSuccessfulEvent(-deliveryLatency);
+            }
+
+            prevDLSN = record.getDlsn();
+        }
+
+        @Override
+        public void onFailure(Throwable cause) {
+            scheduleReinitStream(streamIdx).map(new Function<Void, Void>() {
+                @Override
+                public Void apply(Void value) {
+                    prevDLSN = null;
+                    prevSequenceId = Long.MIN_VALUE;
+                    readLoop();
+                    return null;
+                }
+            });
+        }
+
+        void readLoop() {
+            if (!running) {
+                return;
+            }
+            logReaders[streamIdx].readBulk(10).addEventListener(this);
+        }
+
+        @Override
+        public void run() {
+            final DLSN dlsnToTruncate = prevDLSN;
+            if (null == dlsnToTruncate) {
+                return;
+            }
+            final Stopwatch stopwatch = Stopwatch.createStarted();
+            dlc.truncate(streamName, dlsnToTruncate).addEventListener(
+                    new FutureEventListener<Boolean>() {
+                        @Override
+                        public void onSuccess(Boolean value) {
+                            truncationStat.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MILLISECONDS));
+                        }
+
+                        @Override
+                        public void onFailure(Throwable cause) {
+                            truncationStat.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MILLISECONDS));
+                            LOG.error("Failed to truncate stream {} to {} : ",
+                                    new Object[]{streamName, dlsnToTruncate, cause});
+                        }
+                    });
+        }
+
+        @Override
+        public Number getDefaultValue() {
+            return Long.MIN_VALUE;
+        }
+
+        @Override
+        public synchronized Number getSample() {
+            return prevSequenceId;
+        }
+
+        void unregisterGauge() {
+            statsLogger.scope(streamName).unregisterGauge(gaugeLabel, this);
+        }
+    }
+
+    public ReaderWorker(DistributedLogConfiguration conf,
+                        URI uri,
+                        String streamPrefix,
+                        int startStreamId,
+                        int endStreamId,
+                        int readThreadPoolSize,
+                        List<String> serverSetPaths,
+                        List<String> finagleNames,
+                        int truncationIntervalInSeconds,
+                        boolean readFromHead, /* read from the earliest data of log */
+                        StatsReceiver statsReceiver,
+                        StatsLogger statsLogger) throws IOException {
+        checkArgument(startStreamId <= endStreamId);
+        this.streamPrefix = streamPrefix;
+        this.startStreamId = startStreamId;
+        this.endStreamId = endStreamId;
+        this.truncationIntervalInSeconds = truncationIntervalInSeconds;
+        this.readFromHead = readFromHead;
+        this.statsReceiver = statsReceiver;
+        this.statsLogger = statsLogger;
+        this.e2eStat = this.statsLogger.getOpStatsLogger("e2e");
+        this.negativeE2EStat = this.statsLogger.getOpStatsLogger("e2eNegative");
+        this.deliveryStat = this.statsLogger.getOpStatsLogger("delivery");
+        this.negativeDeliveryStat = this.statsLogger.getOpStatsLogger("deliveryNegative");
+        this.truncationStat = this.statsLogger.getOpStatsLogger("truncation");
+        this.invalidRecordsCounter = this.statsLogger.getCounter("invalid_records");
+        this.outOfOrderSequenceIdCounter = this.statsLogger.getCounter("out_of_order_seq_id");
+        this.executorService = Executors.newScheduledThreadPool(
+                readThreadPoolSize, new ThreadFactoryBuilder().setNameFormat("benchmark.reader-%d").build());
+        this.callbackExecutor = Executors.newFixedThreadPool(
+                Runtime.getRuntime().availableProcessors(),
+                new ThreadFactoryBuilder().setNameFormat("benchmark.reader-callback-%d").build());
+        this.finagleNames = finagleNames;
+        this.serverSets = createServerSets(serverSetPaths);
+
+        conf.setDeserializeRecordSetOnReads(false);
+
+        if (truncationIntervalInSeconds > 0 && (!finagleNames.isEmpty() || !serverSetPaths.isEmpty())) {
+            // Construct client for truncation
+            DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
+                    .clientId(ClientId$.MODULE$.apply("dlog_loadtest_reader"))
+                    .clientBuilder(ClientBuilder.get()
+                        .hostConnectionLimit(10)
+                        .hostConnectionCoresize(10)
+                        .tcpConnectTimeout(Duration$.MODULE$.fromSeconds(1))
+                        .requestTimeout(Duration$.MODULE$.fromSeconds(2)))
+                    .redirectBackoffStartMs(100)
+                    .redirectBackoffMaxMs(500)
+                    .requestTimeoutMs(2000)
+                    .statsReceiver(statsReceiver)
+                    .thriftmux(true)
+                    .name("reader");
+
+            if (serverSetPaths.isEmpty()) {
+                // Prepare finagle names
+                String local = finagleNames.get(0);
+                String[] remotes = new String[finagleNames.size() - 1];
+                finagleNames.subList(1, finagleNames.size()).toArray(remotes);
+
+                builder = builder.finagleNameStrs(local, remotes);
+                LOG.info("Initialized distributedlog client for truncation @ {}.", finagleNames);
+            } else if (serverSets.length != 0){
+                ServerSet local = this.serverSets[0].getServerSet();
+                ServerSet[] remotes = new ServerSet[this.serverSets.length - 1];
+                for (int i = 1; i < serverSets.length; i++) {
+                    remotes[i - 1] = serverSets[i].getServerSet();
+                }
+
+                builder = builder.serverSets(local, remotes);
+                LOG.info("Initialized distributedlog client for truncation @ {}.", serverSetPaths);
+            } else {
+                builder = builder.uri(uri);
+                LOG.info("Initialized distributedlog client for namespace {}", uri);
+            }
+            dlc = builder.build();
+        } else {
+            dlc = null;
+        }
+
+        // construct the factory
+        this.namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .statsLogger(statsLogger.scope("dl"))
+                .build();
+        this.numStreams = endStreamId - startStreamId;
+        this.dlms = new DistributedLogManager[numStreams];
+        this.logReaders = new AsyncLogReader[numStreams];
+        final CountDownLatch latch = new CountDownLatch(numStreams);
+        for (int i = 0; i < numStreams; i++) {
+            final int idx = i;
+            executorService.submit(new Runnable() {
+                @Override
+                public void run() {
+                    reinitStream(idx).map(new Function<Void, Void>() {
+                        @Override
+                        public Void apply(Void value) {
+                            LOG.info("Initialized stream reader {}.", idx);
+                            latch.countDown();
+                            return null;
+                        }
+                    });
+                }
+            });
+        }
+        try {
+            latch.await();
+        } catch (InterruptedException e) {
+            throw new DLInterruptedException("Failed to intialize benchmark readers : ", e);
+        }
+        this.streamReaders = new StreamReader[numStreams];
+        for (int i = 0; i < numStreams; i++) {
+            streamReaders[i] = new StreamReader(i, statsLogger.scope("perstream"));
+            if (truncationIntervalInSeconds > 0) {
+                executorService.scheduleWithFixedDelay(streamReaders[i],
+                        truncationIntervalInSeconds, truncationIntervalInSeconds, TimeUnit.SECONDS);
+            }
+        }
+        LOG.info("Initialized benchmark reader on {} streams {} : [{} - {})",
+                 new Object[] { numStreams, streamPrefix, startStreamId, endStreamId });
+    }
+
+    protected DLZkServerSet[] createServerSets(List<String> serverSetPaths) {
+        DLZkServerSet[] serverSets = new DLZkServerSet[serverSetPaths.size()];
+        for (int i = 0; i < serverSets.length; i++) {
+            String serverSetPath = serverSetPaths.get(i);
+            serverSets[i] = DLZkServerSet.of(URI.create(serverSetPath), 60000);
+        }
+        return serverSets;
+    }
+
+    private Future<Void> reinitStream(int idx) {
+        Promise<Void> promise = new Promise<Void>();
+        reinitStream(idx, promise);
+        return promise;
+    }
+
+    private void reinitStream(int idx, Promise<Void> promise) {
+        int streamId = startStreamId + idx;
+        String streamName = String.format("%s_%d", streamPrefix, streamId);
+
+        if (logReaders[idx] != null) {
+            try {
+                FutureUtils.result(logReaders[idx].asyncClose());
+            } catch (IOException e) {
+                LOG.warn("Failed on closing stream reader {} : ", streamName, e);
+            }
+            logReaders[idx] = null;
+        }
+        if (dlms[idx] != null) {
+            try {
+                dlms[idx].close();
+            } catch (IOException e) {
+                LOG.warn("Failed on closing dlm {} : ", streamName, e);
+            }
+            dlms[idx] = null;
+        }
+
+        try {
+            dlms[idx] = namespace.openLog(streamName);
+        } catch (IOException ioe) {
+            LOG.error("Failed on creating dlm {} : ", streamName, ioe);
+            scheduleReinitStream(idx, promise);
+            return;
+        }
+        DLSN lastDLSN;
+        if (readFromHead) {
+            lastDLSN = DLSN.InitialDLSN;
+        } else {
+            try {
+                lastDLSN = dlms[idx].getLastDLSN();
+            } catch (IOException ioe) {
+                LOG.error("Failed on getting last dlsn from stream {} : ", streamName, ioe);
+                scheduleReinitStream(idx, promise);
+                return;
+            }
+        }
+        try {
+            logReaders[idx] = dlms[idx].getAsyncLogReader(lastDLSN);
+        } catch (IOException ioe) {
+            LOG.error("Failed on opening reader for stream {} starting from {} : ",
+                      new Object[] { streamName, lastDLSN, ioe });
+            scheduleReinitStream(idx, promise);
+            return;
+        }
+        LOG.info("Opened reader for stream {}, starting from {}.", streamName, lastDLSN);
+        promise.setValue(null);
+    }
+
+    Future<Void> scheduleReinitStream(int idx) {
+        Promise<Void> promise = new Promise<Void>();
+        scheduleReinitStream(idx, promise);
+        return promise;
+    }
+
+    void scheduleReinitStream(final int idx, final Promise<Void> promise) {
+        executorService.schedule(new Runnable() {
+            @Override
+            public void run() {
+                reinitStream(idx, promise);
+            }
+        }, BACKOFF_MS, TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.running = false;
+        for (AsyncLogReader reader : logReaders) {
+            if (null != reader) {
+                FutureUtils.result(reader.asyncClose());
+            }
+        }
+        for (DistributedLogManager dlm : dlms) {
+            if (null != dlm) {
+                dlm.close();
+            }
+        }
+        namespace.close();
+        SchedulerUtils.shutdownScheduler(executorService, 2, TimeUnit.MINUTES);
+        SchedulerUtils.shutdownScheduler(callbackExecutor, 2, TimeUnit.MINUTES);
+        if (this.dlc != null) {
+            this.dlc.close();
+        }
+        for (DLZkServerSet serverSet: serverSets) {
+            serverSet.close();
+        }
+        // Unregister gauges to prevent GC spirals
+        for (StreamReader sr : streamReaders) {
+            sr.unregisterGauge();
+        }
+    }
+
+    @Override
+    public void run() {
+        LOG.info("Starting reader (prefix = {}, numStreams = {}).",
+                 streamPrefix, numStreams);
+        for (StreamReader sr : streamReaders) {
+            sr.readLoop();
+        }
+    }
+}
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Utils.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Utils.java
new file mode 100644
index 0000000..81f99ef
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Utils.java
@@ -0,0 +1,57 @@
+/**
+ * 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.benchmark;
+
+import org.apache.distributedlog.benchmark.thrift.Message;
+import java.nio.ByteBuffer;
+import java.util.Random;
+import org.apache.thrift.TException;
+import org.apache.thrift.TSerializer;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.apache.thrift.transport.TMemoryInputTransport;
+
+/**
+ * Utils for generating and parsing messages.
+ */
+public class Utils {
+
+    static final Random RAND = new Random(System.currentTimeMillis());
+    static final ThreadLocal<TSerializer> MSG_SERIALIZER =
+            new ThreadLocal<TSerializer>() {
+                @Override
+                public TSerializer initialValue() {
+                    return new TSerializer(new TBinaryProtocol.Factory());
+                }
+            };
+
+    public static byte[] generateMessage(long requestMillis, int payLoadSize) throws TException {
+        byte[] payload = new byte[payLoadSize];
+        RAND.nextBytes(payload);
+        Message msg = new Message(requestMillis, ByteBuffer.wrap(payload));
+        return MSG_SERIALIZER.get().serialize(msg);
+    }
+
+    public static Message parseMessage(byte[] data) throws TException {
+        Message msg = new Message();
+        TMemoryInputTransport transport = new TMemoryInputTransport(data);
+        TBinaryProtocol protocol = new TBinaryProtocol(transport);
+        msg.read(protocol);
+        return msg;
+    }
+
+}
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Worker.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Worker.java
new file mode 100644
index 0000000..a948092
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Worker.java
@@ -0,0 +1,26 @@
+/**
+ * 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.benchmark;
+
+import java.io.Closeable;
+
+/**
+ * Worker to run benchmark.
+ */
+public interface Worker extends Closeable, Runnable {
+}
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
new file mode 100644
index 0000000..9e96765
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/WriterWorker.java
@@ -0,0 +1,387 @@
+/**
+ * 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.benchmark;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.benchmark.utils.ShiftableRateLimiter;
+import org.apache.distributedlog.client.DistributedLogMultiStreamWriter;
+import org.apache.distributedlog.client.serverset.DLZkServerSet;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import org.apache.distributedlog.util.SchedulerUtils;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.finagle.thrift.ClientId;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Duration$;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Random;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.thrift.TException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Benchmark for distributedlog proxy client.
+ */
+public class WriterWorker implements Worker {
+
+    static final Logger LOG = LoggerFactory.getLogger(WriterWorker.class);
+
+    final String streamPrefix;
+    final int startStreamId;
+    final int endStreamId;
+    final int writeConcurrency;
+    final int messageSizeBytes;
+    final int hostConnectionCoreSize;
+    final int hostConnectionLimit;
+    final ExecutorService executorService;
+    final ShiftableRateLimiter rateLimiter;
+    final URI dlUri;
+    final DLZkServerSet[] serverSets;
+    final List<String> finagleNames;
+    final Random random;
+    final List<String> streamNames;
+    final int numStreams;
+    final int batchSize;
+    final boolean thriftmux;
+    final boolean handshakeWithClientInfo;
+    final int sendBufferSize;
+    final int recvBufferSize;
+    final boolean enableBatching;
+    final int batchBufferSize;
+    final int batchFlushIntervalMicros;
+    private final String routingServiceFinagleName;
+
+    volatile boolean running = true;
+
+    final StatsReceiver statsReceiver;
+    final StatsLogger statsLogger;
+    final OpStatsLogger requestStat;
+    final StatsLogger exceptionsLogger;
+    final StatsLogger dlErrorCodeLogger;
+
+    // callback thread
+    final ExecutorService executor;
+
+    public WriterWorker(String streamPrefix,
+                        URI uri,
+                        int startStreamId,
+                        int endStreamId,
+                        ShiftableRateLimiter rateLimiter,
+                        int writeConcurrency,
+                        int messageSizeBytes,
+                        int batchSize,
+                        int hostConnectionCoreSize,
+                        int hostConnectionLimit,
+                        List<String> serverSetPaths,
+                        List<String> finagleNames,
+                        StatsReceiver statsReceiver,
+                        StatsLogger statsLogger,
+                        boolean thriftmux,
+                        boolean handshakeWithClientInfo,
+                        int sendBufferSize,
+                        int recvBufferSize,
+                        boolean enableBatching,
+                        int batchBufferSize,
+                        int batchFlushIntervalMicros,
+                        String routingServiceFinagleName) {
+        checkArgument(startStreamId <= endStreamId);
+        checkArgument(!finagleNames.isEmpty() || !serverSetPaths.isEmpty());
+        this.streamPrefix = streamPrefix;
+        this.dlUri = uri;
+        this.startStreamId = startStreamId;
+        this.endStreamId = endStreamId;
+        this.rateLimiter = rateLimiter;
+        this.writeConcurrency = writeConcurrency;
+        this.messageSizeBytes = messageSizeBytes;
+        this.statsReceiver = statsReceiver;
+        this.statsLogger = statsLogger;
+        this.requestStat = this.statsLogger.getOpStatsLogger("requests");
+        this.exceptionsLogger = statsLogger.scope("exceptions");
+        this.dlErrorCodeLogger = statsLogger.scope("dl_error_code");
+        this.executorService = Executors.newCachedThreadPool();
+        this.random = new Random(System.currentTimeMillis());
+        this.batchSize = batchSize;
+        this.hostConnectionCoreSize = hostConnectionCoreSize;
+        this.hostConnectionLimit = hostConnectionLimit;
+        this.thriftmux = thriftmux;
+        this.handshakeWithClientInfo = handshakeWithClientInfo;
+        this.sendBufferSize = sendBufferSize;
+        this.recvBufferSize = recvBufferSize;
+        this.enableBatching = enableBatching;
+        this.batchBufferSize = batchBufferSize;
+        this.batchFlushIntervalMicros = batchFlushIntervalMicros;
+        this.finagleNames = finagleNames;
+        this.serverSets = createServerSets(serverSetPaths);
+        this.executor = Executors.newFixedThreadPool(Runtime.getRuntime().availableProcessors());
+        this.routingServiceFinagleName = routingServiceFinagleName;
+
+        // Streams
+        streamNames = new ArrayList<String>(endStreamId - startStreamId);
+        for (int i = startStreamId; i < endStreamId; i++) {
+            streamNames.add(String.format("%s_%d", streamPrefix, i));
+        }
+        numStreams = streamNames.size();
+        LOG.info("Writing to {} streams : {}", numStreams, streamNames);
+    }
+
+    protected DLZkServerSet[] createServerSets(List<String> serverSetPaths) {
+        DLZkServerSet[] serverSets = new DLZkServerSet[serverSetPaths.size()];
+        for (int i = 0; i < serverSets.length; i++) {
+            String serverSetPath = serverSetPaths.get(i);
+            serverSets[i] = DLZkServerSet.of(URI.create(serverSetPath), 60000);
+        }
+        return serverSets;
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.running = false;
+        SchedulerUtils.shutdownScheduler(this.executorService, 2, TimeUnit.MINUTES);
+        for (DLZkServerSet serverSet: serverSets) {
+            serverSet.close();
+        }
+    }
+
+    private DistributedLogClient buildDlogClient() {
+        ClientBuilder clientBuilder = ClientBuilder.get()
+            .hostConnectionLimit(hostConnectionLimit)
+            .hostConnectionCoresize(hostConnectionCoreSize)
+            .tcpConnectTimeout(Duration$.MODULE$.fromMilliseconds(200))
+            .connectTimeout(Duration$.MODULE$.fromMilliseconds(200))
+            .requestTimeout(Duration$.MODULE$.fromSeconds(10))
+            .sendBufferSize(sendBufferSize)
+            .recvBufferSize(recvBufferSize);
+
+        ClientId clientId = ClientId$.MODULE$.apply("dlog_loadtest_writer");
+
+        DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
+            .clientId(clientId)
+            .clientBuilder(clientBuilder)
+            .thriftmux(thriftmux)
+            .redirectBackoffStartMs(100)
+            .redirectBackoffMaxMs(500)
+            .requestTimeoutMs(10000)
+            .statsReceiver(statsReceiver)
+            .streamNameRegex("^" + streamPrefix + "_[0-9]+$")
+            .handshakeWithClientInfo(handshakeWithClientInfo)
+            .periodicHandshakeIntervalMs(TimeUnit.SECONDS.toMillis(30))
+            .periodicOwnershipSyncIntervalMs(TimeUnit.MINUTES.toMillis(5))
+            .periodicDumpOwnershipCache(true)
+            .handshakeTracing(true)
+            .serverRoutingServiceFinagleNameStr(routingServiceFinagleName)
+            .name("writer");
+
+        if (!finagleNames.isEmpty()) {
+            String local = finagleNames.get(0);
+            String[] remotes = new String[finagleNames.size() - 1];
+            finagleNames.subList(1, finagleNames.size()).toArray(remotes);
+
+            builder = builder.finagleNameStrs(local, remotes);
+        } else if (serverSets.length != 0){
+            ServerSet local = serverSets[0].getServerSet();
+            ServerSet[] remotes = new ServerSet[serverSets.length - 1];
+            for (int i = 1; i < serverSets.length; i++) {
+                remotes[i - 1] = serverSets[i].getServerSet();
+            }
+            builder = builder.serverSets(local, remotes);
+        } else {
+            builder = builder.uri(dlUri);
+        }
+
+        return builder.build();
+    }
+
+    ByteBuffer buildBuffer(long requestMillis, int messageSizeBytes) {
+        ByteBuffer data;
+        try {
+            data = ByteBuffer.wrap(Utils.generateMessage(requestMillis, messageSizeBytes));
+            return data;
+        } catch (TException e) {
+            LOG.error("Error generating message : ", e);
+            return null;
+        }
+    }
+
+    List<ByteBuffer> buildBufferList(int batchSize, long requestMillis, int messageSizeBytes) {
+        ArrayList<ByteBuffer> bufferList = new ArrayList<ByteBuffer>(batchSize);
+        for (int i = 0; i < batchSize; i++) {
+            ByteBuffer buf = buildBuffer(requestMillis, messageSizeBytes);
+            if (null == buf) {
+                return null;
+            }
+            bufferList.add(buf);
+        }
+        return bufferList;
+    }
+
+    class TimedRequestHandler implements FutureEventListener<DLSN>, Runnable {
+        final String streamName;
+        final long requestMillis;
+        DLSN dlsn = null;
+        Throwable cause = null;
+
+        TimedRequestHandler(String streamName,
+                            long requestMillis) {
+            this.streamName = streamName;
+            this.requestMillis = requestMillis;
+        }
+        @Override
+        public void onSuccess(DLSN value) {
+            dlsn = value;
+            executor.submit(this);
+        }
+        @Override
+        public void onFailure(Throwable cause) {
+            this.cause = cause;
+            executor.submit(this);
+        }
+
+        @Override
+        public void run() {
+            if (null != dlsn) {
+                requestStat.registerSuccessfulEvent(System.currentTimeMillis() - requestMillis);
+            } else {
+                LOG.error("Failed to publish to {} : ", streamName, cause);
+                requestStat.registerFailedEvent(System.currentTimeMillis() - requestMillis);
+                exceptionsLogger.getCounter(cause.getClass().getName()).inc();
+                if (cause instanceof DLException) {
+                    DLException dle = (DLException) cause;
+                    dlErrorCodeLogger.getCounter(dle.getCode().toString()).inc();
+                }
+            }
+        }
+    }
+
+    class Writer implements Runnable {
+
+        final int idx;
+        final DistributedLogClient dlc;
+        DistributedLogMultiStreamWriter writer = null;
+        final ShiftableRateLimiter limiter;
+
+        Writer(int idx) {
+            this.idx = idx;
+            this.dlc = buildDlogClient();
+            if (enableBatching) {
+                writer = DistributedLogMultiStreamWriter.newBuilder()
+                        .client(this.dlc)
+                        .streams(streamNames)
+                        .compressionCodec(CompressionCodec.Type.NONE)
+                        .flushIntervalMicros(batchFlushIntervalMicros)
+                        .bufferSize(batchBufferSize)
+                        .firstSpeculativeTimeoutMs(9000)
+                        .maxSpeculativeTimeoutMs(9000)
+                        .requestTimeoutMs(10000)
+                        .speculativeBackoffMultiplier(2)
+                        .build();
+            }
+            this.limiter = rateLimiter.duplicate();
+        }
+
+        @Override
+        public void run() {
+            LOG.info("Started writer {}.", idx);
+            while (running) {
+                this.limiter.getLimiter().acquire();
+                final String streamName = streamNames.get(random.nextInt(numStreams));
+                final long requestMillis = System.currentTimeMillis();
+                final ByteBuffer data = buildBuffer(requestMillis, messageSizeBytes);
+                if (null == data) {
+                    break;
+                }
+                if (null != writer) {
+                    writer.write(data).addEventListener(
+                            new TimedRequestHandler(streamName, requestMillis));
+                } else {
+                    dlc.write(streamName, data).addEventListener(
+                            new TimedRequestHandler(streamName, requestMillis));
+                }
+            }
+            if (null != writer) {
+                writer.close();
+            }
+            dlc.close();
+        }
+    }
+
+    class BulkWriter implements Runnable {
+
+        final int idx;
+        final DistributedLogClient dlc;
+
+        BulkWriter(int idx) {
+            this.idx = idx;
+            this.dlc = buildDlogClient();
+        }
+
+        @Override
+        public void run() {
+            LOG.info("Started writer {}.", idx);
+            while (running) {
+                rateLimiter.getLimiter().acquire(batchSize);
+                String streamName = streamNames.get(random.nextInt(numStreams));
+                final long requestMillis = System.currentTimeMillis();
+                final List<ByteBuffer> data = buildBufferList(batchSize, requestMillis, messageSizeBytes);
+                if (null == data) {
+                    break;
+                }
+                List<Future<DLSN>> results = dlc.writeBulk(streamName, data);
+                for (Future<DLSN> result : results) {
+                    result.addEventListener(new TimedRequestHandler(streamName, requestMillis));
+                }
+            }
+            dlc.close();
+        }
+    }
+
+    @Override
+    public void run() {
+        LOG.info("Starting writer (concurrency = {}, prefix = {}, batchSize = {})",
+                 new Object[] { writeConcurrency, streamPrefix, batchSize });
+        try {
+            for (int i = 0; i < writeConcurrency; i++) {
+                Runnable writer = null;
+                if (batchSize > 0) {
+                    writer = new BulkWriter(i);
+                } else {
+                    writer = new Writer(i);
+                }
+                executorService.submit(writer);
+            }
+        } catch (Throwable t) {
+            LOG.error("Unhandled exception caught", t);
+        }
+    }
+}
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/package-info.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/package-info.java
new file mode 100644
index 0000000..7e87644
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Benchmarks for distributedlog.
+ */
+package org.apache.distributedlog.benchmark;
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AbstractReaderBenchmark.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AbstractReaderBenchmark.java
new file mode 100644
index 0000000..a1f1f9f
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AbstractReaderBenchmark.java
@@ -0,0 +1,70 @@
+/**
+ * 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.benchmark.stream;
+
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.commons.cli.CommandLine;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+abstract class AbstractReaderBenchmark extends StreamBenchmark {
+
+    private static final Logger logger = LoggerFactory.getLogger(SyncReaderBenchmark.class);
+
+    protected ReadMode readMode = ReadMode.LATEST;
+    protected long fromTxId = DistributedLogConstants.INVALID_TXID;
+    protected long rewindMs = 0L;
+    protected int batchSize = 1;
+
+    protected AbstractReaderBenchmark() {
+        options.addOption("t", "tx-id", true,
+            "Transaction ID to start read from when reading in mode 'position'");
+        options.addOption("r", "rewind", true,
+            "Time to rewind back to read from when reading in mode 'rewind' (in milliseconds)");
+        options.addOption("m", "mode", true,
+            "Read Mode : [oldest, latest, rewind, position]");
+        options.addOption("b", "batch-size", true, "Read batch size");
+    }
+
+    @Override
+    protected void parseCommandLine(CommandLine cmdline) {
+        if (cmdline.hasOption("m")) {
+            String mode = cmdline.getOptionValue("m");
+            try {
+                readMode = ReadMode.valueOf(mode.toUpperCase());
+            } catch (IllegalArgumentException iae) {
+                logger.error("Invalid read mode {}.", mode);
+                printUsage();
+                System.exit(0);
+            }
+        } else {
+            printUsage();
+            System.exit(0);
+        }
+        if (cmdline.hasOption("t")) {
+            fromTxId = Long.parseLong(cmdline.getOptionValue("t"));
+        }
+        if (cmdline.hasOption("r")) {
+            rewindMs = Long.parseLong(cmdline.getOptionValue("r"));
+        }
+        if (cmdline.hasOption("b")) {
+            batchSize = Integer.parseInt(cmdline.getOptionValue("b"));
+        }
+        logger.info("Start reading from transaction id {}, rewind {} ms.", fromTxId, rewindMs);
+    }
+}
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
new file mode 100644
index 0000000..4930b8a
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
@@ -0,0 +1,158 @@
+/**
+ * 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.benchmark.stream;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.AsyncLogReader;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.util.FutureUtils;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+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;
+
+/**
+ * Benchmark on {@link org.apache.distributedlog.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) {
+        DistributedLogManager dlm = null;
+        while (null == dlm) {
+            try {
+                dlm = namespace.openLog(streamName);
+            } catch (IOException ioe) {
+                logger.warn("Failed to create dlm for stream {} : ", streamName, ioe);
+            }
+            if (null == dlm) {
+                try {
+                    TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+                } catch (InterruptedException e) {
+                    logger.warn("Interrupted from sleep while creating dlm for stream {} : ",
+                        streamName, e);
+                }
+            }
+        }
+        logger.info("Created dlm for stream {}.", streamName);
+
+        // Stats
+        OpStatsLogger openReaderStats = statsLogger.getOpStatsLogger("open_reader");
+        OpStatsLogger blockingReadStats = statsLogger.getOpStatsLogger("blocking_read");
+        Counter readCounter = statsLogger.getCounter("reads");
+
+        AsyncLogReader reader = null;
+        DLSN lastDLSN = null;
+        Long lastTxId = null;
+        while (null == reader) {
+            // initialize the last txid
+            if (null == lastTxId) {
+                switch (readMode) {
+                    case OLDEST:
+                        lastTxId = 0L;
+                        lastDLSN = DLSN.InitialDLSN;
+                        break;
+                    case LATEST:
+                        lastTxId = Long.MAX_VALUE;
+                        try {
+                            lastDLSN = dlm.getLastDLSN();
+                        } catch (IOException ioe) {
+                            continue;
+                        }
+                        break;
+                    case REWIND:
+                        lastTxId = System.currentTimeMillis() - rewindMs;
+                        lastDLSN = null;
+                        break;
+                    case POSITION:
+                        lastTxId = fromTxId;
+                        lastDLSN = null;
+                        break;
+                    default:
+                        logger.warn("Unsupported mode {}", readMode);
+                        printUsage();
+                        System.exit(0);
+                        break;
+                }
+                logger.info("Reading from transaction id = {}, dlsn = {}", lastTxId, lastDLSN);
+            }
+            // Open the reader
+            Stopwatch stopwatch = Stopwatch.createStarted();
+            try {
+                if (null == lastDLSN) {
+                    reader = FutureUtils.result(dlm.openAsyncLogReader(lastTxId));
+                } else {
+                    reader = FutureUtils.result(dlm.openAsyncLogReader(lastDLSN));
+                }
+                long elapsedMs = stopwatch.elapsed(TimeUnit.MICROSECONDS);
+                openReaderStats.registerSuccessfulEvent(elapsedMs);
+                logger.info("It took {} ms to position the reader to transaction id = {}, dlsn = {}",
+                        lastTxId, lastDLSN);
+            } catch (IOException 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 });
+            }
+            if (null == reader) {
+                try {
+                    TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+                } catch (InterruptedException e) {
+                    logger.warn("Interrupted from sleep after reader was reassigned null for stream {} : ",
+                        streamName, e);
+                }
+                continue;
+            }
+            List<LogRecordWithDLSN> records;
+            stopwatch = Stopwatch.createUnstarted();
+            while (true) {
+                try {
+                    stopwatch.start();
+                    records = FutureUtils.result(reader.readBulk(batchSize));
+                    long elapsedMicros = stopwatch.stop().elapsed(TimeUnit.MICROSECONDS);
+                    blockingReadStats.registerSuccessfulEvent(elapsedMicros);
+                    if (!records.isEmpty()) {
+                        readCounter.add(records.size());
+                        LogRecordWithDLSN lastRecord = records.get(records.size() - 1);
+                        lastTxId = lastRecord.getTransactionId();
+                        lastDLSN = lastRecord.getDlsn();
+                    }
+                    stopwatch.reset();
+                } catch (IOException e) {
+                    logger.warn("Encountered reading record from stream {} : ", streamName, e);
+                    reader = null;
+                    break;
+                }
+            }
+            try {
+                TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+            } catch (InterruptedException e) {
+                logger.warn("Interrupted from sleep while creating reader for stream {} : ",
+                    streamName, e);
+            }
+        }
+    }
+}
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerBatchReader.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerBatchReader.java
new file mode 100644
index 0000000..b115192
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerBatchReader.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.benchmark.stream;
+
+import java.util.Enumeration;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Read ledgers in batches.
+ */
+public class LedgerBatchReader implements Runnable {
+
+    private static final Logger logger = LoggerFactory.getLogger(LedgerBatchReader.class);
+
+    private final LedgerHandle lh;
+    private final ReadEntryListener readEntryListener;
+    private final int batchSize;
+
+    public LedgerBatchReader(LedgerHandle lh,
+                             ReadEntryListener readEntryListener,
+                             int batchSize) {
+        this.lh = lh;
+        this.batchSize = batchSize;
+        this.readEntryListener = readEntryListener;
+    }
+
+    @Override
+    public void run() {
+        long lac = lh.getLastAddConfirmed();
+
+        long entryId = 0L;
+
+        while (entryId <= lac) {
+            long startEntryId = entryId;
+            long endEntryId = Math.min(startEntryId + batchSize - 1, lac);
+
+            Enumeration<LedgerEntry> entries = null;
+            while (null == entries) {
+                try {
+                    entries = lh.readEntries(startEntryId, endEntryId);
+                } catch (BKException bke) {
+                    logger.error("Encountered exceptions on reading [ {} - {} ] ",
+                            new Object[] { startEntryId, endEntryId, bke });
+                } catch (InterruptedException ie) {
+                    Thread.currentThread().interrupt();
+                    break;
+                }
+            }
+            if (null == entries) {
+                break;
+            }
+
+            while (entries.hasMoreElements()) {
+                LedgerEntry entry = entries.nextElement();
+                readEntryListener.onEntryComplete(BKException.Code.OK, lh, entry, null);
+            }
+
+            entryId = endEntryId + 1;
+        }
+
+    }
+}
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
new file mode 100644
index 0000000..489e5af
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerReadBenchmark.java
@@ -0,0 +1,151 @@
+/**
+ * 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.benchmark.stream;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.BookKeeperClientBuilder;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+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;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Benchmark ledger reading.
+ */
+public class LedgerReadBenchmark extends AbstractReaderBenchmark {
+
+    private static final Logger logger = LoggerFactory.getLogger(AsyncReaderBenchmark.class);
+
+    @Override
+    protected void benchmark(DistributedLogNamespace namespace, String logName, StatsLogger statsLogger) {
+        DistributedLogManager dlm = null;
+        while (null == dlm) {
+            try {
+                dlm = namespace.openLog(streamName);
+            } catch (IOException ioe) {
+                logger.warn("Failed to create dlm for stream {} : ", streamName, ioe);
+            }
+            if (null == dlm) {
+                try {
+                    TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+                } catch (InterruptedException e) {
+                    logger.warn("Interrupted from sleep while creating dlm for stream {} : ",
+                        streamName, e);
+                }
+            }
+        }
+        logger.info("Created dlm for stream {}.", streamName);
+
+        List<LogSegmentMetadata> segments = null;
+        while (null == segments) {
+            try {
+                segments = dlm.getLogSegments();
+            } catch (IOException ioe) {
+                logger.warn("Failed to get log segments for stream {} : ", streamName, ioe);
+            }
+            if (null == segments) {
+                try {
+                    TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+                } catch (InterruptedException e) {
+                    logger.warn("Interrupted from sleep while geting log segments for stream {} : ",
+                        streamName, e);
+                }
+            }
+        }
+
+        final Counter readCounter = statsLogger.getCounter("reads");
+
+        logger.info("Reading from log segments : {}", segments);
+
+        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
+                .uri(uri)
+                .name("benchmark-zkc")
+                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                .zkAclId(null)
+                .build();
+        BKDLConfig bkdlConfig;
+        try {
+            bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
+        } catch (IOException e) {
+            return;
+        }
+
+        BookKeeper bk;
+        try {
+            bk = BookKeeperClientBuilder.newBuilder()
+                    .name("benchmark-bkc")
+                    .dlConfig(conf)
+                    .zkServers(bkdlConfig.getBkZkServersForReader())
+                    .ledgersPath(bkdlConfig.getBkLedgersPath())
+                    .build()
+                    .get();
+        } catch (IOException e) {
+            return;
+        }
+
+        final int readConcurrency = conf.getInt("ledger_read_concurrency", 1000);
+        boolean streamRead = conf.getBoolean("ledger_stream_read", true);
+        try {
+            for (LogSegmentMetadata segment : segments) {
+                Stopwatch stopwatch = Stopwatch.createStarted();
+                long lid = segment.getLogSegmentId();
+                LedgerHandle lh = bk.openLedgerNoRecovery(
+                        lid, BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
+                logger.info("It took {} ms to open log segment {}",
+                    new Object[] { stopwatch.elapsed(TimeUnit.MILLISECONDS), (lh.getLastAddConfirmed() + 1), segment });
+                stopwatch.reset().start();
+                Runnable reader;
+                if (streamRead) {
+                    reader = new LedgerStreamReader(lh, new BookkeeperInternalCallbacks.ReadEntryListener() {
+                        @Override
+                        public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object ctx) {
+                            readCounter.inc();
+                        }
+                    }, readConcurrency);
+                } else {
+                    reader = new LedgerStreamReader(lh, new BookkeeperInternalCallbacks.ReadEntryListener() {
+                        @Override
+                        public void onEntryComplete(int rc, LedgerHandle lh, LedgerEntry entry, Object ctx) {
+                            readCounter.inc();
+                        }
+                    }, readConcurrency);
+                }
+                reader.run();
+                logger.info("It took {} ms to complete reading {} entries from log segment {}",
+                    new Object[] { stopwatch.elapsed(TimeUnit.MILLISECONDS), (lh.getLastAddConfirmed() + 1), segment });
+            }
+        } catch (Exception e) {
+            logger.error("Error on reading bk ", e);
+        }
+    }
+}
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerStreamReader.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerStreamReader.java
new file mode 100644
index 0000000..11c3482
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerStreamReader.java
@@ -0,0 +1,131 @@
+/**
+ * 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.benchmark.stream;
+
+import java.util.Enumeration;
+import java.util.concurrent.ConcurrentLinkedQueue;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicLong;
+import org.apache.bookkeeper.client.AsyncCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks.ReadEntryListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Reading ledger in a streaming way.
+ */
+public class LedgerStreamReader implements Runnable {
+
+    private static final Logger logger = LoggerFactory.getLogger(LedgerStreamReader.class);
+
+    class PendingReadRequest implements AsyncCallback.ReadCallback {
+
+        final long entryId;
+        boolean isDone = false;
+        int rc;
+        LedgerEntry entry = null;
+
+        PendingReadRequest(long entryId) {
+            this.entryId = entryId;
+        }
+
+        void read() {
+            lh.asyncReadEntries(entryId, entryId, this, null);
+        }
+
+        void complete(ReadEntryListener listener) {
+            listener.onEntryComplete(rc, lh, entry, null);
+        }
+
+        @Override
+        public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> enumeration, Object ctx) {
+            this.rc = rc;
+            if (BKException.Code.OK == rc && enumeration.hasMoreElements()) {
+                entry = enumeration.nextElement();
+            } else {
+                entry = null;
+            }
+            isDone = true;
+            // construct a new read request
+            long nextEntry = nextReadEntry.getAndIncrement();
+            if (nextEntry <= lac) {
+                PendingReadRequest nextRead =
+                        new PendingReadRequest(nextEntry);
+                pendingReads.add(nextRead);
+                nextRead.read();
+            }
+            triggerCallbacks();
+        }
+    }
+
+    private final LedgerHandle lh;
+    private final long lac;
+    private final ReadEntryListener readEntryListener;
+    private final int concurrency;
+    private final AtomicLong nextReadEntry = new AtomicLong(0);
+    private final CountDownLatch done = new CountDownLatch(1);
+    private final ConcurrentLinkedQueue<PendingReadRequest> pendingReads =
+            new ConcurrentLinkedQueue<PendingReadRequest>();
+
+    public LedgerStreamReader(LedgerHandle lh,
+                              ReadEntryListener readEntryListener,
+                              int concurrency) {
+        this.lh = lh;
+        this.lac = lh.getLastAddConfirmed();
+        this.readEntryListener = readEntryListener;
+        this.concurrency = concurrency;
+        for (int i = 0; i < concurrency; i++) {
+            long entryId = nextReadEntry.getAndIncrement();
+            if (entryId > lac) {
+                break;
+            }
+            PendingReadRequest request = new PendingReadRequest(entryId);
+            pendingReads.add(request);
+            request.read();
+        }
+        if (pendingReads.isEmpty()) {
+            done.countDown();
+        }
+    }
+
+    synchronized void triggerCallbacks() {
+        PendingReadRequest request;
+        while ((request = pendingReads.peek()) != null) {
+            if (!request.isDone) {
+                break;
+            }
+            pendingReads.remove();
+            request.complete(readEntryListener);
+        }
+        if (pendingReads.isEmpty()) {
+            done.countDown();
+        }
+    }
+
+    @Override
+    public void run() {
+        try {
+            done.await();
+        } catch (InterruptedException e) {
+            logger.info("Interrupted on stream reading ledger {} : ", lh.getId(), e);
+        }
+    }
+}
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/ReadMode.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/ReadMode.java
new file mode 100644
index 0000000..ea5ed36
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/ReadMode.java
@@ -0,0 +1,28 @@
+/**
+ * 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.benchmark.stream;
+
+/**
+ * The read mode for streaming read benchmark.
+ */
+public enum ReadMode {
+    OLDEST,
+    LATEST,
+    REWIND,
+    POSITION
+}
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
new file mode 100644
index 0000000..d3083ca
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/StreamBenchmark.java
@@ -0,0 +1,138 @@
+/**
+ * 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.benchmark.stream;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import java.io.File;
+import java.net.URI;
+import org.apache.bookkeeper.stats.NullStatsProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.cli.BasicParser;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Benchmark Streams.
+ */
+public abstract class StreamBenchmark {
+
+    private static final Logger logger = LoggerFactory.getLogger(StreamBenchmark.class);
+
+    private static final String USAGE = "StreamBenchmark <benchmark-class> [options]";
+
+    protected final Options options = new Options();
+    protected URI uri;
+    protected DistributedLogConfiguration conf;
+    protected StatsProvider statsProvider;
+    protected String streamName;
+
+    protected StreamBenchmark() {
+        options.addOption("c", "conf", true, "Configuration File");
+        options.addOption("u", "uri", true, "DistributedLog URI");
+        options.addOption("p", "stats-provider", true, "Stats Provider");
+        options.addOption("s", "stream", true, "Stream Name");
+        options.addOption("h", "help", false, "Print usage.");
+    }
+
+    protected Options getOptions() {
+        return options;
+    }
+
+    protected void printUsage() {
+        HelpFormatter hf = new HelpFormatter();
+        hf.printHelp(USAGE, options);
+    }
+
+    protected void parseCommandLine(String[] args)
+            throws Exception {
+        BasicParser parser = new BasicParser();
+        CommandLine cmdline = parser.parse(options, args);
+        if (cmdline.hasOption("h")) {
+            printUsage();
+            System.exit(0);
+        }
+        if (cmdline.hasOption("u")) {
+            this.uri = URI.create(cmdline.getOptionValue("u"));
+        } else {
+            printUsage();
+            System.exit(0);
+        }
+        this.conf = new DistributedLogConfiguration();
+        if (cmdline.hasOption("c")) {
+            String configFile = cmdline.getOptionValue("c");
+            this.conf.loadConf(new File(configFile).toURI().toURL());
+        }
+        if (cmdline.hasOption("p")) {
+            statsProvider = ReflectionUtils.newInstance(cmdline.getOptionValue("p"), StatsProvider.class);
+        } else {
+            statsProvider = new NullStatsProvider();
+        }
+        if (cmdline.hasOption("s")) {
+            this.streamName = cmdline.getOptionValue("s");
+        } else {
+            printUsage();
+            System.exit(0);
+        }
+        parseCommandLine(cmdline);
+    }
+
+    protected abstract void parseCommandLine(CommandLine cmdline);
+
+    protected void run(String[] args) throws Exception {
+        logger.info("Parsing arguments for benchmark : {}", args);
+        // parse command line
+        parseCommandLine(args);
+        statsProvider.start(conf);
+        // run the benchmark
+        StatsLogger statsLogger = statsProvider.getStatsLogger("dl");
+        DistributedLogNamespace namespace =
+                DistributedLogNamespaceBuilder.newBuilder()
+                        .conf(conf)
+                        .uri(uri)
+                        .statsLogger(statsLogger)
+                        .build();
+        try {
+            benchmark(namespace, streamName, statsProvider.getStatsLogger("benchmark"));
+        } finally {
+            namespace.close();
+            statsProvider.stop();
+        }
+    }
+
+    protected abstract void benchmark(DistributedLogNamespace namespace,
+                                      String logName,
+                                      StatsLogger statsLogger);
+
+    public static void main(String[] args) throws Exception {
+        if (args.length <= 0) {
+            System.err.println(USAGE);
+            return;
+        }
+        String benchmarkClassName = args[0];
+        StreamBenchmark benchmark = ReflectionUtils.newInstance(
+                benchmarkClassName, StreamBenchmark.class);
+        benchmark.run(args);
+    }
+}
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
new file mode 100644
index 0000000..4abb317
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/SyncReaderBenchmark.java
@@ -0,0 +1,164 @@
+/**
+ * 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.benchmark.stream;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogReader;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+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;
+
+/**
+ * Benchmark on {@link org.apache.distributedlog.LogReader} reading from a stream.
+ */
+public class SyncReaderBenchmark extends AbstractReaderBenchmark {
+
+    private static final Logger logger = LoggerFactory.getLogger(SyncReaderBenchmark.class);
+
+    public SyncReaderBenchmark() {}
+
+    @Override
+    protected void benchmark(DistributedLogNamespace namespace, String streamName, StatsLogger statsLogger) {
+        DistributedLogManager dlm = null;
+        while (null == dlm) {
+            try {
+                dlm = namespace.openLog(streamName);
+            } catch (IOException ioe) {
+                logger.warn("Failed to create dlm for stream {} : ", streamName, ioe);
+            }
+            if (null == dlm) {
+                try {
+                    TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+                } catch (InterruptedException e) {
+                    logger.warn("Interrupted from sleep while creating dlm for stream {} : ",
+                        streamName, e);
+                }
+            }
+        }
+        OpStatsLogger openReaderStats = statsLogger.getOpStatsLogger("open_reader");
+        OpStatsLogger nonBlockingReadStats = statsLogger.getOpStatsLogger("non_blocking_read");
+        OpStatsLogger blockingReadStats = statsLogger.getOpStatsLogger("blocking_read");
+        Counter nullReadCounter = statsLogger.getCounter("null_read");
+
+        logger.info("Created dlm for stream {}.", streamName);
+        LogReader reader = null;
+        Long lastTxId = null;
+        while (null == reader) {
+            // initialize the last txid
+            if (null == lastTxId) {
+                switch (readMode) {
+                    case OLDEST:
+                        lastTxId = 0L;
+                        break;
+                    case LATEST:
+                        try {
+                            lastTxId = dlm.getLastTxId();
+                        } catch (IOException ioe) {
+                            continue;
+                        }
+                        break;
+                    case REWIND:
+                        lastTxId = System.currentTimeMillis() - rewindMs;
+                        break;
+                    case POSITION:
+                        lastTxId = fromTxId;
+                        break;
+                    default:
+                        logger.warn("Unsupported mode {}", readMode);
+                        printUsage();
+                        System.exit(0);
+                        break;
+                }
+                logger.info("Reading from transaction id {}", lastTxId);
+            }
+            // Open the reader
+            Stopwatch stopwatch = Stopwatch.createStarted();
+            try {
+                reader = dlm.getInputStream(lastTxId);
+                long elapsedMs = stopwatch.elapsed(TimeUnit.MICROSECONDS);
+                openReaderStats.registerSuccessfulEvent(elapsedMs);
+                logger.info("It took {} ms to position the reader to transaction id {}", lastTxId);
+            } catch (IOException ioe) {
+                openReaderStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+                logger.warn("Failed to create reader for stream {} reading from {}.", streamName, lastTxId);
+            }
+            if (null == reader) {
+                try {
+                    TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+                } catch (InterruptedException e) {
+                    logger.warn("Interrupted from sleep after reader was reassigned null for stream {} : ",
+                        streamName, e);
+                }
+                continue;
+            }
+
+            // read loop
+
+            LogRecord record;
+            boolean nonBlocking = false;
+            stopwatch = Stopwatch.createUnstarted();
+            long numCatchupReads = 0L;
+            long numCatchupBytes = 0L;
+            Stopwatch catchupStopwatch = Stopwatch.createStarted();
+            while (true) {
+                try {
+                    stopwatch.start();
+                    record = reader.readNext(nonBlocking);
+                    if (null != record) {
+                        long elapsedMicros = stopwatch.stop().elapsed(TimeUnit.MICROSECONDS);
+                        if (nonBlocking) {
+                            nonBlockingReadStats.registerSuccessfulEvent(elapsedMicros);
+                        } else {
+                            numCatchupBytes += record.getPayload().length;
+                            ++numCatchupReads;
+                            blockingReadStats.registerSuccessfulEvent(elapsedMicros);
+                        }
+                        lastTxId = record.getTransactionId();
+                    } else {
+                        nullReadCounter.inc();
+                    }
+                    if (null == record && !nonBlocking) {
+                        nonBlocking = true;
+                        catchupStopwatch.stop();
+                        logger.info("Catchup {} records (total {} bytes) in {} milliseconds",
+                                new Object[] { numCatchupReads, numCatchupBytes,
+                                    stopwatch.elapsed(TimeUnit.MILLISECONDS) });
+                    }
+                    stopwatch.reset();
+                } catch (IOException e) {
+                    logger.warn("Encountered reading record from stream {} : ", streamName, e);
+                    reader = null;
+                    break;
+                }
+            }
+            try {
+                TimeUnit.MILLISECONDS.sleep(conf.getZKSessionTimeoutMilliseconds());
+            } catch (InterruptedException e) {
+                logger.warn("Interrupted from sleep while creating reader for stream {} : ",
+                    streamName, e);
+            }
+        }
+    }
+}
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/package-info.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/package-info.java
new file mode 100644
index 0000000..b95a40f
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Stream level benchmarks.
+ */
+package org.apache.distributedlog.benchmark.stream;
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/ShiftableRateLimiter.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/ShiftableRateLimiter.java
new file mode 100644
index 0000000..03c561c
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/ShiftableRateLimiter.java
@@ -0,0 +1,71 @@
+/**
+ * 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.benchmark.utils;
+
+import com.google.common.util.concurrent.RateLimiter;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * A wrapper over rate limiter.
+ */
+public class ShiftableRateLimiter implements Runnable {
+
+    private final RateLimiter rateLimiter;
+    private final ScheduledExecutorService executor;
+    private final double initialRate, maxRate, changeRate;
+    private final long changeInterval;
+    private final TimeUnit changeIntervalUnit;
+    private double nextRate;
+
+    public ShiftableRateLimiter(double initialRate,
+                                double maxRate,
+                                double changeRate,
+                                long changeInterval,
+                                TimeUnit changeIntervalUnit) {
+        this.initialRate = initialRate;
+        this.maxRate = maxRate;
+        this.changeRate = changeRate;
+        this.nextRate = initialRate;
+        this.changeInterval = changeInterval;
+        this.changeIntervalUnit = changeIntervalUnit;
+        this.rateLimiter = RateLimiter.create(initialRate);
+        this.executor = Executors.newSingleThreadScheduledExecutor();
+        this.executor.scheduleAtFixedRate(this, changeInterval, changeInterval, changeIntervalUnit);
+    }
+
+    public ShiftableRateLimiter duplicate() {
+        return new ShiftableRateLimiter(
+                initialRate,
+                maxRate,
+                changeRate,
+                changeInterval,
+                changeIntervalUnit);
+    }
+
+    @Override
+    public void run() {
+        this.nextRate = Math.min(nextRate + changeRate, maxRate);
+        this.rateLimiter.setRate(nextRate);
+    }
+
+    public RateLimiter getLimiter() {
+        return this.rateLimiter;
+    }
+}
diff --git a/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/package-info.java b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/package-info.java
new file mode 100644
index 0000000..c650bab
--- /dev/null
+++ b/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Utils for benchmarking.
+ */
+package org.apache.distributedlog.benchmark.utils;
diff --git a/distributedlog-benchmark/src/main/resources/findbugsExclude.xml b/distributedlog-benchmark/src/main/resources/findbugsExclude.xml
index b7a1ecb..0ab2b6b 100644
--- a/distributedlog-benchmark/src/main/resources/findbugsExclude.xml
+++ b/distributedlog-benchmark/src/main/resources/findbugsExclude.xml
@@ -18,6 +18,6 @@
 <FindBugsFilter>
   <Match>
     <!-- generated code, we can't be held responsible for findbugs in it //-->
-    <Class name="~com\.twitter\.distributedlog\.benchmark\.thrift.*" />
+    <Class name="~org\.apache\.distributedlog\.benchmark\.thrift.*" />
   </Match>
 </FindBugsFilter>
diff --git a/distributedlog-benchmark/src/main/thrift/loadtest.thrift b/distributedlog-benchmark/src/main/thrift/loadtest.thrift
index 6d98cec..48c5d5a 100644
--- a/distributedlog-benchmark/src/main/thrift/loadtest.thrift
+++ b/distributedlog-benchmark/src/main/thrift/loadtest.thrift
@@ -16,7 +16,7 @@
  * limitations under the License.
  */
 
-namespace java com.twitter.distributedlog.benchmark.thrift
+namespace java org.apache.distributedlog.benchmark.thrift
 
 struct Message {
     1: i64 publishTime;
diff --git a/distributedlog-client/pom.xml b/distributedlog-client/pom.xml
index aad5093..f09caf1 100644
--- a/distributedlog-client/pom.xml
+++ b/distributedlog-client/pom.xml
@@ -137,7 +137,7 @@
           <properties>
             <property>
               <name>listener</name>
-              <value>com.twitter.distributedlog.TimedOutTestsListener</value>
+              <value>org.apache.distributedlog.TimedOutTestsListener</value>
             </property>
           </properties>
         </configuration>
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ClientConfig.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ClientConfig.java
deleted file mode 100644
index de74f5a..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ClientConfig.java
+++ /dev/null
@@ -1,187 +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 com.twitter.distributedlog.client;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Client Config.
- */
-public class ClientConfig {
-    int redirectBackoffStartMs = 25;
-    int redirectBackoffMaxMs = 100;
-    int maxRedirects = -1;
-    int requestTimeoutMs = -1;
-    boolean thriftmux = false;
-    boolean streamFailfast = false;
-    String streamNameRegex = ".*";
-    boolean handshakeWithClientInfo = true;
-    long periodicHandshakeIntervalMs = TimeUnit.MINUTES.toMillis(5);
-    long periodicOwnershipSyncIntervalMs = TimeUnit.MINUTES.toMillis(5);
-    boolean periodicDumpOwnershipCacheEnabled = false;
-    long periodicDumpOwnershipCacheIntervalMs = TimeUnit.MINUTES.toMillis(10);
-    boolean enableHandshakeTracing = false;
-    boolean enableChecksum = true;
-
-    public ClientConfig setMaxRedirects(int maxRedirects) {
-        this.maxRedirects = maxRedirects;
-        return this;
-    }
-
-    public int getMaxRedirects() {
-        return this.maxRedirects;
-    }
-
-    public ClientConfig setRequestTimeoutMs(int timeoutInMillis) {
-        this.requestTimeoutMs = timeoutInMillis;
-        return this;
-    }
-
-    public int getRequestTimeoutMs() {
-        return this.requestTimeoutMs;
-    }
-
-    public ClientConfig setRedirectBackoffStartMs(int ms) {
-        this.redirectBackoffStartMs = ms;
-        return this;
-    }
-
-    public int getRedirectBackoffStartMs() {
-        return this.redirectBackoffStartMs;
-    }
-
-    public ClientConfig setRedirectBackoffMaxMs(int ms) {
-        this.redirectBackoffMaxMs = ms;
-        return this;
-    }
-
-    public int getRedirectBackoffMaxMs() {
-        return this.redirectBackoffMaxMs;
-    }
-
-    public ClientConfig setThriftMux(boolean enabled) {
-        this.thriftmux = enabled;
-        return this;
-    }
-
-    public boolean getThriftMux() {
-        return this.thriftmux;
-    }
-
-    public ClientConfig setStreamFailfast(boolean enabled) {
-        this.streamFailfast = enabled;
-        return this;
-    }
-
-    public boolean getStreamFailfast() {
-        return this.streamFailfast;
-    }
-
-    public ClientConfig setStreamNameRegex(String nameRegex) {
-        checkNotNull(nameRegex);
-        this.streamNameRegex = nameRegex;
-        return this;
-    }
-
-    public String getStreamNameRegex() {
-        return this.streamNameRegex;
-    }
-
-    public ClientConfig setHandshakeWithClientInfo(boolean enabled) {
-        this.handshakeWithClientInfo = enabled;
-        return this;
-    }
-
-    public boolean getHandshakeWithClientInfo() {
-        return this.handshakeWithClientInfo;
-    }
-
-    public ClientConfig setPeriodicHandshakeIntervalMs(long intervalMs) {
-        this.periodicHandshakeIntervalMs = intervalMs;
-        return this;
-    }
-
-    public long getPeriodicHandshakeIntervalMs() {
-        return this.periodicHandshakeIntervalMs;
-    }
-
-    public ClientConfig setPeriodicOwnershipSyncIntervalMs(long intervalMs) {
-        this.periodicOwnershipSyncIntervalMs = intervalMs;
-        return this;
-    }
-
-    public long getPeriodicOwnershipSyncIntervalMs() {
-        return this.periodicOwnershipSyncIntervalMs;
-    }
-
-    public ClientConfig setPeriodicDumpOwnershipCacheEnabled(boolean enabled) {
-        this.periodicDumpOwnershipCacheEnabled = enabled;
-        return this;
-    }
-
-    public boolean isPeriodicDumpOwnershipCacheEnabled() {
-        return this.periodicDumpOwnershipCacheEnabled;
-    }
-
-    public ClientConfig setPeriodicDumpOwnershipCacheIntervalMs(long intervalMs) {
-        this.periodicDumpOwnershipCacheIntervalMs = intervalMs;
-        return this;
-    }
-
-    public long getPeriodicDumpOwnershipCacheIntervalMs() {
-        return this.periodicDumpOwnershipCacheIntervalMs;
-    }
-
-    public ClientConfig setHandshakeTracingEnabled(boolean enabled) {
-        this.enableHandshakeTracing = enabled;
-        return this;
-    }
-
-    public boolean isHandshakeTracingEnabled() {
-        return this.enableHandshakeTracing;
-    }
-
-    public ClientConfig setChecksumEnabled(boolean enabled) {
-        this.enableChecksum = enabled;
-        return this;
-    }
-
-    public boolean isChecksumEnabled() {
-        return this.enableChecksum;
-    }
-
-    public static ClientConfig newConfig(ClientConfig config) {
-        ClientConfig newConfig = new ClientConfig();
-        newConfig.setMaxRedirects(config.getMaxRedirects())
-                 .setRequestTimeoutMs(config.getRequestTimeoutMs())
-                 .setRedirectBackoffStartMs(config.getRedirectBackoffStartMs())
-                 .setRedirectBackoffMaxMs(config.getRedirectBackoffMaxMs())
-                 .setThriftMux(config.getThriftMux())
-                 .setStreamFailfast(config.getStreamFailfast())
-                 .setStreamNameRegex(config.getStreamNameRegex())
-                 .setHandshakeWithClientInfo(config.getHandshakeWithClientInfo())
-                 .setPeriodicHandshakeIntervalMs(config.getPeriodicHandshakeIntervalMs())
-                 .setPeriodicDumpOwnershipCacheEnabled(config.isPeriodicDumpOwnershipCacheEnabled())
-                 .setPeriodicDumpOwnershipCacheIntervalMs(config.getPeriodicDumpOwnershipCacheIntervalMs())
-                 .setHandshakeTracingEnabled(config.isHandshakeTracingEnabled())
-                 .setChecksumEnabled(config.isChecksumEnabled());
-        return newConfig;
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogClientImpl.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogClientImpl.java
deleted file mode 100644
index 1077cd0..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogClientImpl.java
+++ /dev/null
@@ -1,1200 +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 com.twitter.distributedlog.client;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecordSetBuffer;
-import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
-import com.twitter.distributedlog.client.ownership.OwnershipCache;
-import com.twitter.distributedlog.client.proxy.ClusterClient;
-import com.twitter.distributedlog.client.proxy.HostProvider;
-import com.twitter.distributedlog.client.proxy.ProxyClient;
-import com.twitter.distributedlog.client.proxy.ProxyClientManager;
-import com.twitter.distributedlog.client.proxy.ProxyListener;
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.distributedlog.client.routing.RoutingService;
-import com.twitter.distributedlog.client.routing.RoutingService.RoutingContext;
-import com.twitter.distributedlog.client.stats.ClientStats;
-import com.twitter.distributedlog.client.stats.OpStats;
-import com.twitter.distributedlog.exceptions.DLClientClosedException;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.ServiceUnavailableException;
-import com.twitter.distributedlog.exceptions.StreamUnavailableException;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.thrift.service.BulkWriteResponse;
-import com.twitter.distributedlog.thrift.service.HeartbeatOptions;
-import com.twitter.distributedlog.thrift.service.ResponseHeader;
-import com.twitter.distributedlog.thrift.service.ServerInfo;
-import com.twitter.distributedlog.thrift.service.ServerStatus;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.distributedlog.thrift.service.WriteContext;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.util.ProtocolUtils;
-import com.twitter.finagle.CancelledRequestException;
-import com.twitter.finagle.ConnectionFailedException;
-import com.twitter.finagle.Failure;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.RequestTimeoutException;
-import com.twitter.finagle.ServiceException;
-import com.twitter.finagle.ServiceTimeoutException;
-import com.twitter.finagle.WriteException;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.finagle.thrift.ClientId;
-import com.twitter.util.Duration;
-import com.twitter.util.Function;
-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 java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-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.locks.ReentrantReadWriteLock;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-import org.apache.thrift.TApplicationException;
-import org.jboss.netty.channel.ChannelException;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.jboss.netty.util.Timeout;
-import org.jboss.netty.util.TimerTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.collection.Seq;
-import scala.runtime.AbstractFunction1;
-
-
-/**
- * Implementation of distributedlog client.
- */
-public class DistributedLogClientImpl implements DistributedLogClient, MonitorServiceClient,
-        RoutingService.RoutingListener, ProxyListener, HostProvider {
-
-    private static final Logger logger = LoggerFactory.getLogger(DistributedLogClientImpl.class);
-
-    private final String clientName;
-    private final ClientId clientId;
-    private final ClientConfig clientConfig;
-    private final RoutingService routingService;
-    private final ProxyClient.Builder clientBuilder;
-    private final boolean streamFailfast;
-    private final Pattern streamNameRegexPattern;
-
-    // Timer
-    private final HashedWheelTimer dlTimer;
-
-    // region resolver
-    private final RegionResolver regionResolver;
-
-    // Ownership maintenance
-    private final OwnershipCache ownershipCache;
-    // Channel/Client management
-    private final ProxyClientManager clientManager;
-    // Cluster Client (for routing service)
-    private final Optional<ClusterClient> clusterClient;
-
-    // Close Status
-    private boolean closed = false;
-    private final ReentrantReadWriteLock closeLock =
-            new ReentrantReadWriteLock();
-
-    abstract class StreamOp implements TimerTask {
-        final String stream;
-
-        final AtomicInteger tries = new AtomicInteger(0);
-        final RoutingContext routingContext = RoutingContext.of(regionResolver);
-        final WriteContext ctx = new WriteContext();
-        final Stopwatch stopwatch;
-        final OpStats opStats;
-        SocketAddress nextAddressToSend;
-
-        StreamOp(final String stream, final OpStats opStats) {
-            this.stream = stream;
-            this.stopwatch = Stopwatch.createStarted();
-            this.opStats = opStats;
-        }
-
-        boolean shouldTimeout() {
-            long elapsedMs = stopwatch.elapsed(TimeUnit.MILLISECONDS);
-            return shouldTimeout(elapsedMs);
-        }
-
-        boolean shouldTimeout(long elapsedMs) {
-            return clientConfig.getRequestTimeoutMs() > 0
-                && elapsedMs >= clientConfig.getRequestTimeoutMs();
-        }
-
-        void send(SocketAddress address) {
-            long elapsedMs = stopwatch.elapsed(TimeUnit.MILLISECONDS);
-            if (clientConfig.getMaxRedirects() > 0
-                && tries.get() >= clientConfig.getMaxRedirects()) {
-                fail(address, new RequestTimeoutException(Duration.fromMilliseconds(elapsedMs),
-                        "Exhausted max redirects in " + elapsedMs + " ms"));
-                return;
-            } else if (shouldTimeout(elapsedMs)) {
-                fail(address, new RequestTimeoutException(Duration.fromMilliseconds(elapsedMs),
-                        "Exhausted max request timeout " + clientConfig.getRequestTimeoutMs()
-                                + " in " + elapsedMs + " ms"));
-                return;
-            }
-            synchronized (this) {
-                String addrStr = address.toString();
-                if (ctx.isSetTriedHosts() && ctx.getTriedHosts().contains(addrStr)) {
-                    nextAddressToSend = address;
-                    dlTimer.newTimeout(this,
-                            Math.min(clientConfig.getRedirectBackoffMaxMs(),
-                                    tries.get() * clientConfig.getRedirectBackoffStartMs()),
-                            TimeUnit.MILLISECONDS);
-                } else {
-                    doSend(address);
-                }
-            }
-        }
-
-        abstract Future<ResponseHeader> sendRequest(ProxyClient sc);
-
-        void doSend(SocketAddress address) {
-            ctx.addToTriedHosts(address.toString());
-            if (clientConfig.isChecksumEnabled()) {
-                Long crc32 = computeChecksum();
-                if (null != crc32) {
-                    ctx.setCrc32(crc32);
-                }
-            }
-            tries.incrementAndGet();
-            sendWriteRequest(address, this);
-        }
-
-        void beforeComplete(ProxyClient sc, ResponseHeader responseHeader) {
-            ownershipCache.updateOwner(stream, sc.getAddress());
-        }
-
-        void complete(SocketAddress address) {
-            stopwatch.stop();
-            opStats.completeRequest(address,
-                    stopwatch.elapsed(TimeUnit.MICROSECONDS), tries.get());
-        }
-
-        void fail(SocketAddress address, Throwable t) {
-            stopwatch.stop();
-            opStats.failRequest(address,
-                    stopwatch.elapsed(TimeUnit.MICROSECONDS), tries.get());
-        }
-
-        Long computeChecksum() {
-            return null;
-        }
-
-        @Override
-        public synchronized void run(Timeout timeout) throws Exception {
-            if (!timeout.isCancelled() && null != nextAddressToSend) {
-                doSend(nextAddressToSend);
-            } else {
-                fail(null, new CancelledRequestException());
-            }
-        }
-    }
-
-    class BulkWriteOp extends StreamOp {
-
-        final List<ByteBuffer> data;
-        final ArrayList<Promise<DLSN>> results;
-
-        BulkWriteOp(final String name, final List<ByteBuffer> data) {
-            super(name, clientStats.getOpStats("bulk_write"));
-            this.data = data;
-
-            // This could take a while (relatively speaking) for very large inputs. We probably don't want
-            // to go so large for other reasons though.
-            this.results = new ArrayList<Promise<DLSN>>(data.size());
-            for (int i = 0; i < data.size(); i++) {
-                checkNotNull(data.get(i));
-                this.results.add(new Promise<DLSN>());
-            }
-        }
-
-        @Override
-        Future<ResponseHeader> sendRequest(final ProxyClient sc) {
-            return sc.getService().writeBulkWithContext(stream, data, ctx)
-                .addEventListener(new FutureEventListener<BulkWriteResponse>() {
-                @Override
-                public void onSuccess(BulkWriteResponse response) {
-                    // For non-success case, the ResponseHeader handler (the caller) will handle it.
-                    // Note success in this case means no finagle errors have occurred
-                    // (such as finagle connection issues). In general code != SUCCESS means there's some error
-                    // reported by dlog service. The caller will handle such errors.
-                    if (response.getHeader().getCode() == StatusCode.SUCCESS) {
-                        beforeComplete(sc, response.getHeader());
-                        BulkWriteOp.this.complete(sc.getAddress(), response);
-                        if (response.getWriteResponses().size() == 0 && data.size() > 0) {
-                            logger.error("non-empty bulk write got back empty response without failure for stream {}",
-                                stream);
-                        }
-                    }
-                }
-                @Override
-                public void onFailure(Throwable cause) {
-                    // Handled by the ResponseHeader listener (attached by the caller).
-                }
-            }).map(new AbstractFunction1<BulkWriteResponse, ResponseHeader>() {
-                @Override
-                public ResponseHeader apply(BulkWriteResponse response) {
-                    // We need to return the ResponseHeader to the caller's listener to process DLOG errors.
-                    return response.getHeader();
-                }
-            });
-        }
-
-        void complete(SocketAddress address, BulkWriteResponse bulkWriteResponse) {
-            super.complete(address);
-            Iterator<WriteResponse> writeResponseIterator = bulkWriteResponse.getWriteResponses().iterator();
-            Iterator<Promise<DLSN>> resultIterator = results.iterator();
-
-            // Fill in errors from thrift responses.
-            while (resultIterator.hasNext() && writeResponseIterator.hasNext()) {
-                Promise<DLSN> result = resultIterator.next();
-                WriteResponse writeResponse = writeResponseIterator.next();
-                if (StatusCode.SUCCESS == writeResponse.getHeader().getCode()) {
-                    result.setValue(DLSN.deserialize(writeResponse.getDlsn()));
-                } else {
-                    result.setException(DLException.of(writeResponse.getHeader()));
-                }
-            }
-
-            // Should never happen, but just in case so there's some record.
-            if (bulkWriteResponse.getWriteResponses().size() != data.size()) {
-                logger.error("wrong number of results, response = {} records = {}",
-                    bulkWriteResponse.getWriteResponses().size(), data.size());
-            }
-        }
-
-        @Override
-        void fail(SocketAddress address, Throwable t) {
-
-            // StreamOp.fail is called to fail the overall request. In case of BulkWriteOp we take the request level
-            // exception to apply to the first write. In fact for request level exceptions no request has ever been
-            // attempted, but logically we associate the error with the first write.
-            super.fail(address, t);
-            Iterator<Promise<DLSN>> resultIterator = results.iterator();
-
-            // Fail the first write with the batch level failure.
-            if (resultIterator.hasNext()) {
-                Promise<DLSN> result = resultIterator.next();
-                result.setException(t);
-            }
-
-            // Fail the remaining writes as cancelled requests.
-            while (resultIterator.hasNext()) {
-                Promise<DLSN> result = resultIterator.next();
-                result.setException(new CancelledRequestException());
-            }
-        }
-
-        @SuppressWarnings("unchecked")
-        List<Future<DLSN>> result() {
-            return (List) results;
-        }
-    }
-
-    abstract class AbstractWriteOp extends StreamOp {
-
-        final Promise<WriteResponse> result = new Promise<WriteResponse>();
-        Long crc32 = null;
-
-        AbstractWriteOp(final String name, final OpStats opStats) {
-            super(name, opStats);
-        }
-
-        void complete(SocketAddress address, WriteResponse response) {
-            super.complete(address);
-            result.setValue(response);
-        }
-
-        @Override
-        void fail(SocketAddress address, Throwable t) {
-            super.fail(address, t);
-            result.setException(t);
-        }
-
-        @Override
-        Long computeChecksum() {
-            if (null == crc32) {
-                crc32 = ProtocolUtils.streamOpCRC32(stream);
-            }
-            return crc32;
-        }
-
-        @Override
-        Future<ResponseHeader> sendRequest(final ProxyClient sc) {
-            return this.sendWriteRequest(sc).addEventListener(new FutureEventListener<WriteResponse>() {
-                @Override
-                public void onSuccess(WriteResponse response) {
-                    if (response.getHeader().getCode() == StatusCode.SUCCESS) {
-                        beforeComplete(sc, response.getHeader());
-                        AbstractWriteOp.this.complete(sc.getAddress(), response);
-                    }
-                }
-                @Override
-                public void onFailure(Throwable cause) {
-                    // handled by the ResponseHeader listener
-                }
-            }).map(new AbstractFunction1<WriteResponse, ResponseHeader>() {
-                @Override
-                public ResponseHeader apply(WriteResponse response) {
-                    return response.getHeader();
-                }
-            });
-        }
-
-        abstract Future<WriteResponse> sendWriteRequest(ProxyClient sc);
-    }
-
-    class WriteOp extends AbstractWriteOp {
-        final ByteBuffer data;
-
-        WriteOp(final String name, final ByteBuffer data) {
-            super(name, clientStats.getOpStats("write"));
-            this.data = data;
-        }
-
-        @Override
-        Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
-            return sc.getService().writeWithContext(stream, data, ctx);
-        }
-
-        @Override
-        Long computeChecksum() {
-            if (null == crc32) {
-                byte[] dataBytes = new byte[data.remaining()];
-                data.duplicate().get(dataBytes);
-                crc32 = ProtocolUtils.writeOpCRC32(stream, dataBytes);
-            }
-            return crc32;
-        }
-
-        Future<DLSN> result() {
-            return result.map(new AbstractFunction1<WriteResponse, DLSN>() {
-                @Override
-                public DLSN apply(WriteResponse response) {
-                    return DLSN.deserialize(response.getDlsn());
-                }
-            });
-        }
-    }
-
-    class TruncateOp extends AbstractWriteOp {
-        final DLSN dlsn;
-
-        TruncateOp(String name, DLSN dlsn) {
-            super(name, clientStats.getOpStats("truncate"));
-            this.dlsn = dlsn;
-        }
-
-        @Override
-        Long computeChecksum() {
-            if (null == crc32) {
-                crc32 = ProtocolUtils.truncateOpCRC32(stream, dlsn);
-            }
-            return crc32;
-        }
-
-        @Override
-        Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
-            return sc.getService().truncate(stream, dlsn.serialize(), ctx);
-        }
-
-        Future<Boolean> result() {
-            return result.map(new AbstractFunction1<WriteResponse, Boolean>() {
-                @Override
-                public Boolean apply(WriteResponse response) {
-                    return true;
-                }
-            });
-        }
-    }
-
-    class WriteRecordSetOp extends WriteOp {
-
-        WriteRecordSetOp(String name, LogRecordSetBuffer recordSet) {
-            super(name, recordSet.getBuffer());
-            ctx.setIsRecordSet(true);
-        }
-
-    }
-
-
-    class ReleaseOp extends AbstractWriteOp {
-
-        ReleaseOp(String name) {
-            super(name, clientStats.getOpStats("release"));
-        }
-
-        @Override
-        Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
-            return sc.getService().release(stream, ctx);
-        }
-
-        @Override
-        void beforeComplete(ProxyClient sc, ResponseHeader header) {
-            ownershipCache.removeOwnerFromStream(stream, sc.getAddress(), "Stream Deleted");
-        }
-
-        Future<Void> result() {
-            return result.map(new AbstractFunction1<WriteResponse, Void>() {
-                @Override
-                public Void apply(WriteResponse response) {
-                    return null;
-                }
-            });
-        }
-    }
-
-    class DeleteOp extends AbstractWriteOp {
-
-        DeleteOp(String name) {
-            super(name, clientStats.getOpStats("delete"));
-        }
-
-        @Override
-        Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
-            return sc.getService().delete(stream, ctx);
-        }
-
-        @Override
-        void beforeComplete(ProxyClient sc, ResponseHeader header) {
-            ownershipCache.removeOwnerFromStream(stream, sc.getAddress(), "Stream Deleted");
-        }
-
-        Future<Void> result() {
-            return result.map(new AbstractFunction1<WriteResponse, Void>() {
-                @Override
-                public Void apply(WriteResponse v1) {
-                    return null;
-                }
-            });
-        }
-    }
-
-    class CreateOp extends AbstractWriteOp {
-
-        CreateOp(String name) {
-            super(name, clientStats.getOpStats("create"));
-        }
-
-        @Override
-        Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
-            return sc.getService().create(stream, ctx);
-        }
-
-        @Override
-        void beforeComplete(ProxyClient sc, ResponseHeader header) {
-            ownershipCache.updateOwner(stream, sc.getAddress());
-        }
-
-        Future<Void> result() {
-            return result.map(new AbstractFunction1<WriteResponse, Void>() {
-                @Override
-                public Void apply(WriteResponse v1) {
-                    return null;
-                }
-            }).voided();
-        }
-    }
-
-    class HeartbeatOp extends AbstractWriteOp {
-        HeartbeatOptions options;
-
-        HeartbeatOp(String name, boolean sendReaderHeartBeat) {
-            super(name, clientStats.getOpStats("heartbeat"));
-            options = new HeartbeatOptions();
-            options.setSendHeartBeatToReader(sendReaderHeartBeat);
-        }
-
-        @Override
-        Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
-            return sc.getService().heartbeatWithOptions(stream, ctx, options);
-        }
-
-        Future<Void> result() {
-            return result.map(new AbstractFunction1<WriteResponse, Void>() {
-                @Override
-                public Void apply(WriteResponse response) {
-                    return null;
-                }
-            });
-        }
-    }
-
-    // Stats
-    private final ClientStats clientStats;
-
-    public DistributedLogClientImpl(String name,
-                                    ClientId clientId,
-                                    RoutingService routingService,
-                                    ClientBuilder clientBuilder,
-                                    ClientConfig clientConfig,
-                                    Optional<ClusterClient> clusterClient,
-                                    StatsReceiver statsReceiver,
-                                    StatsReceiver streamStatsReceiver,
-                                    RegionResolver regionResolver,
-                                    boolean enableRegionStats) {
-        this.clientName = name;
-        this.clientId = clientId;
-        this.routingService = routingService;
-        this.clientConfig = clientConfig;
-        this.streamFailfast = clientConfig.getStreamFailfast();
-        this.streamNameRegexPattern = Pattern.compile(clientConfig.getStreamNameRegex());
-        this.regionResolver = regionResolver;
-        // Build the timer
-        this.dlTimer = new HashedWheelTimer(
-                new ThreadFactoryBuilder().setNameFormat("DLClient-" + name + "-timer-%d").build(),
-                this.clientConfig.getRedirectBackoffStartMs(),
-                TimeUnit.MILLISECONDS);
-        // register routing listener
-        this.routingService.registerListener(this);
-        // build the ownership cache
-        this.ownershipCache = new OwnershipCache(this.clientConfig, this.dlTimer, statsReceiver, streamStatsReceiver);
-        // Client Stats
-        this.clientStats = new ClientStats(statsReceiver, enableRegionStats, regionResolver);
-        // Client Manager
-        this.clientBuilder = ProxyClient.newBuilder(clientName, clientId, clientBuilder, clientConfig, clientStats);
-        this.clientManager = new ProxyClientManager(
-                this.clientConfig,  // client config
-                this.clientBuilder, // client builder
-                this.dlTimer,       // timer
-                this,               // host provider
-                clientStats);       // client stats
-        this.clusterClient = clusterClient;
-        this.clientManager.registerProxyListener(this);
-
-        // Cache Stats
-        StatsReceiver cacheStatReceiver = statsReceiver.scope("cache");
-        Seq<String> numCachedStreamsGaugeName =
-                scala.collection.JavaConversions.asScalaBuffer(Arrays.asList("num_streams")).toList();
-        cacheStatReceiver.provideGauge(numCachedStreamsGaugeName, new Function0<Object>() {
-            @Override
-            public Object apply() {
-                return (float) ownershipCache.getNumCachedStreams();
-            }
-        });
-        Seq<String> numCachedHostsGaugeName =
-                scala.collection.JavaConversions.asScalaBuffer(Arrays.asList("num_hosts")).toList();
-        cacheStatReceiver.provideGauge(numCachedHostsGaugeName, new Function0<Object>() {
-            @Override
-            public Object apply() {
-                return (float) clientManager.getNumProxies();
-            }
-        });
-
-        logger.info("Build distributedlog client : name = {}, client_id = {}, routing_service = {},"
-            + " stats_receiver = {}, thriftmux = {}",
-            new Object[] {
-                name,
-                clientId,
-                routingService.getClass(),
-                statsReceiver.getClass(),
-                clientConfig.getThriftMux()
-            });
-    }
-
-    @Override
-    public Set<SocketAddress> getHosts() {
-        Set<SocketAddress> hosts = Sets.newHashSet();
-        // if using server side routing, we only handshake with the hosts in ownership cache.
-        if (!clusterClient.isPresent()) {
-            hosts.addAll(this.routingService.getHosts());
-        }
-        hosts.addAll(this.ownershipCache.getStreamOwnershipDistribution().keySet());
-        return hosts;
-    }
-
-    @Override
-    public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
-        if (null != serverInfo
-            && serverInfo.isSetServerStatus()
-            && ServerStatus.DOWN == serverInfo.getServerStatus()) {
-            logger.info("{} is detected as DOWN during handshaking", address);
-            // server is shutting down
-            handleServiceUnavailable(address, client, Optional.<StreamOp>absent());
-            return;
-        }
-
-        if (null != serverInfo && serverInfo.isSetOwnerships()) {
-            Map<String, String> ownerships = serverInfo.getOwnerships();
-            logger.debug("Handshaked with {} : {} ownerships returned.", address, ownerships.size());
-            for (Map.Entry<String, String> entry : ownerships.entrySet()) {
-                Matcher matcher = streamNameRegexPattern.matcher(entry.getKey());
-                if (!matcher.matches()) {
-                    continue;
-                }
-                updateOwnership(entry.getKey(), entry.getValue());
-            }
-        } else {
-            logger.debug("Handshaked with {} : no ownerships returned", address);
-        }
-    }
-
-    @Override
-    public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
-        cause = showRootCause(Optional.<StreamOp>absent(), cause);
-        handleRequestException(address, client, Optional.<StreamOp>absent(), cause);
-    }
-
-    @VisibleForTesting
-    public void handshake() {
-        clientManager.handshake();
-        logger.info("Handshaked with {} hosts, cached {} streams",
-                clientManager.getNumProxies(), ownershipCache.getNumCachedStreams());
-    }
-
-    @Override
-    public void onServerLeft(SocketAddress address) {
-        onServerLeft(address, null);
-    }
-
-    private void onServerLeft(SocketAddress address, ProxyClient sc) {
-        ownershipCache.removeAllStreamsFromOwner(address);
-        if (null == sc) {
-            clientManager.removeClient(address);
-        } else {
-            clientManager.removeClient(address, sc);
-        }
-    }
-
-    @Override
-    public void onServerJoin(SocketAddress address) {
-        // we only pre-create connection for client-side routing
-        // if it is server side routing, we only know the exact proxy address
-        // when #getOwner.
-        if (!clusterClient.isPresent()) {
-            clientManager.createClient(address);
-        }
-    }
-
-    public void close() {
-        closeLock.writeLock().lock();
-        try {
-            if (closed) {
-                return;
-            }
-            closed = true;
-        } finally {
-            closeLock.writeLock().unlock();
-        }
-        clientManager.close();
-        routingService.unregisterListener(this);
-        routingService.stopService();
-        dlTimer.stop();
-    }
-
-    @Override
-    public Future<Void> check(String stream) {
-        final HeartbeatOp op = new HeartbeatOp(stream, false);
-        sendRequest(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<Void> heartbeat(String stream) {
-        final HeartbeatOp op = new HeartbeatOp(stream, true);
-        sendRequest(op);
-        return op.result();
-    }
-
-    @Override
-    public Map<SocketAddress, Set<String>> getStreamOwnershipDistribution() {
-        return ownershipCache.getStreamOwnershipDistribution();
-    }
-
-    @Override
-    public Future<Void> setAcceptNewStream(boolean enabled) {
-        Map<SocketAddress, ProxyClient> snapshot = clientManager.getAllClients();
-        List<Future<Void>> futures = new ArrayList<Future<Void>>(snapshot.size());
-        for (Map.Entry<SocketAddress, ProxyClient> entry : snapshot.entrySet()) {
-            futures.add(entry.getValue().getService().setAcceptNewStream(enabled));
-        }
-        return Future.collect(futures).map(new Function<List<Void>, Void>() {
-            @Override
-            public Void apply(List<Void> list) {
-                return null;
-            }
-        });
-    }
-
-    @Override
-    public Future<DLSN> write(String stream, ByteBuffer data) {
-        final WriteOp op = new WriteOp(stream, data);
-        sendRequest(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<DLSN> writeRecordSet(String stream, final LogRecordSetBuffer recordSet) {
-        final WriteRecordSetOp op = new WriteRecordSetOp(stream, recordSet);
-        sendRequest(op);
-        return op.result();
-    }
-
-    @Override
-    public List<Future<DLSN>> writeBulk(String stream, List<ByteBuffer> data) {
-        if (data.size() > 0) {
-            final BulkWriteOp op = new BulkWriteOp(stream, data);
-            sendRequest(op);
-            return op.result();
-        } else {
-            return Collections.emptyList();
-        }
-    }
-
-    @Override
-    public Future<Boolean> truncate(String stream, DLSN dlsn) {
-        final TruncateOp op = new TruncateOp(stream, dlsn);
-        sendRequest(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<Void> delete(String stream) {
-        final DeleteOp op = new DeleteOp(stream);
-        sendRequest(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<Void> release(String stream) {
-        final ReleaseOp op = new ReleaseOp(stream);
-        sendRequest(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<Void> create(String stream) {
-        final CreateOp op = new CreateOp(stream);
-        sendRequest(op);
-        return op.result();
-    }
-
-    private void sendRequest(final StreamOp op) {
-        closeLock.readLock().lock();
-        try {
-            if (closed) {
-                op.fail(null, new DLClientClosedException("Client " + clientName + " is closed."));
-            } else {
-                doSend(op, null);
-            }
-        } finally {
-            closeLock.readLock().unlock();
-        }
-    }
-
-    /**
-     * Send the stream operation by routing service, excluding previous address if it is not null.
-     *
-     * @param op
-     *          stream operation.
-     * @param previousAddr
-     *          previous tried address.
-     */
-    private void doSend(final StreamOp op, final SocketAddress previousAddr) {
-        if (null != previousAddr) {
-            op.routingContext.addTriedHost(previousAddr, StatusCode.WRITE_EXCEPTION);
-        }
-        // Get host first
-        final SocketAddress address = ownershipCache.getOwner(op.stream);
-        if (null == address || op.routingContext.isTriedHost(address)) {
-            getOwner(op).addEventListener(new FutureEventListener<SocketAddress>() {
-                @Override
-                public void onFailure(Throwable cause) {
-                    op.fail(null, cause);
-                }
-
-                @Override
-                public void onSuccess(SocketAddress ownerAddr) {
-                    op.send(ownerAddr);
-                }
-            });
-        } else {
-            op.send(address);
-        }
-    }
-
-    private void retryGetOwnerFromResourcePlacementServer(final StreamOp op,
-                                                final Promise<SocketAddress> getOwnerPromise,
-                                                final Throwable cause) {
-        if (op.shouldTimeout()) {
-            op.fail(null, cause);
-            return;
-        }
-        getOwnerFromResourcePlacementServer(op, getOwnerPromise);
-    }
-
-    private void getOwnerFromResourcePlacementServer(final StreamOp op,
-                                                     final Promise<SocketAddress> getOwnerPromise) {
-        clusterClient.get().getService().getOwner(op.stream, op.ctx)
-            .addEventListener(new FutureEventListener<WriteResponse>() {
-                @Override
-                public void onFailure(Throwable cause) {
-                    getOwnerPromise.updateIfEmpty(new Throw<SocketAddress>(cause));
-                }
-
-                @Override
-                public void onSuccess(WriteResponse value) {
-                    if (StatusCode.FOUND == value.getHeader().getCode()
-                          && null != value.getHeader().getLocation()) {
-                        try {
-                            InetSocketAddress addr = DLSocketAddress.deserialize(
-                                value.getHeader().getLocation()
-                            ).getSocketAddress();
-                            getOwnerPromise.updateIfEmpty(new Return<SocketAddress>(addr));
-                        } catch (IOException e) {
-                            // retry from the routing server again
-                            logger.error("ERROR in getOwner", e);
-                            retryGetOwnerFromResourcePlacementServer(op, getOwnerPromise, e);
-                            return;
-                        }
-                    } else {
-                        // retry from the routing server again
-                        retryGetOwnerFromResourcePlacementServer(op, getOwnerPromise,
-                                new StreamUnavailableException("Stream " + op.stream + "'s owner is unknown"));
-                    }
-                }
-            });
-    }
-
-    private Future<SocketAddress> getOwner(final StreamOp op) {
-        if (clusterClient.isPresent()) {
-            final Promise<SocketAddress> getOwnerPromise = new Promise<SocketAddress>();
-            getOwnerFromResourcePlacementServer(op, getOwnerPromise);
-            return getOwnerPromise;
-        }
-        // pickup host by hashing
-        try {
-            return Future.value(routingService.getHost(op.stream, op.routingContext));
-        } catch (NoBrokersAvailableException nbae) {
-            return Future.exception(nbae);
-        }
-    }
-
-    private void sendWriteRequest(final SocketAddress addr, final StreamOp op) {
-        // Get corresponding finagle client
-        final ProxyClient sc = clientManager.getClient(addr);
-        final long startTimeNanos = System.nanoTime();
-        // write the request to that host.
-        op.sendRequest(sc).addEventListener(new FutureEventListener<ResponseHeader>() {
-            @Override
-            public void onSuccess(ResponseHeader header) {
-                if (logger.isDebugEnabled()) {
-                    logger.debug("Received response; header: {}", header);
-                }
-                clientStats.completeProxyRequest(addr, header.getCode(), startTimeNanos);
-                // update routing context
-                op.routingContext.addTriedHost(addr, header.getCode());
-                switch (header.getCode()) {
-                    case SUCCESS:
-                        // success handling is done per stream op
-                        break;
-                    case FOUND:
-                        handleRedirectResponse(header, op, addr);
-                        break;
-                    // for overcapacity, dont report failure since this normally happens quite a bit
-                    case OVER_CAPACITY:
-                        logger.debug("Failed to write request to {} : {}", op.stream, header);
-                        op.fail(addr, DLException.of(header));
-                        break;
-                    // for responses that indicate the requests definitely failed,
-                    // we should fail them immediately (e.g. TOO_LARGE_RECORD, METADATA_EXCEPTION)
-                    case NOT_IMPLEMENTED:
-                    case METADATA_EXCEPTION:
-                    case LOG_EMPTY:
-                    case LOG_NOT_FOUND:
-                    case TRUNCATED_TRANSACTION:
-                    case END_OF_STREAM:
-                    case TRANSACTION_OUT_OF_ORDER:
-                    case INVALID_STREAM_NAME:
-                    case REQUEST_DENIED:
-                    case TOO_LARGE_RECORD:
-                    case CHECKSUM_FAILED:
-                    // status code NOT_READY is returned if failfast is enabled in the server. don't redirect
-                    // since the proxy may still own the stream.
-                    case STREAM_NOT_READY:
-                        op.fail(addr, DLException.of(header));
-                        break;
-                    case SERVICE_UNAVAILABLE:
-                        handleServiceUnavailable(addr, sc, Optional.of(op));
-                        break;
-                    case REGION_UNAVAILABLE:
-                        // region is unavailable, redirect the request to hosts in other region
-                        redirect(op, null);
-                        break;
-                    // Proxy was overloaded and refused to try to acquire the stream. Don't remove ownership, since
-                    // we didn't have it in the first place.
-                    case TOO_MANY_STREAMS:
-                        handleRedirectableError(addr, op, header);
-                        break;
-                    case STREAM_UNAVAILABLE:
-                    case ZOOKEEPER_ERROR:
-                    case LOCKING_EXCEPTION:
-                    case UNEXPECTED:
-                    case INTERRUPTED:
-                    case BK_TRANSMIT_ERROR:
-                    case FLUSH_TIMEOUT:
-                    default:
-                        // when we are receiving these exceptions from proxy, it means proxy or the stream is closed
-                        // redirect the request.
-                        ownershipCache.removeOwnerFromStream(op.stream, addr, header.getCode().name());
-                        handleRedirectableError(addr, op, header);
-                        break;
-                }
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                Optional<StreamOp> opOptional = Optional.of(op);
-                cause = showRootCause(opOptional, cause);
-                clientStats.failProxyRequest(addr, cause, startTimeNanos);
-                handleRequestException(addr, sc, opOptional, cause);
-            }
-        });
-    }
-
-    // Response Handlers
-
-    Throwable showRootCause(Optional<StreamOp> op, Throwable cause) {
-        if (cause instanceof Failure) {
-            Failure failure = (Failure) cause;
-            if (failure.isFlagged(Failure.Wrapped())) {
-                try {
-                    // if it is a wrapped failure, unwrap it first
-                    cause = failure.show();
-                } catch (IllegalArgumentException iae) {
-                    if (op.isPresent()) {
-                        logger.warn("Failed to unwrap finagle failure of stream {} : ", op.get().stream, iae);
-                    } else {
-                        logger.warn("Failed to unwrap finagle failure : ", iae);
-                    }
-                }
-            }
-        }
-        return cause;
-    }
-
-    private void handleRedirectableError(SocketAddress addr,
-                                         StreamOp op,
-                                         ResponseHeader header) {
-        if (streamFailfast) {
-            op.fail(addr, DLException.of(header));
-        } else {
-            redirect(op, null);
-        }
-    }
-
-    void handleServiceUnavailable(SocketAddress addr,
-                                  ProxyClient sc,
-                                  Optional<StreamOp> op) {
-        // service is unavailable, remove it out of routing service
-        routingService.removeHost(addr, new ServiceUnavailableException(addr + " is unavailable now."));
-        onServerLeft(addr);
-        if (op.isPresent()) {
-            ownershipCache.removeOwnerFromStream(op.get().stream, addr, addr + " is unavailable now.");
-            // redirect the request to other host.
-            redirect(op.get(), null);
-        }
-    }
-
-    void handleRequestException(SocketAddress addr,
-                                ProxyClient sc,
-                                Optional<StreamOp> op,
-                                Throwable cause) {
-        boolean resendOp = false;
-        boolean removeOwnerFromStream = false;
-        SocketAddress previousAddr = addr;
-        String reason = cause.getMessage();
-        if (cause instanceof ConnectionFailedException || cause instanceof java.net.ConnectException) {
-            routingService.removeHost(addr, cause);
-            onServerLeft(addr, sc);
-            removeOwnerFromStream = true;
-            // redirect the request to other host.
-            resendOp = true;
-        } else if (cause instanceof ChannelException) {
-            // java.net.ConnectException typically means connection is refused remotely
-            // no process listening on remote address/port.
-            if (cause.getCause() instanceof java.net.ConnectException) {
-                routingService.removeHost(addr, cause.getCause());
-                onServerLeft(addr);
-                reason = cause.getCause().getMessage();
-            } else {
-                routingService.removeHost(addr, cause);
-                reason = cause.getMessage();
-            }
-            removeOwnerFromStream = true;
-            // redirect the request to other host.
-            resendOp = true;
-        } else if (cause instanceof ServiceTimeoutException) {
-            // redirect the request to itself again, which will backoff for a while
-            resendOp = true;
-            previousAddr = null;
-        } else if (cause instanceof WriteException) {
-            // redirect the request to other host.
-            resendOp = true;
-        } else if (cause instanceof ServiceException) {
-            // redirect the request to other host.
-            clientManager.removeClient(addr, sc);
-            resendOp = true;
-        } else if (cause instanceof TApplicationException) {
-            handleTApplicationException(cause, op, addr, sc);
-        } else if (cause instanceof Failure) {
-            handleFinagleFailure((Failure) cause, op, addr);
-        } else {
-            // Default handler
-            handleException(cause, op, addr);
-        }
-
-        if (op.isPresent()) {
-            if (removeOwnerFromStream) {
-                ownershipCache.removeOwnerFromStream(op.get().stream, addr, reason);
-            }
-            if (resendOp) {
-                doSend(op.get(), previousAddr);
-            }
-        }
-    }
-
-    /**
-     * Redirect the request to new proxy <i>newAddr</i>. If <i>newAddr</i> is null,
-     * it would pick up a host from routing service.
-     *
-     * @param op
-     *          stream operation
-     * @param newAddr
-     *          new proxy address
-     */
-    void redirect(StreamOp op, SocketAddress newAddr) {
-        ownershipCache.getOwnershipStatsLogger().onRedirect(op.stream);
-        if (null != newAddr) {
-            logger.debug("Redirect request {} to new owner {}.", op, newAddr);
-            op.send(newAddr);
-        } else {
-            doSend(op, null);
-        }
-    }
-
-    void handleFinagleFailure(Failure failure,
-                              Optional<StreamOp> op,
-                              SocketAddress addr) {
-        if (failure.isFlagged(Failure.Restartable())) {
-            if (op.isPresent()) {
-                // redirect the request to other host
-                doSend(op.get(), addr);
-            }
-        } else {
-            // fail the request if it is other types of failures
-            handleException(failure, op, addr);
-        }
-    }
-
-    void handleException(Throwable cause,
-                         Optional<StreamOp> op,
-                         SocketAddress addr) {
-        // RequestTimeoutException: fail it and let client decide whether to retry or not.
-
-        // FailedFastException:
-        // We don't actually know when FailedFastException will be thrown
-        // so properly we just throw it back to application to let application
-        // handle it.
-
-        // Other Exceptions: as we don't know how to handle them properly so throw them to client
-        if (op.isPresent()) {
-            logger.error("Failed to write request to {} @ {} : {}",
-                    new Object[]{op.get().stream, addr, cause.toString()});
-            op.get().fail(addr, cause);
-        }
-    }
-
-    void handleTApplicationException(Throwable cause,
-                                     Optional<StreamOp> op,
-                                     SocketAddress addr,
-                                     ProxyClient sc) {
-        TApplicationException ex = (TApplicationException) cause;
-        if (ex.getType() == TApplicationException.UNKNOWN_METHOD) {
-            // if we encountered unknown method exception on thrift server, it means this proxy
-            // has problem. we should remove it from routing service, clean up ownerships
-            routingService.removeHost(addr, cause);
-            onServerLeft(addr, sc);
-            if (op.isPresent()) {
-                ownershipCache.removeOwnerFromStream(op.get().stream, addr, cause.getMessage());
-                doSend(op.get(), addr);
-            }
-        } else {
-            handleException(cause, op, addr);
-        }
-    }
-
-    void handleRedirectResponse(ResponseHeader header, StreamOp op, SocketAddress curAddr) {
-        SocketAddress ownerAddr = null;
-        if (header.isSetLocation()) {
-            String owner = header.getLocation();
-            try {
-                ownerAddr = DLSocketAddress.deserialize(owner).getSocketAddress();
-                // if we are receiving a direct request to same host, we won't try the same host.
-                // as the proxy will shut itself down if it redirects client to itself.
-                if (curAddr.equals(ownerAddr)) {
-                    logger.warn("Request to stream {} is redirected to same server {}!", op.stream, curAddr);
-                    ownerAddr = null;
-                } else {
-                    // update ownership when redirects.
-                    ownershipCache.updateOwner(op.stream, ownerAddr);
-                }
-            } catch (IOException e) {
-                ownerAddr = null;
-            }
-        }
-        redirect(op, ownerAddr);
-    }
-
-    void updateOwnership(String stream, String location) {
-        try {
-            SocketAddress ownerAddr = DLSocketAddress.deserialize(location).getSocketAddress();
-            // update ownership
-            ownershipCache.updateOwner(stream, ownerAddr);
-        } catch (IOException e) {
-            logger.warn("Invalid ownership {} found for stream {} : ",
-                new Object[] { location, stream, e });
-        }
-    }
-
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogMultiStreamWriter.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogMultiStreamWriter.java
deleted file mode 100644
index 8ccbbfc..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogMultiStreamWriter.java
+++ /dev/null
@@ -1,486 +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 com.twitter.distributedlog.client;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-
-import com.google.common.base.Stopwatch;
-import com.google.common.base.Ticker;
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecordSet;
-import com.twitter.distributedlog.LogRecordSetBuffer;
-import com.twitter.distributedlog.client.speculative.DefaultSpeculativeRequestExecutionPolicy;
-import com.twitter.distributedlog.client.speculative.SpeculativeRequestExecutionPolicy;
-import com.twitter.distributedlog.client.speculative.SpeculativeRequestExecutor;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.finagle.IndividualRequestTimeoutException;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Write to multiple streams.
- */
-public class DistributedLogMultiStreamWriter implements Runnable {
-
-    /**
-     * Create a new builder to create a multi stream writer.
-     *
-     * @return a new builder to create a multi stream writer.
-     */
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    /**
-     * Builder for the multi stream writer.
-     */
-    public static class Builder {
-
-        private DistributedLogClient client = null;
-        private List<String> streams = null;
-        private int bufferSize = 16 * 1024; // 16k
-        private long flushIntervalMicros = 2000; // 2ms
-        private CompressionCodec.Type codec = CompressionCodec.Type.NONE;
-        private ScheduledExecutorService executorService = null;
-        private long requestTimeoutMs = 500; // 500ms
-        private int firstSpeculativeTimeoutMs = 50; // 50ms
-        private int maxSpeculativeTimeoutMs = 200; // 200ms
-        private float speculativeBackoffMultiplier = 2;
-        private Ticker ticker = Ticker.systemTicker();
-
-        private Builder() {}
-
-        /**
-         * Set the distributedlog client used for multi stream writer.
-         *
-         * @param client
-         *          distributedlog client
-         * @return builder
-         */
-        public Builder client(DistributedLogClient client) {
-            this.client = client;
-            return this;
-        }
-
-        /**
-         * Set the list of streams to write to.
-         *
-         * @param streams
-         *          list of streams to write
-         * @return builder
-         */
-        public Builder streams(List<String> streams) {
-            this.streams = streams;
-            return this;
-        }
-
-        /**
-         * Set the output buffer size.
-         *
-         * <p>If output buffer size is 0, the writes will be transmitted to
-         * wire immediately.
-         *
-         * @param bufferSize
-         *          output buffer size
-         * @return builder
-         */
-        public Builder bufferSize(int bufferSize) {
-            this.bufferSize = bufferSize;
-            return this;
-        }
-
-        /**
-         * Set the flush interval in milliseconds.
-         *
-         * @param flushIntervalMs
-         *          flush interval in milliseconds.
-         * @return builder
-         */
-        public Builder flushIntervalMs(int flushIntervalMs) {
-            this.flushIntervalMicros = TimeUnit.MILLISECONDS.toMicros(flushIntervalMs);
-            return this;
-        }
-
-        /**
-         * Set the flush interval in microseconds.
-         *
-         * @param flushIntervalMicros
-         *          flush interval in microseconds.
-         * @return builder
-         */
-        public Builder flushIntervalMicros(int flushIntervalMicros) {
-            this.flushIntervalMicros = flushIntervalMicros;
-            return this;
-        }
-
-        /**
-         * Set compression codec.
-         *
-         * @param codec compression codec.
-         * @return builder
-         */
-        public Builder compressionCodec(CompressionCodec.Type codec) {
-            this.codec = codec;
-            return this;
-        }
-
-        /**
-         * Set the scheduler to flush output buffers.
-         *
-         * @param executorService
-         *          executor service to flush output buffers.
-         * @return builder
-         */
-        public Builder scheduler(ScheduledExecutorService executorService) {
-            this.executorService = executorService;
-            return this;
-        }
-
-        /**
-         * Set request timeout in milliseconds.
-         *
-         * @param requestTimeoutMs
-         *          request timeout in milliseconds.
-         * @return builder
-         */
-        public Builder requestTimeoutMs(long requestTimeoutMs) {
-            this.requestTimeoutMs = requestTimeoutMs;
-            return this;
-        }
-
-        /**
-         * Set the first speculative timeout in milliseconds.
-         *
-         * <p>The multi-streams writer does speculative writes on streams.
-         * The write issues first write request to a stream, if the write request
-         * doesn't respond within speculative timeout. it issues next write request
-         * to a different stream. It does such speculative retries until receive
-         * a success or request timeout ({@link #requestTimeoutMs(long)}).
-         *
-         * <p>This setting is to configure the first speculative timeout, in milliseconds.
-         *
-         * @param timeoutMs
-         *          timeout in milliseconds
-         * @return builder
-         */
-        public Builder firstSpeculativeTimeoutMs(int timeoutMs) {
-            this.firstSpeculativeTimeoutMs = timeoutMs;
-            return this;
-        }
-
-        /**
-         * Set the max speculative timeout in milliseconds.
-         *
-         * <p>The multi-streams writer does speculative writes on streams.
-         * The write issues first write request to a stream, if the write request
-         * doesn't respond within speculative timeout. it issues next write request
-         * to a different stream. It does such speculative retries until receive
-         * a success or request timeout ({@link #requestTimeoutMs(long)}).
-         *
-         * <p>This setting is to configure the max speculative timeout, in milliseconds.
-         *
-         * @param timeoutMs
-         *          timeout in milliseconds
-         * @return builder
-         */
-        public Builder maxSpeculativeTimeoutMs(int timeoutMs) {
-            this.maxSpeculativeTimeoutMs = timeoutMs;
-            return this;
-        }
-
-        /**
-         * Set the speculative timeout backoff multiplier.
-         *
-         * <p>The multi-streams writer does speculative writes on streams.
-         * The write issues first write request to a stream, if the write request
-         * doesn't respond within speculative timeout. it issues next write request
-         * to a different stream. It does such speculative retries until receive
-         * a success or request timeout ({@link #requestTimeoutMs(long)}).
-         *
-         * <p>This setting is to configure the speculative timeout backoff multiplier.
-         *
-         * @param multiplier
-         *          backoff multiplier
-         * @return builder
-         */
-        public Builder speculativeBackoffMultiplier(float multiplier) {
-            this.speculativeBackoffMultiplier = multiplier;
-            return this;
-        }
-
-        /**
-         * Ticker for timing.
-         *
-         * @param ticker
-         *          ticker
-         * @return builder
-         * @see Ticker
-         */
-        public Builder clockTicker(Ticker ticker) {
-            this.ticker = ticker;
-            return this;
-        }
-
-        /**
-         * Build the multi stream writer.
-         *
-         * @return the multi stream writer.
-         */
-        public DistributedLogMultiStreamWriter build() {
-            checkArgument((null != streams && !streams.isEmpty()),
-                    "No streams provided");
-            checkNotNull(client,
-                    "No distributedlog client provided");
-            checkNotNull(codec,
-                    "No compression codec provided");
-            checkArgument(firstSpeculativeTimeoutMs > 0
-                    && firstSpeculativeTimeoutMs <= maxSpeculativeTimeoutMs
-                    && speculativeBackoffMultiplier > 0
-                    && maxSpeculativeTimeoutMs < requestTimeoutMs,
-                    "Invalid speculative timeout settings");
-            return new DistributedLogMultiStreamWriter(
-                streams,
-                client,
-                Math.min(bufferSize, MAX_LOGRECORDSET_SIZE),
-                flushIntervalMicros,
-                requestTimeoutMs,
-                firstSpeculativeTimeoutMs,
-                maxSpeculativeTimeoutMs,
-                speculativeBackoffMultiplier,
-                codec,
-                ticker,
-                executorService);
-        }
-    }
-
-    /**
-     * Pending Write Request.
-     */
-    class PendingWriteRequest implements FutureEventListener<DLSN>,
-            SpeculativeRequestExecutor {
-
-        private final LogRecordSetBuffer recordSet;
-        private AtomicBoolean complete = new AtomicBoolean(false);
-        private final Stopwatch stopwatch = Stopwatch.createStarted(clockTicker);
-        private int nextStream;
-        private int numTriedStreams = 0;
-
-        PendingWriteRequest(LogRecordSetBuffer recordSet) {
-            this.recordSet = recordSet;
-            this.nextStream = Math.abs(nextStreamId.incrementAndGet()) % numStreams;
-        }
-
-        synchronized String sendNextWrite() {
-            long elapsedMs = stopwatch.elapsed(TimeUnit.MILLISECONDS);
-            if (elapsedMs > requestTimeoutMs || numTriedStreams >= numStreams) {
-                fail(new IndividualRequestTimeoutException(Duration.fromMilliseconds(elapsedMs)));
-                return null;
-            }
-            try {
-                return sendWriteToStream(nextStream);
-            } finally {
-                nextStream = (nextStream + 1) % numStreams;
-                ++numTriedStreams;
-            }
-        }
-
-        synchronized String sendWriteToStream(int streamId) {
-            String stream = getStream(streamId);
-            client.writeRecordSet(stream, recordSet)
-                    .addEventListener(this);
-            return stream;
-        }
-
-        @Override
-        public void onSuccess(DLSN dlsn) {
-            if (!complete.compareAndSet(false, true)) {
-                return;
-            }
-            recordSet.completeTransmit(
-                    dlsn.getLogSegmentSequenceNo(),
-                    dlsn.getEntryId(),
-                    dlsn.getSlotId());
-        }
-
-        @Override
-        public void onFailure(Throwable cause) {
-            sendNextWrite();
-        }
-
-        private void fail(Throwable cause) {
-            if (!complete.compareAndSet(false, true)) {
-                return;
-            }
-            recordSet.abortTransmit(cause);
-        }
-
-        @Override
-        public Future<Boolean> issueSpeculativeRequest() {
-            return Future.value(!complete.get() && null != sendNextWrite());
-        }
-    }
-
-    private final int numStreams;
-    private final List<String> streams;
-    private final DistributedLogClient client;
-    private final int bufferSize;
-    private final long requestTimeoutMs;
-    private final SpeculativeRequestExecutionPolicy speculativePolicy;
-    private final Ticker clockTicker;
-    private final CompressionCodec.Type codec;
-    private final ScheduledExecutorService scheduler;
-    private final boolean ownScheduler;
-    private final AtomicInteger nextStreamId;
-    private LogRecordSet.Writer recordSetWriter;
-
-    private DistributedLogMultiStreamWriter(List<String> streams,
-                                            DistributedLogClient client,
-                                            int bufferSize,
-                                            long flushIntervalMicros,
-                                            long requestTimeoutMs,
-                                            int firstSpecultiveTimeoutMs,
-                                            int maxSpeculativeTimeoutMs,
-                                            float speculativeBackoffMultiplier,
-                                            CompressionCodec.Type codec,
-                                            Ticker clockTicker,
-                                            ScheduledExecutorService scheduler) {
-        this.streams = Lists.newArrayList(streams);
-        this.numStreams = this.streams.size();
-        this.client = client;
-        this.bufferSize = bufferSize;
-        this.requestTimeoutMs = requestTimeoutMs;
-        this.codec = codec;
-        this.clockTicker = clockTicker;
-        if (null == scheduler) {
-            this.scheduler = Executors.newSingleThreadScheduledExecutor(
-                    new ThreadFactoryBuilder()
-                            .setDaemon(true)
-                            .setNameFormat("MultiStreamWriterFlushThread-%d")
-                            .build());
-            this.ownScheduler = true;
-        } else {
-            this.scheduler = scheduler;
-            this.ownScheduler = false;
-        }
-        this.speculativePolicy = new DefaultSpeculativeRequestExecutionPolicy(
-                firstSpecultiveTimeoutMs,
-                maxSpeculativeTimeoutMs,
-                speculativeBackoffMultiplier);
-        // shuffle the streams
-        Collections.shuffle(this.streams);
-        this.nextStreamId = new AtomicInteger(0);
-        this.recordSetWriter = newRecordSetWriter();
-
-        if (flushIntervalMicros > 0) {
-            this.scheduler.scheduleAtFixedRate(
-                    this,
-                    flushIntervalMicros,
-                    flushIntervalMicros,
-                    TimeUnit.MICROSECONDS);
-        }
-    }
-
-    String getStream(int streamId) {
-        return streams.get(streamId);
-    }
-
-    synchronized LogRecordSet.Writer getLogRecordSetWriter() {
-        return recordSetWriter;
-    }
-
-    private LogRecordSet.Writer newRecordSetWriter() {
-        return LogRecordSet.newWriter(
-                bufferSize,
-                codec);
-    }
-
-    public synchronized Future<DLSN> write(ByteBuffer buffer) {
-        int logRecordSize = buffer.remaining();
-        if (logRecordSize > MAX_LOGRECORD_SIZE) {
-            return Future.exception(new LogRecordTooLongException(
-                    "Log record of size " + logRecordSize + " written when only "
-                            + MAX_LOGRECORD_SIZE + " is allowed"));
-        }
-        // if exceed max number of bytes
-        if ((recordSetWriter.getNumBytes() + logRecordSize) > MAX_LOGRECORDSET_SIZE) {
-            flush();
-        }
-        Promise<DLSN> writePromise = new Promise<DLSN>();
-        try {
-            recordSetWriter.writeRecord(buffer, writePromise);
-        } catch (LogRecordTooLongException e) {
-            return Future.exception(e);
-        } catch (WriteException e) {
-            recordSetWriter.abortTransmit(e);
-            recordSetWriter = newRecordSetWriter();
-            return Future.exception(e);
-        }
-        if (recordSetWriter.getNumBytes() >= bufferSize) {
-            flush();
-        }
-        return writePromise;
-    }
-
-    @Override
-    public void run() {
-        flush();
-    }
-
-    private void flush() {
-        LogRecordSet.Writer recordSetToFlush;
-        synchronized (this) {
-            if (recordSetWriter.getNumRecords() == 0) {
-                return;
-            }
-            recordSetToFlush = recordSetWriter;
-            recordSetWriter = newRecordSetWriter();
-        }
-        transmit(recordSetToFlush);
-    }
-
-    private void transmit(LogRecordSet.Writer recordSetToFlush) {
-        PendingWriteRequest writeRequest =
-                new PendingWriteRequest(recordSetToFlush);
-        this.speculativePolicy.initiateSpeculativeRequest(scheduler, writeRequest);
-    }
-
-    public void close() {
-        if (ownScheduler) {
-            this.scheduler.shutdown();
-        }
-    }
-
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/MonitorServiceClient.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/MonitorServiceClient.java
deleted file mode 100644
index e541578..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/MonitorServiceClient.java
+++ /dev/null
@@ -1,68 +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 com.twitter.distributedlog.client.monitor;
-
-import com.twitter.util.Future;
-import java.net.SocketAddress;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Interface for distributedlog monitor service.
- */
-public interface MonitorServiceClient {
-
-    /**
-     * Check a given stream.
-     *
-     * @param stream
-     *          stream.
-     * @return check result.
-     */
-    Future<Void> check(String stream);
-
-    /**
-     * Send heartbeat to the stream and its readers.
-     *
-     * @param stream
-     *          stream.
-     * @return check result.
-     */
-    Future<Void> heartbeat(String stream);
-
-    /**
-     * Get current ownership distribution from current monitor service view.
-     *
-     * @return current ownership distribution
-     */
-    Map<SocketAddress, Set<String>> getStreamOwnershipDistribution();
-
-    /**
-     * Enable/Disable accepting new stream on a given proxy.
-     *
-     * @param enabled
-     *          flag to enable/disable accepting new streams on a given proxy
-     * @return void
-     */
-    Future<Void> setAcceptNewStream(boolean enabled);
-
-    /**
-     * Close the client.
-     */
-    void close();
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/package-info.java
deleted file mode 100644
index c4e7df0..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Monitor Client.
- */
-package com.twitter.distributedlog.client.monitor;
\ No newline at end of file
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/OwnershipCache.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/OwnershipCache.java
deleted file mode 100644
index 387d727..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/OwnershipCache.java
+++ /dev/null
@@ -1,235 +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 com.twitter.distributedlog.client.ownership;
-
-import com.google.common.collect.ImmutableMap;
-import com.twitter.distributedlog.client.ClientConfig;
-import com.twitter.distributedlog.client.stats.OwnershipStatsLogger;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.jboss.netty.util.Timeout;
-import org.jboss.netty.util.TimerTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Client Side Ownership Cache.
- */
-public class OwnershipCache implements TimerTask {
-
-    private static final Logger logger = LoggerFactory.getLogger(OwnershipCache.class);
-
-    private final ConcurrentHashMap<String, SocketAddress> stream2Addresses =
-            new ConcurrentHashMap<String, SocketAddress>();
-    private final ConcurrentHashMap<SocketAddress, Set<String>> address2Streams =
-            new ConcurrentHashMap<SocketAddress, Set<String>>();
-    private final ClientConfig clientConfig;
-    private final HashedWheelTimer timer;
-
-    // Stats
-    private final OwnershipStatsLogger ownershipStatsLogger;
-
-    public OwnershipCache(ClientConfig clientConfig,
-                          HashedWheelTimer timer,
-                          StatsReceiver statsReceiver,
-                          StatsReceiver streamStatsReceiver) {
-        this.clientConfig = clientConfig;
-        this.timer = timer;
-        this.ownershipStatsLogger = new OwnershipStatsLogger(statsReceiver, streamStatsReceiver);
-        scheduleDumpOwnershipCache();
-    }
-
-    private void scheduleDumpOwnershipCache() {
-        if (clientConfig.isPeriodicDumpOwnershipCacheEnabled()
-            && clientConfig.getPeriodicDumpOwnershipCacheIntervalMs() > 0) {
-            timer.newTimeout(this, clientConfig.getPeriodicDumpOwnershipCacheIntervalMs(),
-                    TimeUnit.MILLISECONDS);
-        }
-    }
-
-    @Override
-    public void run(Timeout timeout) throws Exception {
-        if (timeout.isCancelled()) {
-            return;
-        }
-        logger.info("Ownership cache : {} streams cached, {} hosts cached",
-                stream2Addresses.size(), address2Streams.size());
-        logger.info("Cached streams : {}", stream2Addresses);
-        scheduleDumpOwnershipCache();
-    }
-
-    public OwnershipStatsLogger getOwnershipStatsLogger() {
-        return ownershipStatsLogger;
-    }
-
-    /**
-     * Update ownership of <i>stream</i> to <i>addr</i>.
-     *
-     * @param stream
-     *          Stream Name.
-     * @param addr
-     *          Owner Address.
-     * @return true if owner is updated
-     */
-    public boolean updateOwner(String stream, SocketAddress addr) {
-        // update ownership
-        SocketAddress oldAddr = stream2Addresses.putIfAbsent(stream, addr);
-        if (null != oldAddr && oldAddr.equals(addr)) {
-            return true;
-        }
-        if (null != oldAddr) {
-            if (stream2Addresses.replace(stream, oldAddr, addr)) {
-                // Store the relevant mappings for this topic and host combination
-                logger.info("Storing ownership for stream : {}, old host : {}, new host : {}.",
-                        new Object[] { stream, oldAddr, addr });
-                StringBuilder sb = new StringBuilder();
-                sb.append("Ownership changed '")
-                  .append(oldAddr).append("' -> '").append(addr).append("'");
-                removeOwnerFromStream(stream, oldAddr, sb.toString());
-
-                // update stats
-                ownershipStatsLogger.onRemove(stream);
-                ownershipStatsLogger.onAdd(stream);
-            } else {
-                logger.warn("Ownership of stream : {} has been changed from {} to {} when storing host : {}.",
-                        new Object[] { stream, oldAddr, stream2Addresses.get(stream), addr });
-                return false;
-            }
-        } else {
-            logger.info("Storing ownership for stream : {}, host : {}.", stream, addr);
-            // update stats
-            ownershipStatsLogger.onAdd(stream);
-        }
-
-        Set<String> streamsForHost = address2Streams.get(addr);
-        if (null == streamsForHost) {
-            Set<String> newStreamsForHost = new HashSet<String>();
-            streamsForHost = address2Streams.putIfAbsent(addr, newStreamsForHost);
-            if (null == streamsForHost) {
-                streamsForHost = newStreamsForHost;
-            }
-        }
-        synchronized (streamsForHost) {
-            // check whether the ownership changed, since it might happend after replace succeed
-            if (addr.equals(stream2Addresses.get(stream))) {
-                streamsForHost.add(stream);
-            }
-        }
-        return true;
-    }
-
-    /**
-     * Get the cached owner for stream <code>stream</code>.
-     *
-     * @param stream
-     *          stream to lookup ownership
-     * @return owner's address
-     */
-    public SocketAddress getOwner(String stream) {
-        SocketAddress address = stream2Addresses.get(stream);
-        if (null == address) {
-            ownershipStatsLogger.onMiss(stream);
-        } else {
-            ownershipStatsLogger.onHit(stream);
-        }
-        return address;
-    }
-
-    /**
-     * Remove the owner <code>addr</code> from <code>stream</code> for a given <code>reason</code>.
-     *
-     * @param stream stream name
-     * @param addr owner address
-     * @param reason reason to remove ownership
-     */
-    public void removeOwnerFromStream(String stream, SocketAddress addr, String reason) {
-        if (stream2Addresses.remove(stream, addr)) {
-            logger.info("Removed stream to host mapping for (stream: {} -> host: {}) : reason = '{}'.",
-                    new Object[] { stream, addr, reason });
-        }
-        Set<String> streamsForHost = address2Streams.get(addr);
-        if (null != streamsForHost) {
-            synchronized (streamsForHost) {
-                if (streamsForHost.remove(stream)) {
-                    logger.info("Removed stream ({}) from host {} : reason = '{}'.",
-                            new Object[] { stream, addr, reason });
-                    if (streamsForHost.isEmpty()) {
-                        address2Streams.remove(addr, streamsForHost);
-                    }
-                    ownershipStatsLogger.onRemove(stream);
-                }
-            }
-        }
-    }
-
-    /**
-     * Remove all streams from host <code>addr</code>.
-     *
-     * @param addr
-     *          host to remove ownerships
-     */
-    public void removeAllStreamsFromOwner(SocketAddress addr) {
-        logger.info("Remove streams mapping for host {}", addr);
-        Set<String> streamsForHost = address2Streams.get(addr);
-        if (null != streamsForHost) {
-            synchronized (streamsForHost) {
-                for (String s : streamsForHost) {
-                    if (stream2Addresses.remove(s, addr)) {
-                        logger.info("Removing mapping for stream : {} from host : {}", s, addr);
-                        ownershipStatsLogger.onRemove(s);
-                    }
-                }
-                address2Streams.remove(addr, streamsForHost);
-            }
-        }
-    }
-
-    /**
-     * Get the number cached streams.
-     *
-     * @return number cached streams.
-     */
-    public int getNumCachedStreams() {
-        return stream2Addresses.size();
-    }
-
-    /**
-     * Get the stream ownership distribution across proxies.
-     *
-     * @return stream ownership distribution
-     */
-    public Map<SocketAddress, Set<String>> getStreamOwnershipDistribution() {
-        return ImmutableMap.copyOf(address2Streams);
-    }
-
-    /**
-     * Get the stream ownership mapping.
-     *
-     * @return stream ownership mapping.
-     */
-    public Map<String, SocketAddress> getStreamOwnerMapping() {
-        return stream2Addresses;
-    }
-
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/package-info.java
deleted file mode 100644
index 721702e..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Utils for managing ownership at client side.
- */
-package com.twitter.distributedlog.client.ownership;
\ No newline at end of file
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/package-info.java
deleted file mode 100644
index aa167fb..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Client.
- */
-package com.twitter.distributedlog.client;
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ClusterClient.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ClusterClient.java
deleted file mode 100644
index f8bdae7..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ClusterClient.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 com.twitter.distributedlog.client.proxy;
-
-import com.twitter.distributedlog.thrift.service.DistributedLogService;
-import com.twitter.finagle.Service;
-import com.twitter.finagle.thrift.ThriftClientRequest;
-import com.twitter.util.Future;
-import scala.runtime.BoxedUnit;
-
-/**
- * Cluster client.
- */
-public class ClusterClient {
-
-    private final Service<ThriftClientRequest, byte[]> client;
-    private final DistributedLogService.ServiceIface service;
-
-    public ClusterClient(Service<ThriftClientRequest, byte[]> client,
-                         DistributedLogService.ServiceIface service) {
-        this.client = client;
-        this.service = service;
-    }
-
-    public Service<ThriftClientRequest, byte[]> getClient() {
-        return client;
-    }
-
-    public DistributedLogService.ServiceIface getService() {
-        return service;
-    }
-
-    public Future<BoxedUnit> close() {
-        return client.close();
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/HostProvider.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/HostProvider.java
deleted file mode 100644
index 4878c1c..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/HostProvider.java
+++ /dev/null
@@ -1,35 +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 com.twitter.distributedlog.client.proxy;
-
-import java.net.SocketAddress;
-import java.util.Set;
-
-/**
- * Provider to provider list of hosts for handshaking.
- */
-public interface HostProvider {
-
-    /**
-     * Get the list of hosts for handshaking.
-     *
-     * @return list of hosts for handshaking.
-     */
-    Set<SocketAddress> getHosts();
-
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClient.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClient.java
deleted file mode 100644
index d131e28..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClient.java
+++ /dev/null
@@ -1,165 +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 com.twitter.distributedlog.client.proxy;
-
-import com.twitter.distributedlog.client.ClientConfig;
-import com.twitter.distributedlog.client.stats.ClientStats;
-import com.twitter.distributedlog.thrift.service.DistributedLogService;
-import com.twitter.finagle.Service;
-import com.twitter.finagle.ThriftMux;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.thrift.ClientId;
-import com.twitter.finagle.thrift.ThriftClientFramedCodec;
-import com.twitter.finagle.thrift.ThriftClientRequest;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import scala.Option;
-import scala.runtime.BoxedUnit;
-
-/**
- * Client talks to a single proxy.
- */
-public class ProxyClient {
-
-  /**
-   * Builder to build a proxy client talking to given host <code>address</code>.
-   */
-  public interface Builder {
-        /**
-         * Build a proxy client to <code>address</code>.
-         *
-         * @param address
-         *          proxy address
-         * @return proxy client
-         */
-        ProxyClient build(SocketAddress address);
-    }
-
-    public static Builder newBuilder(String clientName,
-                                     ClientId clientId,
-                                     ClientBuilder clientBuilder,
-                                     ClientConfig clientConfig,
-                                     ClientStats clientStats) {
-        return new DefaultBuilder(clientName, clientId, clientBuilder, clientConfig, clientStats);
-    }
-
-    /**
-     * Default Builder for {@link ProxyClient}.
-     */
-    public static class DefaultBuilder implements Builder {
-
-        private final String clientName;
-        private final ClientId clientId;
-        private final ClientBuilder clientBuilder;
-        private final ClientStats clientStats;
-
-        private DefaultBuilder(String clientName,
-                               ClientId clientId,
-                               ClientBuilder clientBuilder,
-                               ClientConfig clientConfig,
-                               ClientStats clientStats) {
-            this.clientName = clientName;
-            this.clientId = clientId;
-            this.clientStats = clientStats;
-            // client builder
-            ClientBuilder builder = setDefaultSettings(
-                    null == clientBuilder ? getDefaultClientBuilder(clientConfig) : clientBuilder);
-            this.clientBuilder = configureThriftMux(builder, clientId, clientConfig);
-        }
-
-        @SuppressWarnings("unchecked")
-        private ClientBuilder configureThriftMux(ClientBuilder builder,
-                                                 ClientId clientId,
-                                                 ClientConfig clientConfig) {
-            if (clientConfig.getThriftMux()) {
-                return builder.stack(ThriftMux.client().withClientId(clientId));
-            } else {
-                return builder.codec(ThriftClientFramedCodec.apply(Option.apply(clientId)));
-            }
-        }
-
-        private ClientBuilder getDefaultClientBuilder(ClientConfig clientConfig) {
-            ClientBuilder builder = ClientBuilder.get()
-                .tcpConnectTimeout(Duration.fromMilliseconds(200))
-                .connectTimeout(Duration.fromMilliseconds(200))
-                .requestTimeout(Duration.fromSeconds(1));
-            if (!clientConfig.getThriftMux()) {
-                builder = builder.hostConnectionLimit(1);
-            }
-            return builder;
-        }
-
-        @SuppressWarnings("unchecked")
-        private ClientBuilder setDefaultSettings(ClientBuilder builder) {
-            return builder.name(clientName)
-                   .failFast(false)
-                   .noFailureAccrual()
-                   // disable retries on finagle client builder, as there is only one host per finagle client
-                   // we should throw exception immediately on first failure, so DL client could quickly detect
-                   // failures and retry other proxies.
-                   .retries(1)
-                   .keepAlive(true);
-        }
-
-        @Override
-        @SuppressWarnings("unchecked")
-        public ProxyClient build(SocketAddress address) {
-            Service<ThriftClientRequest, byte[]> client =
-                ClientBuilder.safeBuildFactory(
-                        clientBuilder
-                                .hosts((InetSocketAddress) address)
-                                .reportTo(clientStats.getFinagleStatsReceiver(address))
-                ).toService();
-            DistributedLogService.ServiceIface service =
-                    new DistributedLogService.ServiceToClient(client, new TBinaryProtocol.Factory());
-            return new ProxyClient(address, client, service);
-        }
-
-    }
-
-    private final SocketAddress address;
-    private final Service<ThriftClientRequest, byte[]> client;
-    private final DistributedLogService.ServiceIface service;
-
-    protected ProxyClient(SocketAddress address,
-                          Service<ThriftClientRequest, byte[]> client,
-                          DistributedLogService.ServiceIface service) {
-        this.address = address;
-        this.client  = client;
-        this.service = service;
-    }
-
-    public SocketAddress getAddress() {
-        return address;
-    }
-
-    public Service<ThriftClientRequest, byte[]> getClient() {
-        return client;
-    }
-
-    public DistributedLogService.ServiceIface getService() {
-        return service;
-    }
-
-    public Future<BoxedUnit> close() {
-        return client.close();
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClientManager.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClientManager.java
deleted file mode 100644
index c7d56f6..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClientManager.java
+++ /dev/null
@@ -1,362 +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 com.twitter.distributedlog.client.proxy;
-
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.ImmutableMap;
-import com.twitter.distributedlog.client.ClientConfig;
-import com.twitter.distributedlog.client.stats.ClientStats;
-import com.twitter.distributedlog.client.stats.OpStats;
-import com.twitter.distributedlog.thrift.service.ClientInfo;
-import com.twitter.distributedlog.thrift.service.ServerInfo;
-import com.twitter.util.FutureEventListener;
-import java.net.SocketAddress;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.jboss.netty.util.Timeout;
-import org.jboss.netty.util.TimerTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Manager manages clients (channels) to proxies.
- */
-public class ProxyClientManager implements TimerTask {
-
-    private static final Logger logger = LoggerFactory.getLogger(ProxyClientManager.class);
-
-    private final ClientConfig clientConfig;
-    private final ProxyClient.Builder clientBuilder;
-    private final HashedWheelTimer timer;
-    private final HostProvider hostProvider;
-    private volatile Timeout periodicHandshakeTask;
-    private final ConcurrentHashMap<SocketAddress, ProxyClient> address2Services =
-            new ConcurrentHashMap<SocketAddress, ProxyClient>();
-    private final CopyOnWriteArraySet<ProxyListener> proxyListeners =
-            new CopyOnWriteArraySet<ProxyListener>();
-    private volatile boolean closed = false;
-    private volatile boolean periodicHandshakeEnabled = true;
-    private final Stopwatch lastOwnershipSyncStopwatch;
-
-    private final OpStats handshakeStats;
-
-    public ProxyClientManager(ClientConfig clientConfig,
-                              ProxyClient.Builder clientBuilder,
-                              HashedWheelTimer timer,
-                              HostProvider hostProvider,
-                              ClientStats clientStats) {
-        this.clientConfig = clientConfig;
-        this.clientBuilder = clientBuilder;
-        this.timer = timer;
-        this.hostProvider = hostProvider;
-        this.handshakeStats = clientStats.getOpStats("handshake");
-        scheduleHandshake();
-        this.lastOwnershipSyncStopwatch = Stopwatch.createStarted();
-    }
-
-    private void scheduleHandshake() {
-        if (clientConfig.getPeriodicHandshakeIntervalMs() > 0) {
-            periodicHandshakeTask = timer.newTimeout(this,
-                    clientConfig.getPeriodicHandshakeIntervalMs(), TimeUnit.MILLISECONDS);
-        }
-    }
-
-    void setPeriodicHandshakeEnabled(boolean enabled) {
-        this.periodicHandshakeEnabled = enabled;
-    }
-
-    @Override
-    public void run(Timeout timeout) throws Exception {
-        if (timeout.isCancelled() || closed) {
-            return;
-        }
-        if (periodicHandshakeEnabled) {
-            final boolean syncOwnerships = lastOwnershipSyncStopwatch.elapsed(TimeUnit.MILLISECONDS)
-                >= clientConfig.getPeriodicOwnershipSyncIntervalMs();
-
-            final Set<SocketAddress> hostsSnapshot = hostProvider.getHosts();
-            final AtomicInteger numHosts = new AtomicInteger(hostsSnapshot.size());
-            final AtomicInteger numStreams = new AtomicInteger(0);
-            final AtomicInteger numSuccesses = new AtomicInteger(0);
-            final AtomicInteger numFailures = new AtomicInteger(0);
-            final ConcurrentMap<SocketAddress, Integer> streamDistributions =
-                    new ConcurrentHashMap<SocketAddress, Integer>();
-            final Stopwatch stopwatch = Stopwatch.createStarted();
-            for (SocketAddress host : hostsSnapshot) {
-                final SocketAddress address = host;
-                final ProxyClient client = getClient(address);
-                handshake(address, client, new FutureEventListener<ServerInfo>() {
-                    @Override
-                    public void onSuccess(ServerInfo serverInfo) {
-                        numStreams.addAndGet(serverInfo.getOwnershipsSize());
-                        numSuccesses.incrementAndGet();
-                        notifyHandshakeSuccess(address, client, serverInfo, false, stopwatch);
-                        if (clientConfig.isHandshakeTracingEnabled()) {
-                            streamDistributions.putIfAbsent(address, serverInfo.getOwnershipsSize());
-                        }
-                        complete();
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        numFailures.incrementAndGet();
-                        notifyHandshakeFailure(address, client, cause, stopwatch);
-                        complete();
-                    }
-
-                    private void complete() {
-                        if (0 == numHosts.decrementAndGet()) {
-                            if (syncOwnerships) {
-                                logger.info("Periodic handshaked with {} hosts : {} streams returned,"
-                                    + " {} hosts succeeded, {} hosts failed",
-                                    new Object[] {
-                                        hostsSnapshot.size(),
-                                        numStreams.get(),
-                                        numSuccesses.get(),
-                                        numFailures.get()});
-                                if (clientConfig.isHandshakeTracingEnabled()) {
-                                    logger.info("Periodic handshaked stream distribution : {}", streamDistributions);
-                                }
-                            }
-                        }
-                    }
-                }, false, syncOwnerships);
-            }
-
-            if (syncOwnerships) {
-                lastOwnershipSyncStopwatch.reset().start();
-            }
-        }
-        scheduleHandshake();
-    }
-
-    /**
-     * Register a proxy <code>listener</code> on proxy related changes.
-     *
-     * @param listener
-     *          proxy listener
-     */
-    public void registerProxyListener(ProxyListener listener) {
-        proxyListeners.add(listener);
-    }
-
-    private void notifyHandshakeSuccess(SocketAddress address,
-                                        ProxyClient client,
-                                        ServerInfo serverInfo,
-                                        boolean logging,
-                                        Stopwatch stopwatch) {
-        if (logging) {
-            if (null != serverInfo && serverInfo.isSetOwnerships()) {
-                logger.info("Handshaked with {} : {} ownerships returned.",
-                        address, serverInfo.getOwnerships().size());
-            } else {
-                logger.info("Handshaked with {} : no ownerships returned", address);
-            }
-        }
-        handshakeStats.completeRequest(address, stopwatch.elapsed(TimeUnit.MICROSECONDS), 1);
-        for (ProxyListener listener : proxyListeners) {
-            listener.onHandshakeSuccess(address, client, serverInfo);
-        }
-    }
-
-    private void notifyHandshakeFailure(SocketAddress address,
-                                        ProxyClient client,
-                                        Throwable cause,
-                                        Stopwatch stopwatch) {
-        handshakeStats.failRequest(address, stopwatch.elapsed(TimeUnit.MICROSECONDS), 1);
-        for (ProxyListener listener : proxyListeners) {
-            listener.onHandshakeFailure(address, client, cause);
-        }
-    }
-
-    /**
-     * Retrieve a client to proxy <code>address</code>.
-     *
-     * @param address
-     *          proxy address
-     * @return proxy client
-     */
-    public ProxyClient getClient(final SocketAddress address) {
-        ProxyClient sc = address2Services.get(address);
-        if (null != sc) {
-            return sc;
-        }
-        return createClient(address);
-    }
-
-    /**
-     * Remove the client to proxy <code>address</code>.
-     *
-     * @param address
-     *          proxy address
-     */
-    public void removeClient(SocketAddress address) {
-        ProxyClient sc = address2Services.remove(address);
-        if (null != sc) {
-            logger.info("Removed host {}.", address);
-            sc.close();
-        }
-    }
-
-    /**
-     * Remove the client <code>sc</code> to proxy <code>address</code>.
-     *
-     * @param address
-     *          proxy address
-     * @param sc
-     *          proxy client
-     */
-    public void removeClient(SocketAddress address, ProxyClient sc) {
-        if (address2Services.remove(address, sc)) {
-            logger.info("Remove client {} to host {}.", sc, address);
-            sc.close();
-        }
-    }
-
-    /**
-     * Create a client to proxy <code>address</code>.
-     *
-     * @param address
-     *          proxy address
-     * @return proxy client
-     */
-    public ProxyClient createClient(final SocketAddress address) {
-        final ProxyClient sc = clientBuilder.build(address);
-        ProxyClient oldSC = address2Services.putIfAbsent(address, sc);
-        if (null != oldSC) {
-            sc.close();
-            return oldSC;
-        } else {
-            final Stopwatch stopwatch = Stopwatch.createStarted();
-            FutureEventListener<ServerInfo> listener = new FutureEventListener<ServerInfo>() {
-                @Override
-                public void onSuccess(ServerInfo serverInfo) {
-                    notifyHandshakeSuccess(address, sc, serverInfo, true, stopwatch);
-                }
-                @Override
-                public void onFailure(Throwable cause) {
-                    notifyHandshakeFailure(address, sc, cause, stopwatch);
-                }
-            };
-            // send a ping messaging after creating connections.
-            handshake(address, sc, listener, true, true);
-            return sc;
-        }
-    }
-
-    /**
-     * Handshake with a given proxy.
-     *
-     * @param address
-     *          proxy address
-     * @param sc
-     *          proxy client
-     * @param listener
-     *          listener on handshake result
-     */
-    private void handshake(SocketAddress address,
-                           ProxyClient sc,
-                           FutureEventListener<ServerInfo> listener,
-                           boolean logging,
-                           boolean getOwnerships) {
-        if (clientConfig.getHandshakeWithClientInfo()) {
-            ClientInfo clientInfo = new ClientInfo();
-            clientInfo.setGetOwnerships(getOwnerships);
-            clientInfo.setStreamNameRegex(clientConfig.getStreamNameRegex());
-            if (logging) {
-                logger.info("Handshaking with {} : {}", address, clientInfo);
-            }
-            sc.getService().handshakeWithClientInfo(clientInfo)
-                    .addEventListener(listener);
-        } else {
-            if (logging) {
-                logger.info("Handshaking with {}", address);
-            }
-            sc.getService().handshake().addEventListener(listener);
-        }
-    }
-
-    /**
-     * Handshake with all proxies.
-     *
-     * <p>NOTE: this is a synchronous call.
-     */
-    public void handshake() {
-        Set<SocketAddress> hostsSnapshot = hostProvider.getHosts();
-        logger.info("Handshaking with {} hosts.", hostsSnapshot.size());
-        final CountDownLatch latch = new CountDownLatch(hostsSnapshot.size());
-        final Stopwatch stopwatch = Stopwatch.createStarted();
-        for (SocketAddress host: hostsSnapshot) {
-            final SocketAddress address = host;
-            final ProxyClient client = getClient(address);
-            handshake(address, client, new FutureEventListener<ServerInfo>() {
-                @Override
-                public void onSuccess(ServerInfo serverInfo) {
-                    notifyHandshakeSuccess(address, client, serverInfo, true, stopwatch);
-                    latch.countDown();
-                }
-                @Override
-                public void onFailure(Throwable cause) {
-                    notifyHandshakeFailure(address, client, cause, stopwatch);
-                    latch.countDown();
-                }
-            }, true, true);
-        }
-        try {
-            latch.await(1, TimeUnit.MINUTES);
-        } catch (InterruptedException e) {
-            logger.warn("Interrupted on handshaking with servers : ", e);
-        }
-    }
-
-    /**
-     * Return number of proxies managed by client manager.
-     *
-     * @return number of proxies managed by client manager.
-     */
-    public int getNumProxies() {
-        return address2Services.size();
-    }
-
-    /**
-     * Return all clients.
-     *
-     * @return all clients.
-     */
-    public Map<SocketAddress, ProxyClient> getAllClients() {
-        return ImmutableMap.copyOf(address2Services);
-    }
-
-    public void close() {
-        closed = true;
-        Timeout task = periodicHandshakeTask;
-        if (null != task) {
-            task.cancel();
-        }
-        for (ProxyClient sc : address2Services.values()) {
-            sc.close();
-        }
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyListener.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyListener.java
deleted file mode 100644
index e024825..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyListener.java
+++ /dev/null
@@ -1,50 +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 com.twitter.distributedlog.client.proxy;
-
-import com.twitter.distributedlog.thrift.service.ServerInfo;
-import java.net.SocketAddress;
-
-/**
- * Listener on server changes.
- */
-public interface ProxyListener {
-    /**
-     * When a proxy's server info changed, it would be notified.
-     *
-     * @param address
-     *          proxy address
-     * @param client
-     *          proxy client that executes handshaking
-     * @param serverInfo
-     *          proxy's server info
-     */
-    void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo);
-
-    /**
-     * Failed to handshake with a proxy.
-     *
-     * @param address
-     *          proxy address
-     * @param client
-     *          proxy client
-     * @param cause
-     *          failure reason
-     */
-    void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause);
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/package-info.java
deleted file mode 100644
index dc28c76..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Clients that interact with individual proxies.
- */
-package com.twitter.distributedlog.client.proxy;
\ No newline at end of file
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/DefaultRegionResolver.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/DefaultRegionResolver.java
deleted file mode 100644
index ab2fbed..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/DefaultRegionResolver.java
+++ /dev/null
@@ -1,85 +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 com.twitter.distributedlog.client.resolver;
-
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Default implementation of {@link RegionResolver}.
- */
-public class DefaultRegionResolver implements RegionResolver {
-
-    private static final String DEFAULT_REGION = "default-region";
-
-    private final Map<SocketAddress, String> regionOverrides =
-            new HashMap<SocketAddress, String>();
-    private final ConcurrentMap<SocketAddress, String> regionMap =
-            new ConcurrentHashMap<SocketAddress, String>();
-
-    public DefaultRegionResolver() {
-    }
-
-    public DefaultRegionResolver(Map<SocketAddress, String> regionOverrides) {
-        this.regionOverrides.putAll(regionOverrides);
-    }
-
-    @Override
-    public String resolveRegion(SocketAddress address) {
-        String region = regionMap.get(address);
-        if (null == region) {
-            region = doResolveRegion(address);
-            regionMap.put(address, region);
-        }
-        return region;
-    }
-
-    private String doResolveRegion(SocketAddress address) {
-        String region = regionOverrides.get(address);
-        if (null != region) {
-            return region;
-        }
-
-        String domainName;
-        if (address instanceof InetSocketAddress) {
-            InetSocketAddress iAddr = (InetSocketAddress) address;
-            domainName = iAddr.getHostName();
-        } else {
-            domainName = address.toString();
-        }
-        String[] parts = domainName.split("\\.");
-        if (parts.length <= 0) {
-            return DEFAULT_REGION;
-        }
-        String hostName = parts[0];
-        String[] labels = hostName.split("-");
-        if (labels.length != 4) {
-            return DEFAULT_REGION;
-        }
-        return labels[0];
-    }
-
-    @Override
-    public void removeCachedHost(SocketAddress address) {
-        regionMap.remove(address);
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/RegionResolver.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/RegionResolver.java
deleted file mode 100644
index eff3aad..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/RegionResolver.java
+++ /dev/null
@@ -1,43 +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 com.twitter.distributedlog.client.resolver;
-
-import java.net.SocketAddress;
-
-/**
- * Resolve address to region.
- */
-public interface RegionResolver {
-
-    /**
-     * Resolve address to region.
-     *
-     * @param address
-     *          socket address
-     * @return region
-     */
-    String resolveRegion(SocketAddress address);
-
-    /**
-     * Remove cached host.
-     *
-     * @param address
-     *          socket address.
-     */
-    void removeCachedHost(SocketAddress address);
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/package-info.java
deleted file mode 100644
index 4bb53a5..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Resolver to resolve network addresses.
- */
-package com.twitter.distributedlog.client.resolver;
\ No newline at end of file
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ConsistentHashRoutingService.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ConsistentHashRoutingService.java
deleted file mode 100644
index 6d1e37e..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ConsistentHashRoutingService.java
+++ /dev/null
@@ -1,500 +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 com.twitter.distributedlog.client.routing;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.MapDifference;
-import com.google.common.collect.Maps;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hashing;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.finagle.ChannelException;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.Counter;
-import com.twitter.finagle.stats.Gauge;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.util.Function0;
-import java.net.SocketAddress;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.commons.lang3.tuple.Pair;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.jboss.netty.util.Timeout;
-import org.jboss.netty.util.TimerTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.collection.Seq;
-
-/**
- * Consistent Hashing Based {@link RoutingService}.
- */
-public class ConsistentHashRoutingService extends ServerSetRoutingService {
-
-    private static final Logger logger = LoggerFactory.getLogger(ConsistentHashRoutingService.class);
-
-    @Deprecated
-    public static ConsistentHashRoutingService of(ServerSetWatcher serverSetWatcher, int numReplicas) {
-        return new ConsistentHashRoutingService(serverSetWatcher, numReplicas, 300, NullStatsReceiver.get());
-    }
-
-    /**
-     * Builder helper class to build a consistent hash bashed {@link RoutingService}.
-     *
-     * @return builder to build a consistent hash based {@link RoutingService}.
-     */
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    /**
-     * Builder for building consistent hash based routing service.
-     */
-    public static class Builder implements RoutingService.Builder {
-
-        private ServerSet serverSet;
-        private boolean resolveFromName = false;
-        private int numReplicas;
-        private int blackoutSeconds = 300;
-        private StatsReceiver statsReceiver = NullStatsReceiver.get();
-
-        private Builder() {}
-
-        public Builder serverSet(ServerSet serverSet) {
-            this.serverSet = serverSet;
-            return this;
-        }
-
-        public Builder resolveFromName(boolean enabled) {
-            this.resolveFromName = enabled;
-            return this;
-        }
-
-        public Builder numReplicas(int numReplicas) {
-            this.numReplicas = numReplicas;
-            return this;
-        }
-
-        public Builder blackoutSeconds(int seconds) {
-            this.blackoutSeconds = seconds;
-            return this;
-        }
-
-        public Builder statsReceiver(StatsReceiver statsReceiver) {
-            this.statsReceiver = statsReceiver;
-            return this;
-        }
-
-        @Override
-        public RoutingService build() {
-            checkNotNull(serverSet, "No serverset provided.");
-            checkNotNull(statsReceiver, "No stats receiver provided.");
-            checkArgument(numReplicas > 0, "Invalid number of replicas : " + numReplicas);
-            return new ConsistentHashRoutingService(new TwitterServerSetWatcher(serverSet, resolveFromName),
-                numReplicas, blackoutSeconds, statsReceiver);
-        }
-    }
-
-    static class ConsistentHash {
-        private final HashFunction hashFunction;
-        private final int numOfReplicas;
-        private final SortedMap<Long, SocketAddress> circle;
-
-        // Stats
-        protected final Counter hostAddedCounter;
-        protected final Counter hostRemovedCounter;
-
-        ConsistentHash(HashFunction hashFunction,
-                       int numOfReplicas,
-                       StatsReceiver statsReceiver) {
-            this.hashFunction = hashFunction;
-            this.numOfReplicas = numOfReplicas;
-            this.circle = new TreeMap<Long, SocketAddress>();
-
-            this.hostAddedCounter = statsReceiver.counter0("adds");
-            this.hostRemovedCounter = statsReceiver.counter0("removes");
-        }
-
-        private String replicaName(int shardId, int replica, String address) {
-            if (shardId < 0) {
-                shardId = UNKNOWN_SHARD_ID;
-            }
-
-            StringBuilder sb = new StringBuilder(100);
-            sb.append("shard-");
-            sb.append(shardId);
-            sb.append('-');
-            sb.append(replica);
-            sb.append('-');
-            sb.append(address);
-
-            return sb.toString();
-        }
-
-        private Long replicaHash(int shardId, int replica, String address) {
-            return hashFunction.hashUnencodedChars(replicaName(shardId, replica, address)).asLong();
-        }
-
-        private Long replicaHash(int shardId, int replica, SocketAddress address) {
-            return replicaHash(shardId, replica, address.toString());
-        }
-
-        public synchronized void add(int shardId, SocketAddress address) {
-            String addressStr = address.toString();
-            for (int i = 0; i < numOfReplicas; i++) {
-                Long hash = replicaHash(shardId, i, addressStr);
-                circle.put(hash, address);
-            }
-            hostAddedCounter.incr();
-        }
-
-        public synchronized void remove(int shardId, SocketAddress address) {
-            for (int i = 0; i < numOfReplicas; i++) {
-                long hash = replicaHash(shardId, i, address);
-                SocketAddress oldAddress = circle.get(hash);
-                if (null != oldAddress && oldAddress.equals(address)) {
-                    circle.remove(hash);
-                }
-            }
-            hostRemovedCounter.incr();
-        }
-
-        public SocketAddress get(String key, RoutingContext rContext) {
-            long hash = hashFunction.hashUnencodedChars(key).asLong();
-            return find(hash, rContext);
-        }
-
-        private synchronized SocketAddress find(long hash, RoutingContext rContext) {
-            if (circle.isEmpty()) {
-                return null;
-            }
-
-            Iterator<Map.Entry<Long, SocketAddress>> iterator =
-                    circle.tailMap(hash).entrySet().iterator();
-            while (iterator.hasNext()) {
-                Map.Entry<Long, SocketAddress> entry = iterator.next();
-                if (!rContext.isTriedHost(entry.getValue())) {
-                    return entry.getValue();
-                }
-            }
-            // the tail map has been checked
-            iterator = circle.headMap(hash).entrySet().iterator();
-            while (iterator.hasNext()) {
-                Map.Entry<Long, SocketAddress> entry = iterator.next();
-                if (!rContext.isTriedHost(entry.getValue())) {
-                    return entry.getValue();
-                }
-            }
-
-            return null;
-        }
-
-        private synchronized Pair<Long, SocketAddress> get(long hash) {
-            if (circle.isEmpty()) {
-                return null;
-            }
-
-            if (!circle.containsKey(hash)) {
-                SortedMap<Long, SocketAddress> tailMap = circle.tailMap(hash);
-                hash = tailMap.isEmpty() ? circle.firstKey() : tailMap.firstKey();
-            }
-            return Pair.of(hash, circle.get(hash));
-        }
-
-        synchronized void dumpHashRing() {
-            for (Map.Entry<Long, SocketAddress> entry : circle.entrySet()) {
-                logger.info(entry.getKey() + " : " + entry.getValue());
-            }
-        }
-
-    }
-
-    class BlackoutHost implements TimerTask {
-        final int shardId;
-        final SocketAddress address;
-
-        BlackoutHost(int shardId, SocketAddress address) {
-            this.shardId = shardId;
-            this.address = address;
-            numBlackoutHosts.incrementAndGet();
-        }
-
-        @Override
-        public void run(Timeout timeout) throws Exception {
-            numBlackoutHosts.decrementAndGet();
-            if (!timeout.isExpired()) {
-                return;
-            }
-            Set<SocketAddress> removedList = new HashSet<SocketAddress>();
-            boolean joined;
-            // add the shard back
-            synchronized (shardId2Address) {
-                SocketAddress curHost = shardId2Address.get(shardId);
-                if (null != curHost) {
-                    // there is already new shard joint, so drop the host.
-                    logger.info("Blackout Shard {} ({}) was already replaced by {} permanently.",
-                            new Object[] { shardId, address, curHost });
-                    joined = false;
-                } else {
-                    join(shardId, address, removedList);
-                    joined = true;
-                }
-            }
-            if (joined) {
-                for (RoutingListener listener : listeners) {
-                    listener.onServerJoin(address);
-                }
-            } else {
-                for (RoutingListener listener : listeners) {
-                    listener.onServerLeft(address);
-                }
-            }
-        }
-    }
-
-    protected final HashedWheelTimer hashedWheelTimer;
-    protected final HashFunction hashFunction = Hashing.md5();
-    protected final ConsistentHash circle;
-    protected final Map<Integer, SocketAddress> shardId2Address =
-            new HashMap<Integer, SocketAddress>();
-    protected final Map<SocketAddress, Integer> address2ShardId =
-            new HashMap<SocketAddress, Integer>();
-
-    // blackout period
-    protected final int blackoutSeconds;
-
-    // stats
-    protected final StatsReceiver statsReceiver;
-    protected final AtomicInteger numBlackoutHosts;
-    protected final Gauge numBlackoutHostsGauge;
-    protected final Gauge numHostsGauge;
-
-    private static final int UNKNOWN_SHARD_ID = -1;
-
-    ConsistentHashRoutingService(ServerSetWatcher serverSetWatcher,
-                                 int numReplicas,
-                                 int blackoutSeconds,
-                                 StatsReceiver statsReceiver) {
-        super(serverSetWatcher);
-        this.circle = new ConsistentHash(hashFunction, numReplicas, statsReceiver.scope("ring"));
-        this.hashedWheelTimer = new HashedWheelTimer(new ThreadFactoryBuilder()
-                .setNameFormat("ConsistentHashRoutingService-Timer-%d").build());
-        this.blackoutSeconds = blackoutSeconds;
-        // stats
-        this.statsReceiver = statsReceiver;
-        this.numBlackoutHosts = new AtomicInteger(0);
-        this.numBlackoutHostsGauge = this.statsReceiver.addGauge(gaugeName("num_blackout_hosts"),
-                new Function0<Object>() {
-                    @Override
-                    public Object apply() {
-                        return (float) numBlackoutHosts.get();
-                    }
-                });
-        this.numHostsGauge = this.statsReceiver.addGauge(gaugeName("num_hosts"),
-                new Function0<Object>() {
-                    @Override
-                    public Object apply() {
-                        return (float) address2ShardId.size();
-                    }
-                });
-    }
-
-    private static Seq<String> gaugeName(String name) {
-        return scala.collection.JavaConversions.asScalaBuffer(Arrays.asList(name)).toList();
-    }
-
-    @Override
-    public void startService() {
-        super.startService();
-        this.hashedWheelTimer.start();
-    }
-
-    @Override
-    public void stopService() {
-        this.hashedWheelTimer.stop();
-        super.stopService();
-    }
-
-    @Override
-    public Set<SocketAddress> getHosts() {
-        synchronized (shardId2Address) {
-            return ImmutableSet.copyOf(address2ShardId.keySet());
-        }
-    }
-
-    @Override
-    public SocketAddress getHost(String key, RoutingContext rContext)
-            throws NoBrokersAvailableException {
-        SocketAddress host = circle.get(key, rContext);
-        if (null != host) {
-            return host;
-        }
-        throw new NoBrokersAvailableException("No host found for " + key + ", routing context : " + rContext);
-    }
-
-    @Override
-    public void removeHost(SocketAddress host, Throwable reason) {
-        removeHostInternal(host, Optional.of(reason));
-    }
-
-    private void removeHostInternal(SocketAddress host, Optional<Throwable> reason) {
-        synchronized (shardId2Address) {
-            Integer shardId = address2ShardId.remove(host);
-            if (null != shardId) {
-                SocketAddress curHost = shardId2Address.get(shardId);
-                if (null != curHost && curHost.equals(host)) {
-                    shardId2Address.remove(shardId);
-                }
-                circle.remove(shardId, host);
-                if (reason.isPresent()) {
-                    if (reason.get() instanceof ChannelException) {
-                        logger.info("Shard {} ({}) left due to ChannelException, black it out for {} seconds"
-                            + " (message = {})",
-                            new Object[] { shardId, host, blackoutSeconds, reason.get().toString() });
-                        BlackoutHost blackoutHost = new BlackoutHost(shardId, host);
-                        hashedWheelTimer.newTimeout(blackoutHost, blackoutSeconds, TimeUnit.SECONDS);
-                    } else {
-                        logger.info("Shard {} ({}) left due to exception {}",
-                                new Object[] { shardId, host, reason.get().toString() });
-                    }
-                } else {
-                    logger.info("Shard {} ({}) left after server set change",
-                                shardId, host);
-                }
-            } else if (reason.isPresent()) {
-                logger.info("Node {} left due to exception {}", host, reason.get().toString());
-            } else {
-                logger.info("Node {} left after server set change", host);
-            }
-        }
-    }
-
-    /**
-     * The caller should synchronize on <i>shardId2Address</i>.
-     * @param shardId
-     *          Shard id of new host joined.
-     * @param newHost
-     *          New host joined.
-     * @param removedList
-     *          Old hosts to remove
-     */
-    private void join(int shardId, SocketAddress newHost, Set<SocketAddress> removedList) {
-        SocketAddress oldHost = shardId2Address.put(shardId, newHost);
-        if (null != oldHost) {
-            // remove the old host only when a new shard is kicked in to replace it.
-            address2ShardId.remove(oldHost);
-            circle.remove(shardId, oldHost);
-            removedList.add(oldHost);
-            logger.info("Shard {} ({}) left permanently.", shardId, oldHost);
-        }
-        address2ShardId.put(newHost, shardId);
-        circle.add(shardId, newHost);
-        logger.info("Shard {} ({}) joined to replace ({}).",
-                    new Object[] { shardId, newHost, oldHost });
-    }
-
-    @Override
-    protected synchronized void performServerSetChange(ImmutableSet<DLSocketAddress> serviceInstances) {
-        Set<SocketAddress> joinedList = new HashSet<SocketAddress>();
-        Set<SocketAddress> removedList = new HashSet<SocketAddress>();
-
-        Map<Integer, SocketAddress> newMap = new HashMap<Integer, SocketAddress>();
-        synchronized (shardId2Address) {
-            for (DLSocketAddress serviceInstance : serviceInstances) {
-                if (serviceInstance.getShard() >= 0) {
-                    newMap.put(serviceInstance.getShard(), serviceInstance.getSocketAddress());
-                } else {
-                    Integer shard = address2ShardId.get(serviceInstance.getSocketAddress());
-                    if (null == shard) {
-                        // Assign a random negative shardId
-                        int shardId;
-                        do {
-                            shardId = Math.min(-1 , (int) (Math.random() * Integer.MIN_VALUE));
-                        } while (null != shardId2Address.get(shardId));
-                        shard = shardId;
-                    }
-                    newMap.put(shard, serviceInstance.getSocketAddress());
-                }
-            }
-        }
-
-        Map<Integer, SocketAddress> left;
-        synchronized (shardId2Address) {
-            MapDifference<Integer, SocketAddress> difference =
-                    Maps.difference(shardId2Address, newMap);
-            left = difference.entriesOnlyOnLeft();
-            for (Map.Entry<Integer, SocketAddress> shardEntry : left.entrySet()) {
-                int shard = shardEntry.getKey();
-                if (shard >= 0) {
-                    SocketAddress host = shardId2Address.get(shard);
-                    if (null != host) {
-                        // we don't remove those hosts that just disappered on serverset proactively,
-                        // since it might be just because serverset become flaky
-                        // address2ShardId.remove(host);
-                        // circle.remove(shard, host);
-                        logger.info("Shard {} ({}) left temporarily.", shard, host);
-                    }
-                } else {
-                    // shard id is negative - they are resolved from finagle name, which instances don't have shard id
-                    // in this case, if they are removed from serverset, we removed them directly
-                    SocketAddress host = shardEntry.getValue();
-                    if (null != host) {
-                        removeHostInternal(host, Optional.<Throwable>absent());
-                        removedList.add(host);
-                    }
-                }
-            }
-            // we need to find if any shards are replacing old shards
-            for (Map.Entry<Integer, SocketAddress> shard : newMap.entrySet()) {
-                SocketAddress oldHost = shardId2Address.get(shard.getKey());
-                SocketAddress newHost = shard.getValue();
-                if (!newHost.equals(oldHost)) {
-                    join(shard.getKey(), newHost, removedList);
-                    joinedList.add(newHost);
-                }
-            }
-        }
-
-        for (SocketAddress addr : removedList) {
-            for (RoutingListener listener : listeners) {
-                listener.onServerLeft(addr);
-            }
-        }
-
-        for (SocketAddress addr : joinedList) {
-            for (RoutingListener listener : listeners) {
-                listener.onServerJoin(addr);
-            }
-        }
-    }
-
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/NameServerSet.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/NameServerSet.java
deleted file mode 100644
index eeba4ac..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/NameServerSet.java
+++ /dev/null
@@ -1,263 +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 com.twitter.distributedlog.client.routing;
-
-import com.google.common.collect.ImmutableSet;
-import com.twitter.common.base.Command;
-import com.twitter.common.base.Commands;
-import com.twitter.common.zookeeper.Group;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.finagle.Addr;
-import com.twitter.finagle.Address;
-import com.twitter.finagle.Name;
-import com.twitter.finagle.Resolver$;
-import com.twitter.thrift.Endpoint;
-import com.twitter.thrift.ServiceInstance;
-import com.twitter.thrift.Status;
-import java.net.InetSocketAddress;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-/**
- * Finagle Name based {@link ServerSet} implementation.
- */
-class NameServerSet implements ServerSet {
-
-    private static final Logger logger = LoggerFactory.getLogger(NameServerSet.class);
-
-    private volatile Set<HostChangeMonitor<ServiceInstance>> watchers =
-        new HashSet<HostChangeMonitor<ServiceInstance>>();
-    private volatile ImmutableSet<ServiceInstance> hostSet = ImmutableSet.of();
-    private AtomicBoolean resolutionPending = new AtomicBoolean(true);
-
-    public NameServerSet(String nameStr) {
-        Name name;
-        try {
-            name = Resolver$.MODULE$.eval(nameStr);
-        } catch (Exception exc) {
-            logger.error("Exception in Resolver.eval for name {}", nameStr, exc);
-            // Since this is called from various places that dont handle specific exceptions,
-            // we have no option than to throw a runtime exception to halt the control flow
-            // This should only happen in case of incorrect configuration. Having a log message
-            // would help identify the problem during tests
-            throw new RuntimeException(exc);
-        }
-        initialize(name);
-    }
-
-    public NameServerSet(Name name) {
-        initialize(name);
-    }
-
-    private void initialize(Name name) {
-        if (name instanceof TestName) {
-            ((TestName) name).changes(new AbstractFunction1<Addr, BoxedUnit>() {
-                @Override
-                public BoxedUnit apply(Addr varAddr) {
-                    return NameServerSet.this.respondToChanges(varAddr);
-                }
-            });
-        } else if (name instanceof Name.Bound) {
-            ((Name.Bound) name).addr().changes().respond(new AbstractFunction1<Addr, BoxedUnit>() {
-                @Override
-                public BoxedUnit apply(Addr varAddr) {
-                    return NameServerSet.this.respondToChanges(varAddr);
-                }
-            });
-        } else {
-            logger.error("NameServerSet only supports Name.Bound. While the resolved name {} was {}",
-                name, name.getClass());
-            throw new UnsupportedOperationException("NameServerSet only supports Name.Bound");
-        }
-    }
-
-    private ServiceInstance endpointAddressToServiceInstance(Address endpointAddress) {
-        if (endpointAddress instanceof Address.Inet) {
-            InetSocketAddress inetSocketAddress = ((Address.Inet) endpointAddress).addr();
-            Endpoint endpoint = new Endpoint(inetSocketAddress.getHostString(), inetSocketAddress.getPort());
-            HashMap<String, Endpoint> map = new HashMap<String, Endpoint>();
-            map.put("thrift", endpoint);
-            return new ServiceInstance(
-                endpoint,
-                map,
-                Status.ALIVE);
-        } else {
-            logger.error("We expect InetSocketAddress while the resolved address {} was {}",
-                        endpointAddress, endpointAddress.getClass());
-            throw new UnsupportedOperationException("invalid endpoint address: " + endpointAddress);
-        }
-    }
-
-
-    private BoxedUnit respondToChanges(Addr addr) {
-        ImmutableSet<ServiceInstance> oldHostSet = ImmutableSet.copyOf(hostSet);
-
-        ImmutableSet<ServiceInstance> newHostSet = oldHostSet;
-
-        if (addr instanceof Addr.Bound) {
-            scala.collection.immutable.Set<Address> endpointAddresses = ((Addr.Bound) addr).addrs();
-            scala.collection.Iterator<Address> endpointAddressesIterator = endpointAddresses.toIterator();
-            HashSet<ServiceInstance> serviceInstances = new HashSet<ServiceInstance>();
-            while (endpointAddressesIterator.hasNext()) {
-                serviceInstances.add(endpointAddressToServiceInstance(endpointAddressesIterator.next()));
-            }
-            newHostSet = ImmutableSet.copyOf(serviceInstances);
-
-        } else if (addr instanceof Addr.Failed) {
-            logger.error("Name resolution failed", ((Addr.Failed) addr).cause());
-            newHostSet = ImmutableSet.of();
-        } else if (addr.toString().equals("Pending")) {
-            logger.info("Name resolution pending");
-            newHostSet = oldHostSet;
-        } else if (addr.toString().equals("Neg")) {
-            newHostSet = ImmutableSet.of();
-        } else {
-            logger.error("Invalid Addr type: {}", addr.getClass().getName());
-            throw new UnsupportedOperationException("Invalid Addr type:" + addr.getClass().getName());
-        }
-
-        // Reference comparison is valid as the sets are immutable
-        if (oldHostSet != newHostSet) {
-            logger.info("NameServerSet updated: {} -> {}", hostSetToString(oldHostSet), hostSetToString(newHostSet));
-            resolutionPending.set(false);
-            hostSet = newHostSet;
-            synchronized (watchers) {
-                for (HostChangeMonitor<ServiceInstance> watcher: watchers) {
-                    watcher.onChange(newHostSet);
-                }
-            }
-
-        }
-
-        return BoxedUnit.UNIT;
-    }
-
-
-    private String hostSetToString(ImmutableSet<ServiceInstance> hostSet) {
-        StringBuilder result = new StringBuilder();
-        result.append("(");
-        for (ServiceInstance serviceInstance : hostSet) {
-            Endpoint endpoint = serviceInstance.getServiceEndpoint();
-            result.append(String.format(" %s:%d", endpoint.getHost(), endpoint.getPort()));
-        }
-        result.append(" )");
-
-        return result.toString();
-    }
-
-
-    /**
-     * Attempts to join a server set for this logical service group.
-     *
-     * @param endpoint the primary service endpoint
-     * @param additionalEndpoints and additional endpoints keyed by their logical name
-     * @param status the current service status
-     * @return an EndpointStatus object that allows the endpoint to adjust its status
-     * @throws Group.JoinException if there was a problem joining the server set
-     * @throws InterruptedException if interrupted while waiting to join the server set
-     * @deprecated The status field is deprecated. Please use {@link #join(java.net.InetSocketAddress, java.util.Map)}
-     */
-    @Override
-    public EndpointStatus join(InetSocketAddress endpoint,
-                               Map<String, InetSocketAddress> additionalEndpoints,
-                               Status status)
-            throws Group.JoinException, InterruptedException {
-        throw new UnsupportedOperationException("NameServerSet does not support join");
-    }
-
-    /**
-     * Attempts to join a server set for this logical service group.
-     *
-     * @param endpoint the primary service endpoint
-     * @param additionalEndpoints and additional endpoints keyed by their logical name
-     * @return an EndpointStatus object that allows the endpoint to adjust its status
-     * @throws Group.JoinException if there was a problem joining the server set
-     * @throws InterruptedException if interrupted while waiting to join the server set
-     */
-    @Override
-    public EndpointStatus join(InetSocketAddress endpoint, Map<String, InetSocketAddress> additionalEndpoints)
-            throws Group.JoinException, InterruptedException {
-        throw new UnsupportedOperationException("NameServerSet does not support join");
-    }
-
-    /**
-     * Attempts to join a server set for this logical service group.
-     *
-     * @param endpoint the primary service endpoint
-     * @param additionalEndpoints and additional endpoints keyed by their logical name
-     * @param shardId Unique shard identifier for this member of the service.
-     * @return an EndpointStatus object that allows the endpoint to adjust its status
-     * @throws Group.JoinException if there was a problem joining the server set
-     * @throws InterruptedException if interrupted while waiting to join the server set
-     */
-    @Override
-    public EndpointStatus join(InetSocketAddress endpoint,
-                               Map<String, InetSocketAddress> additionalEndpoints,
-                               int shardId)
-            throws Group.JoinException, InterruptedException {
-        throw new UnsupportedOperationException("NameServerSet does not support join");
-    }
-
-    /**
-     * Registers a monitor to receive change notices for this server set as long as this jvm process
-     * is alive.  Blocks until the initial server set can be gathered and delivered to the monitor.
-     * The monitor will be notified if the membership set or parameters of existing members have
-     * changed.
-     *
-     * @param monitor the server set monitor to call back when the host set changes
-     * @throws com.twitter.common.net.pool.DynamicHostSet.MonitorException if there is a problem monitoring the host set
-     * @deprecated Deprecated in favor of {@link #watch(com.twitter.common.net.pool.DynamicHostSet.HostChangeMonitor)}
-     */
-    @Deprecated
-    @Override
-    public void monitor(HostChangeMonitor<ServiceInstance> monitor) throws MonitorException {
-        throw new UnsupportedOperationException("NameServerSet does not support monitor");
-    }
-
-    /**
-     * Registers a monitor to receive change notices for this server set as long as this jvm process
-     * is alive.  Blocks until the initial server set can be gathered and delivered to the monitor.
-     * The monitor will be notified if the membership set or parameters of existing members have
-     * changed.
-     *
-     * @param monitor the server set monitor to call back when the host set changes
-     * @return A command which, when executed, will stop monitoring the host set.
-     * @throws com.twitter.common.net.pool.DynamicHostSet.MonitorException if there is a problem monitoring the host set
-     */
-    @Override
-    public Command watch(HostChangeMonitor<ServiceInstance> monitor) throws MonitorException {
-        // First add the monitor to the watchers so that it does not miss any changes and invoke
-        // the onChange method
-        synchronized (watchers) {
-            watchers.add(monitor);
-        }
-
-        if (resolutionPending.compareAndSet(false, false)) {
-            monitor.onChange(hostSet);
-        }
-
-        return Commands.NOOP; // Return value is not used
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RegionsRoutingService.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RegionsRoutingService.java
deleted file mode 100644
index 4714270..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RegionsRoutingService.java
+++ /dev/null
@@ -1,192 +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 com.twitter.distributedlog.client.routing;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.Set;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Chain multiple routing services.
- */
-public class RegionsRoutingService implements RoutingService {
-
-    private static final Logger logger = LoggerFactory.getLogger(RegionsRoutingService.class);
-
-    /**
-     * Create a multiple regions routing services based on a list of region routing {@code services}.
-     *
-     * <p>It is deprecated. Please use {@link Builder} to build multiple regions routing service.
-     *
-     * @param regionResolver region resolver
-     * @param services a list of region routing services.
-     * @return multiple regions routing service
-     * @see Builder
-     */
-    @Deprecated
-    public static RegionsRoutingService of(RegionResolver regionResolver,
-                                         RoutingService...services) {
-        return new RegionsRoutingService(regionResolver, services);
-    }
-
-    /**
-     * Create a builder to build a multiple-regions routing service.
-     *
-     * @return builder to build a multiple-regions routing service.
-     */
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    /**
-     * Builder to build a multiple-regions routing service.
-     */
-    public static class Builder implements RoutingService.Builder {
-
-        private RegionResolver resolver;
-        private RoutingService.Builder[] routingServiceBuilders;
-        private StatsReceiver statsReceiver = NullStatsReceiver.get();
-
-        private Builder() {}
-
-        public Builder routingServiceBuilders(RoutingService.Builder...builders) {
-            this.routingServiceBuilders = builders;
-            return this;
-        }
-
-        public Builder resolver(RegionResolver regionResolver) {
-            this.resolver = regionResolver;
-            return this;
-        }
-
-        @Override
-        public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
-            this.statsReceiver = statsReceiver;
-            return this;
-        }
-
-        @Override
-        public RegionsRoutingService build() {
-            checkNotNull(routingServiceBuilders, "No routing service builder provided.");
-            checkNotNull(resolver, "No region resolver provided.");
-            checkNotNull(statsReceiver, "No stats receiver provided");
-            RoutingService[] services = new RoutingService[routingServiceBuilders.length];
-            for (int i = 0; i < services.length; i++) {
-                String statsScope;
-                if (0 == i) {
-                    statsScope = "local";
-                } else {
-                    statsScope = "remote_" + i;
-                }
-                services[i] = routingServiceBuilders[i]
-                        .statsReceiver(statsReceiver.scope(statsScope))
-                        .build();
-            }
-            return new RegionsRoutingService(resolver, services);
-        }
-    }
-
-    protected final RegionResolver regionResolver;
-    protected final RoutingService[] routingServices;
-
-    private RegionsRoutingService(RegionResolver resolver,
-                                  RoutingService[] routingServices) {
-        this.regionResolver = resolver;
-        this.routingServices = routingServices;
-    }
-
-    @Override
-    public Set<SocketAddress> getHosts() {
-        Set<SocketAddress> hosts = Sets.newHashSet();
-        for (RoutingService rs : routingServices) {
-            hosts.addAll(rs.getHosts());
-        }
-        return hosts;
-    }
-
-    @Override
-    public void startService() {
-        for (RoutingService service : routingServices) {
-            service.startService();
-        }
-        logger.info("Regions Routing Service Started");
-    }
-
-    @Override
-    public void stopService() {
-        for (RoutingService service : routingServices) {
-            service.stopService();
-        }
-        logger.info("Regions Routing Service Stopped");
-    }
-
-    @Override
-    public RoutingService registerListener(RoutingListener listener) {
-        for (RoutingService service : routingServices) {
-            service.registerListener(listener);
-        }
-        return this;
-    }
-
-    @Override
-    public RoutingService unregisterListener(RoutingListener listener) {
-        for (RoutingService service : routingServices) {
-            service.registerListener(listener);
-        }
-        return this;
-    }
-
-    @Override
-    public SocketAddress getHost(String key, RoutingContext routingContext)
-            throws NoBrokersAvailableException {
-        for (RoutingService service : routingServices) {
-            try {
-                SocketAddress addr = service.getHost(key, routingContext);
-                if (routingContext.hasUnavailableRegions()) {
-                    // current region is unavailable
-                    String region = regionResolver.resolveRegion(addr);
-                    if (routingContext.isUnavailableRegion(region)) {
-                        continue;
-                    }
-                }
-                if (!routingContext.isTriedHost(addr)) {
-                    return addr;
-                }
-            } catch (NoBrokersAvailableException nbae) {
-                // if there isn't broker available in current service, try next service.
-                logger.debug("No brokers available in region {} : ", service, nbae);
-            }
-        }
-        throw new NoBrokersAvailableException("No host found for " + key + ", routing context : " + routingContext);
-    }
-
-    @Override
-    public void removeHost(SocketAddress address, Throwable reason) {
-        for (RoutingService service : routingServices) {
-            service.removeHost(address, reason);
-        }
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingService.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingService.java
deleted file mode 100644
index 56446c1..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingService.java
+++ /dev/null
@@ -1,206 +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 com.twitter.distributedlog.client.routing;
-
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * Routing Service provides mechanism how to route requests.
- */
-public interface RoutingService {
-
-    /**
-     * Builder to build routing service.
-     */
-    interface Builder {
-
-        /**
-         * Build routing service with stats receiver.
-         *
-         * @param statsReceiver
-         *          stats receiver
-         * @return built routing service
-         */
-        Builder statsReceiver(StatsReceiver statsReceiver);
-
-        /**
-         * Build the routing service.
-         *
-         * @return built routing service
-         */
-        RoutingService build();
-
-    }
-
-    /**
-     * Listener for server changes on routing service.
-     */
-    interface RoutingListener {
-        /**
-         * Trigger when server left.
-         *
-         * @param address left server.
-         */
-        void onServerLeft(SocketAddress address);
-
-        /**
-         * Trigger when server joint.
-         *
-         * @param address joint server.
-         */
-        void onServerJoin(SocketAddress address);
-    }
-
-    /**
-     * Routing Context of a request.
-     */
-    class RoutingContext {
-
-        public static RoutingContext of(RegionResolver resolver) {
-            return new RoutingContext(resolver);
-        }
-
-        final RegionResolver regionResolver;
-        final Map<SocketAddress, StatusCode> triedHosts;
-        final Set<String> unavailableRegions;
-
-        private RoutingContext(RegionResolver regionResolver) {
-            this.regionResolver = regionResolver;
-            this.triedHosts = new HashMap<SocketAddress, StatusCode>();
-            this.unavailableRegions = new HashSet<String>();
-        }
-
-        @Override
-        public synchronized String toString() {
-            return "(tried hosts=" + triedHosts + ")";
-        }
-
-        /**
-         * Add tried host to routing context.
-         *
-         * @param socketAddress
-         *          socket address of tried host.
-         * @param code
-         *          status code returned from tried host.
-         * @return routing context.
-         */
-        public synchronized RoutingContext addTriedHost(SocketAddress socketAddress, StatusCode code) {
-            this.triedHosts.put(socketAddress, code);
-            if (StatusCode.REGION_UNAVAILABLE == code) {
-                unavailableRegions.add(regionResolver.resolveRegion(socketAddress));
-            }
-            return this;
-        }
-
-        /**
-         * Is the host <i>address</i> already tried.
-         *
-         * @param address
-         *          socket address to check
-         * @return true if the address is already tried, otherwise false.
-         */
-        public synchronized boolean isTriedHost(SocketAddress address) {
-            return this.triedHosts.containsKey(address);
-        }
-
-        /**
-         * Whether encountered unavailable regions.
-         *
-         * @return true if encountered unavailable regions, otherwise false.
-         */
-        public synchronized boolean hasUnavailableRegions() {
-            return !unavailableRegions.isEmpty();
-        }
-
-        /**
-         * Whether the <i>region</i> is unavailable.
-         *
-         * @param region
-         *          region
-         * @return true if the region is unavailable, otherwise false.
-         */
-        public synchronized boolean isUnavailableRegion(String region) {
-            return unavailableRegions.contains(region);
-        }
-
-    }
-
-    /**
-     * Start routing service.
-     */
-    void startService();
-
-    /**
-     * Stop routing service.
-     */
-    void stopService();
-
-    /**
-     * Register routing listener.
-     *
-     * @param listener routing listener.
-     * @return routing service.
-     */
-    RoutingService registerListener(RoutingListener listener);
-
-    /**
-     * Unregister routing listener.
-     *
-     * @param listener routing listener.
-     * @return routing service.
-     */
-    RoutingService unregisterListener(RoutingListener listener);
-
-    /**
-     * Get all the hosts that available in routing service.
-     *
-     * @return all the hosts
-     */
-    Set<SocketAddress> getHosts();
-
-    /**
-     * Get the host to route the request by <i>key</i>.
-     *
-     * @param key
-     *          key to route the request.
-     * @param rContext
-     *          routing context.
-     * @return host to route the request
-     * @throws NoBrokersAvailableException
-     */
-    SocketAddress getHost(String key, RoutingContext rContext)
-            throws NoBrokersAvailableException;
-
-    /**
-     * Remove the host <i>address</i> for a specific <i>reason</i>.
-     *
-     * @param address
-     *          host address to remove
-     * @param reason
-     *          reason to remove the host
-     */
-    void removeHost(SocketAddress address, Throwable reason);
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingServiceProvider.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingServiceProvider.java
deleted file mode 100644
index 22cd222..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingServiceProvider.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 com.twitter.distributedlog.client.routing;
-
-import com.twitter.finagle.stats.StatsReceiver;
-
-class RoutingServiceProvider implements RoutingService.Builder {
-
-    final RoutingService routingService;
-
-    RoutingServiceProvider(RoutingService routingService) {
-        this.routingService = routingService;
-    }
-
-    @Override
-    public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
-        return this;
-    }
-
-    @Override
-    public RoutingService build() {
-        return routingService;
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingUtils.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingUtils.java
deleted file mode 100644
index 2302e18..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingUtils.java
+++ /dev/null
@@ -1,88 +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 com.twitter.distributedlog.client.routing;
-
-import com.twitter.common.zookeeper.ServerSet;
-import java.net.SocketAddress;
-
-/**
- * Utils for routing services.
- */
-public class RoutingUtils {
-
-    private static final int NUM_CONSISTENT_HASH_REPLICAS = 997;
-
-    /**
-     * Building routing service from <code>finagleNameStr</code>.
-     *
-     * @param finagleNameStr
-     *          finagle name str of a service
-     * @return routing service builder
-     */
-    public static RoutingService.Builder buildRoutingService(String finagleNameStr) {
-        if (!finagleNameStr.startsWith("serverset!")
-                && !finagleNameStr.startsWith("inet!")
-                && !finagleNameStr.startsWith("zk!")) {
-            // We only support serverset based names at the moment
-            throw new UnsupportedOperationException("Finagle Name format not supported for name: " + finagleNameStr);
-        }
-        return buildRoutingService(new NameServerSet(finagleNameStr), true);
-    }
-
-    /**
-     * Building routing service from <code>serverSet</code>.
-     *
-     * @param serverSet
-     *          server set of a service
-     * @return routing service builder
-     */
-    public static RoutingService.Builder buildRoutingService(ServerSet serverSet) {
-        return buildRoutingService(serverSet, false);
-    }
-
-    /**
-     * Building routing service from <code>address</code>.
-     *
-     * @param address
-     *          host to route the requests
-     * @return routing service builder
-     */
-    public static RoutingService.Builder buildRoutingService(SocketAddress address) {
-        return SingleHostRoutingService.newBuilder().address(address);
-    }
-
-    /**
-     * Build routing service builder of a routing service <code>routingService</code>.
-     *
-     * @param routingService
-     *          routing service to provide
-     * @return routing service builder
-     */
-    public static RoutingService.Builder buildRoutingService(RoutingService routingService) {
-        return new RoutingServiceProvider(routingService);
-    }
-
-    private static RoutingService.Builder buildRoutingService(ServerSet serverSet,
-                                                              boolean resolveFromName) {
-        return ConsistentHashRoutingService.newBuilder()
-                .serverSet(serverSet)
-                .resolveFromName(resolveFromName)
-                .numReplicas(NUM_CONSISTENT_HASH_REPLICAS);
-    }
-
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetRoutingService.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetRoutingService.java
deleted file mode 100644
index 19ccfc4..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetRoutingService.java
+++ /dev/null
@@ -1,274 +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 com.twitter.distributedlog.client.routing;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hashing;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Routing Service based on a given {@link com.twitter.common.zookeeper.ServerSet}.
- */
-class ServerSetRoutingService extends Thread implements RoutingService {
-
-    private static final Logger logger = LoggerFactory.getLogger(ServerSetRoutingService.class);
-
-    static ServerSetRoutingServiceBuilder newServerSetRoutingServiceBuilder() {
-        return new ServerSetRoutingServiceBuilder();
-    }
-
-    /**
-     * Builder to build {@link com.twitter.common.zookeeper.ServerSet} based routing service.
-     */
-    static class ServerSetRoutingServiceBuilder implements RoutingService.Builder {
-
-        private ServerSetWatcher serverSetWatcher;
-
-        private ServerSetRoutingServiceBuilder() {}
-
-        public ServerSetRoutingServiceBuilder serverSetWatcher(ServerSetWatcher serverSetWatcher) {
-            this.serverSetWatcher = serverSetWatcher;
-            return this;
-        }
-
-        @Override
-        public Builder statsReceiver(StatsReceiver statsReceiver) {
-            return this;
-        }
-
-        @Override
-        public RoutingService build() {
-            checkNotNull(serverSetWatcher, "No serverset watcher provided.");
-            return new ServerSetRoutingService(this.serverSetWatcher);
-        }
-    }
-
-    private static class HostComparator implements Comparator<SocketAddress> {
-
-        private static final HostComparator INSTANCE = new HostComparator();
-
-        @Override
-        public int compare(SocketAddress o1, SocketAddress o2) {
-            return o1.toString().compareTo(o2.toString());
-        }
-    }
-
-    private final ServerSetWatcher serverSetWatcher;
-
-    private final Set<SocketAddress> hostSet = new HashSet<SocketAddress>();
-    private List<SocketAddress> hostList = new ArrayList<SocketAddress>();
-    private final HashFunction hasher = Hashing.md5();
-
-    // Server Set Changes
-    private final AtomicReference<ImmutableSet<DLSocketAddress>> serverSetChange =
-            new AtomicReference<ImmutableSet<DLSocketAddress>>(null);
-    private final CountDownLatch changeLatch = new CountDownLatch(1);
-
-    // Listeners
-    protected final CopyOnWriteArraySet<RoutingListener> listeners =
-            new CopyOnWriteArraySet<RoutingListener>();
-
-    ServerSetRoutingService(ServerSetWatcher serverSetWatcher) {
-        super("ServerSetRoutingService");
-        this.serverSetWatcher = serverSetWatcher;
-    }
-
-    @Override
-    public Set<SocketAddress> getHosts() {
-        synchronized (hostSet) {
-            return ImmutableSet.copyOf(hostSet);
-        }
-    }
-
-    @Override
-    public void startService() {
-        start();
-        try {
-            if (!changeLatch.await(1, TimeUnit.MINUTES)) {
-                logger.warn("No serverset change received in 1 minute.");
-            }
-        } catch (InterruptedException e) {
-            logger.warn("Interrupted waiting first serverset change : ", e);
-        }
-        logger.info("{} Routing Service Started.", getClass().getSimpleName());
-    }
-
-    @Override
-    public void stopService() {
-        Thread.currentThread().interrupt();
-        try {
-            join();
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            logger.warn("Interrupted on waiting serverset routing service to finish : ", e);
-        }
-        logger.info("{} Routing Service Stopped.", getClass().getSimpleName());
-    }
-
-    @Override
-    public RoutingService registerListener(RoutingListener listener) {
-        listeners.add(listener);
-        return this;
-    }
-
-    @Override
-    public RoutingService unregisterListener(RoutingListener listener) {
-        listeners.remove(listener);
-        return this;
-    }
-
-    @Override
-    public SocketAddress getHost(String key, RoutingContext rContext)
-            throws NoBrokersAvailableException {
-        SocketAddress address = null;
-        synchronized (hostSet) {
-            if (0 != hostList.size()) {
-                int hashCode = hasher.hashUnencodedChars(key).asInt();
-                int hostId = signSafeMod(hashCode, hostList.size());
-                address = hostList.get(hostId);
-                if (rContext.isTriedHost(address)) {
-                    ArrayList<SocketAddress> newList = new ArrayList<SocketAddress>(hostList);
-                    newList.remove(hostId);
-                    // pickup a new host by rehashing it.
-                    hostId = signSafeMod(hashCode, newList.size());
-                    address = newList.get(hostId);
-                    int i = hostId;
-                    while (rContext.isTriedHost(address)) {
-                        i = (i + 1) % newList.size();
-                        if (i == hostId) {
-                            address = null;
-                            break;
-                        }
-                        address = newList.get(i);
-                    }
-                }
-            }
-        }
-        if (null == address) {
-            throw new NoBrokersAvailableException("No host is available.");
-        }
-        return address;
-    }
-
-    @Override
-    public void removeHost(SocketAddress host, Throwable reason) {
-        synchronized (hostSet) {
-            if (hostSet.remove(host)) {
-                logger.info("Node {} left due to : ", host, reason);
-            }
-            hostList = new ArrayList<SocketAddress>(hostSet);
-            Collections.sort(hostList, HostComparator.INSTANCE);
-            logger.info("Host list becomes : {}.", hostList);
-        }
-    }
-
-    @Override
-    public void run() {
-        try {
-            serverSetWatcher.watch(new ServerSetWatcher.ServerSetMonitor() {
-                @Override
-                public void onChange(ImmutableSet<DLSocketAddress> serviceInstances) {
-                    ImmutableSet<DLSocketAddress> lastValue = serverSetChange.getAndSet(serviceInstances);
-                    if (null == lastValue) {
-                        ImmutableSet<DLSocketAddress> mostRecentValue;
-                        do {
-                            mostRecentValue = serverSetChange.get();
-                            performServerSetChange(mostRecentValue);
-                            changeLatch.countDown();
-                        } while (!serverSetChange.compareAndSet(mostRecentValue, null));
-                    }
-                }
-            });
-        } catch (Exception e) {
-            logger.error("Fail to monitor server set : ", e);
-            Runtime.getRuntime().exit(-1);
-        }
-    }
-
-    protected synchronized void performServerSetChange(ImmutableSet<DLSocketAddress> serverSet) {
-        Set<SocketAddress> newSet = new HashSet<SocketAddress>();
-        for (DLSocketAddress serviceInstance : serverSet) {
-            newSet.add(serviceInstance.getSocketAddress());
-        }
-
-        Set<SocketAddress> removed;
-        Set<SocketAddress> added;
-        synchronized (hostSet) {
-            removed = Sets.difference(hostSet, newSet).immutableCopy();
-            added = Sets.difference(newSet, hostSet).immutableCopy();
-            for (SocketAddress node: removed) {
-                if (hostSet.remove(node)) {
-                    logger.info("Node {} left.", node);
-                }
-            }
-            for (SocketAddress node: added) {
-                if (hostSet.add(node)) {
-                    logger.info("Node {} joined.", node);
-                }
-            }
-        }
-
-        for (SocketAddress addr : removed) {
-            for (RoutingListener listener : listeners) {
-                listener.onServerLeft(addr);
-            }
-        }
-
-        for (SocketAddress addr : added) {
-            for (RoutingListener listener : listeners) {
-                listener.onServerJoin(addr);
-            }
-        }
-
-        synchronized (hostSet) {
-            hostList = new ArrayList<SocketAddress>(hostSet);
-            Collections.sort(hostList, HostComparator.INSTANCE);
-            logger.info("Host list becomes : {}.", hostList);
-        }
-
-    }
-
-    static int signSafeMod(long dividend, int divisor) {
-        int mod = (int) (dividend % divisor);
-
-        if (mod < 0) {
-            mod += divisor;
-        }
-
-        return mod;
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetWatcher.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetWatcher.java
deleted file mode 100644
index 1eccb63..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetWatcher.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 com.twitter.distributedlog.client.routing;
-
-import com.google.common.collect.ImmutableSet;
-import com.twitter.distributedlog.service.DLSocketAddress;
-
-/**
- * Watch on server set changes.
- */
-public interface ServerSetWatcher {
-
-    /**
-     * Exception thrown when failed to monitor serverset.
-     */
-    class MonitorException extends Exception {
-
-        private static final long serialVersionUID = 392751505154339548L;
-
-        public MonitorException(String msg) {
-            super(msg);
-        }
-
-        public MonitorException(String msg, Throwable cause) {
-            super(msg, cause);
-        }
-    }
-
-    /**
-     * An interface to an object that is interested in receiving notification whenever the host set changes.
-     */
-    interface ServerSetMonitor {
-
-        /**
-         * Called when either the available set of services changes.
-         *
-         * <p>It happens either when a service dies or a new INSTANCE comes on-line or
-         * when an existing service advertises a status or health change.
-         *
-         * @param hostSet the current set of available ServiceInstances
-         */
-        void onChange(ImmutableSet<DLSocketAddress> hostSet);
-    }
-
-    /**
-     * Registers a monitor to receive change notices for this server set as long as this jvm process is alive.
-     *
-     * <p>Blocks until the initial server set can be gathered and delivered to the monitor.
-     * The monitor will be notified if the membership set or parameters of existing members have
-     * changed.
-     *
-     * @param monitor the server set monitor to call back when the host set changes
-     * @throws MonitorException if there is a problem monitoring the host set
-     */
-    void watch(final ServerSetMonitor monitor) throws MonitorException;
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/SingleHostRoutingService.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/SingleHostRoutingService.java
deleted file mode 100644
index e526868..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/SingleHostRoutingService.java
+++ /dev/null
@@ -1,128 +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 com.twitter.distributedlog.client.routing;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.collect.Sets;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArraySet;
-
-/**
- * Single Host Routing Service.
- */
-public class SingleHostRoutingService implements RoutingService {
-
-    public static SingleHostRoutingService of(SocketAddress address) {
-        return new SingleHostRoutingService(address);
-    }
-
-    /**
-     * Builder to build single host based routing service.
-     *
-     * @return builder to build single host based routing service.
-     */
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    /**
-     * Builder to build single host based routing service.
-     */
-    public static class Builder implements RoutingService.Builder {
-
-        private SocketAddress address;
-
-        private Builder() {}
-
-        public Builder address(SocketAddress address) {
-            this.address = address;
-            return this;
-        }
-
-        @Override
-        public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
-            return this;
-        }
-
-        @Override
-        public RoutingService build() {
-            checkNotNull(address, "Host is null");
-            return new SingleHostRoutingService(address);
-        }
-    }
-
-    private SocketAddress address;
-    private final CopyOnWriteArraySet<RoutingListener> listeners =
-            new CopyOnWriteArraySet<RoutingListener>();
-
-    SingleHostRoutingService(SocketAddress address) {
-        this.address = address;
-    }
-
-    public void setAddress(SocketAddress address) {
-        this.address = address;
-    }
-
-    @Override
-    public Set<SocketAddress> getHosts() {
-        return Sets.newHashSet(address);
-    }
-
-    @Override
-    public void startService() {
-        // no-op
-        for (RoutingListener listener : listeners) {
-            listener.onServerJoin(address);
-        }
-    }
-
-    @Override
-    public void stopService() {
-        // no-op
-    }
-
-    @Override
-    public RoutingService registerListener(RoutingListener listener) {
-        listeners.add(listener);
-        return this;
-    }
-
-    @Override
-    public RoutingService unregisterListener(RoutingListener listener) {
-        listeners.remove(listener);
-        return null;
-    }
-
-    @Override
-    public SocketAddress getHost(String key, RoutingContext rContext)
-            throws NoBrokersAvailableException {
-        if (rContext.isTriedHost(address)) {
-            throw new NoBrokersAvailableException("No hosts is available : routing context = " + rContext);
-        }
-        return address;
-    }
-
-    @Override
-    public void removeHost(SocketAddress address, Throwable reason) {
-        // no-op
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TestName.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TestName.java
deleted file mode 100644
index 8101075..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TestName.java
+++ /dev/null
@@ -1,49 +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 com.twitter.distributedlog.client.routing;
-
-import com.twitter.finagle.Addr;
-import com.twitter.finagle.Address;
-import com.twitter.finagle.Addrs;
-import com.twitter.finagle.Name;
-import java.util.List;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-/**
- * A {@link Name} implementation for testing purpose.
- */
-public class TestName implements Name {
-
-    private static final Logger LOG = LoggerFactory.getLogger(TestName.class);
-
-    private AbstractFunction1<Addr, BoxedUnit> callback = null;
-
-    public void changes(AbstractFunction1<Addr, BoxedUnit> callback) {
-        this.callback = callback;
-    }
-
-    public void changeAddrs(List<Address> addresses) {
-        if (null != callback) {
-            LOG.info("Sending a callback {}", addresses);
-            callback.apply(Addrs.newBoundAddr(addresses));
-        }
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TwitterServerSetWatcher.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TwitterServerSetWatcher.java
deleted file mode 100644
index cffb9b9..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TwitterServerSetWatcher.java
+++ /dev/null
@@ -1,83 +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 com.twitter.distributedlog.client.routing;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Sets;
-import com.twitter.common.net.pool.DynamicHostSet;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.thrift.Endpoint;
-import com.twitter.thrift.ServiceInstance;
-import java.net.InetSocketAddress;
-import java.util.Set;
-
-/**
- * Twitter {@link ServerSet} based watcher.
- */
-public class TwitterServerSetWatcher implements ServerSetWatcher {
-
-    private final ServerSet serverSet;
-    private final boolean resolvedFromName;
-
-    /**
-     * Construct a {@link ServerSet} based watcher.
-     *
-     * @param serverSet server set.
-     * @param resolvedFromName whether to resolve hosts from {@link com.twitter.finagle.Name}.
-     */
-    public TwitterServerSetWatcher(ServerSet serverSet,
-                                   boolean resolvedFromName) {
-        this.serverSet = serverSet;
-        this.resolvedFromName = resolvedFromName;
-    }
-
-    /**
-     * Registers a monitor to receive change notices for this server set as long as this jvm process is alive.
-     *
-     * <p>Blocks until the initial server set can be gathered and delivered to the monitor.
-     * The monitor will be notified if the membership set or parameters of existing members have
-     * changed.
-     *
-     * @param monitor the server set monitor to call back when the host set changes
-     * @throws MonitorException if there is a problem monitoring the host set
-     */
-    public void watch(final ServerSetMonitor monitor)
-            throws MonitorException {
-        try {
-            serverSet.watch(new DynamicHostSet.HostChangeMonitor<ServiceInstance>() {
-                @Override
-                public void onChange(ImmutableSet<ServiceInstance> serviceInstances) {
-                    Set<DLSocketAddress> dlServers = Sets.newHashSet();
-                    for (ServiceInstance serviceInstance : serviceInstances) {
-                        Endpoint endpoint = serviceInstance.getAdditionalEndpoints().get("thrift");
-                        InetSocketAddress inetAddr =
-                                new InetSocketAddress(endpoint.getHost(), endpoint.getPort());
-                        int shardId = resolvedFromName ? -1 : serviceInstance.getShard();
-                        DLSocketAddress address = new DLSocketAddress(shardId, inetAddr);
-                        dlServers.add(address);
-                    }
-                    monitor.onChange(ImmutableSet.copyOf(dlServers));
-                }
-            });
-        } catch (DynamicHostSet.MonitorException me) {
-            throw new MonitorException("Failed to monitor server set : ", me);
-        }
-    }
-
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/package-info.java
deleted file mode 100644
index a282b42..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Routing Mechanisms to route the traffic to the owner of streams.
- */
-package com.twitter.distributedlog.client.routing;
\ No newline at end of file
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/DLZkServerSet.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/DLZkServerSet.java
deleted file mode 100644
index 4ca3aa6..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/DLZkServerSet.java
+++ /dev/null
@@ -1,91 +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 com.twitter.distributedlog.client.serverset;
-
-import com.google.common.collect.ImmutableList;
-import com.google.common.net.HostAndPort;
-import com.twitter.common.quantity.Amount;
-import com.twitter.common.quantity.Time;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.common.zookeeper.ServerSets;
-import com.twitter.common.zookeeper.ZooKeeperClient;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import org.apache.commons.lang.StringUtils;
-import org.apache.zookeeper.ZooDefs;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A wrapper over zookeeper client and its server set.
- */
-public class DLZkServerSet {
-
-    private static final Logger logger = LoggerFactory.getLogger(DLZkServerSet.class);
-
-    static final String ZNODE_WRITE_PROXY = ".write_proxy";
-
-    private static String getZKServersFromDLUri(URI uri) {
-        return uri.getAuthority().replace(";", ",");
-    }
-
-    private static Iterable<InetSocketAddress> getZkAddresses(URI uri) {
-        String zkServers = getZKServersFromDLUri(uri);
-        String[] zkServerList = StringUtils.split(zkServers, ',');
-        ImmutableList.Builder<InetSocketAddress> builder = ImmutableList.builder();
-        for (String zkServer : zkServerList) {
-            HostAndPort hostAndPort = HostAndPort.fromString(zkServer).withDefaultPort(2181);
-            builder.add(InetSocketAddress.createUnresolved(
-                    hostAndPort.getHostText(),
-                    hostAndPort.getPort()));
-        }
-        return builder.build();
-    }
-
-    public static DLZkServerSet of(URI uri,
-                                   int zkSessionTimeoutMs) {
-        // Create zookeeper and server set
-        String zkPath = uri.getPath() + "/" + ZNODE_WRITE_PROXY;
-        Iterable<InetSocketAddress> zkAddresses = getZkAddresses(uri);
-        ZooKeeperClient zkClient =
-                new ZooKeeperClient(Amount.of(zkSessionTimeoutMs, Time.MILLISECONDS), zkAddresses);
-        ServerSet serverSet = ServerSets.create(zkClient, ZooDefs.Ids.OPEN_ACL_UNSAFE, zkPath);
-        return new DLZkServerSet(zkClient, serverSet);
-    }
-
-    private final ZooKeeperClient zkClient;
-    private final ServerSet zkServerSet;
-
-    public DLZkServerSet(ZooKeeperClient zkClient,
-                         ServerSet zkServerSet) {
-        this.zkClient = zkClient;
-        this.zkServerSet = zkServerSet;
-    }
-
-    public ZooKeeperClient getZkClient() {
-        return zkClient;
-    }
-
-    public ServerSet getServerSet() {
-        return zkServerSet;
-    }
-
-    public void close() {
-        zkClient.close();
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/package-info.java
deleted file mode 100644
index 49166ec..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Utils related to server set.
- */
-package com.twitter.distributedlog.client.serverset;
\ No newline at end of file
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
deleted file mode 100644
index 489fc00..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
+++ /dev/null
@@ -1,119 +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 com.twitter.distributedlog.client.speculative;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Default implementation of {@link SpeculativeRequestExecutionPolicy}.
- */
-public class DefaultSpeculativeRequestExecutionPolicy implements SpeculativeRequestExecutionPolicy {
-
-    private static final Logger LOG = LoggerFactory.getLogger(DefaultSpeculativeRequestExecutionPolicy.class);
-    final int firstSpeculativeRequestTimeout;
-    final int maxSpeculativeRequestTimeout;
-    final float backoffMultiplier;
-    int nextSpeculativeRequestTimeout;
-
-    public DefaultSpeculativeRequestExecutionPolicy(int firstSpeculativeRequestTimeout,
-                                                    int maxSpeculativeRequestTimeout,
-                                                    float backoffMultiplier) {
-        this.firstSpeculativeRequestTimeout = firstSpeculativeRequestTimeout;
-        this.maxSpeculativeRequestTimeout = maxSpeculativeRequestTimeout;
-        this.backoffMultiplier = backoffMultiplier;
-        this.nextSpeculativeRequestTimeout = firstSpeculativeRequestTimeout;
-
-        if (backoffMultiplier <= 0) {
-            throw new IllegalArgumentException("Invalid value provided for backoffMultiplier");
-        }
-
-        // Prevent potential over flow
-        if (Math.round((double) maxSpeculativeRequestTimeout * (double) backoffMultiplier) > Integer.MAX_VALUE) {
-            throw new IllegalArgumentException("Invalid values for maxSpeculativeRequestTimeout and backoffMultiplier");
-        }
-    }
-
-    @VisibleForTesting
-    int getNextSpeculativeRequestTimeout() {
-        return nextSpeculativeRequestTimeout;
-    }
-
-    /**
-     * Initialize the speculative request execution policy.
-     *
-     * @param scheduler The scheduler service to issue the speculative request
-     * @param requestExecutor The executor is used to issue the actual speculative requests
-     */
-    @Override
-    public void initiateSpeculativeRequest(final ScheduledExecutorService scheduler,
-                                           final SpeculativeRequestExecutor requestExecutor) {
-        issueSpeculativeRequest(scheduler, requestExecutor);
-    }
-
-    private void issueSpeculativeRequest(final ScheduledExecutorService scheduler,
-                                         final SpeculativeRequestExecutor requestExecutor) {
-        Future<Boolean> issueNextRequest = requestExecutor.issueSpeculativeRequest();
-        issueNextRequest.addEventListener(new FutureEventListener<Boolean>() {
-            // we want this handler to run immediately after we push the big red button!
-            @Override
-            public void onSuccess(Boolean issueNextRequest) {
-                if (issueNextRequest) {
-                    scheduleSpeculativeRequest(scheduler, requestExecutor, nextSpeculativeRequestTimeout);
-                    nextSpeculativeRequestTimeout = Math.min(maxSpeculativeRequestTimeout,
-                            (int) (nextSpeculativeRequestTimeout * backoffMultiplier));
-                } else {
-                    if (LOG.isTraceEnabled()) {
-                        LOG.trace("Stopped issuing speculative requests for {}, "
-                            + "speculativeReadTimeout = {}", requestExecutor, nextSpeculativeRequestTimeout);
-                    }
-                }
-            }
-
-            @Override
-            public void onFailure(Throwable thrown) {
-                LOG.warn("Failed to issue speculative request for {}, speculativeReadTimeout = {} : ",
-                        new Object[] { requestExecutor, nextSpeculativeRequestTimeout, thrown });
-            }
-        });
-    }
-
-    private void scheduleSpeculativeRequest(final ScheduledExecutorService scheduler,
-                                            final SpeculativeRequestExecutor requestExecutor,
-                                            final int speculativeRequestTimeout) {
-        try {
-            scheduler.schedule(new Runnable() {
-                @Override
-                public void run() {
-                    issueSpeculativeRequest(scheduler, requestExecutor);
-                }
-            }, speculativeRequestTimeout, TimeUnit.MILLISECONDS);
-        } catch (RejectedExecutionException re) {
-            if (!scheduler.isShutdown()) {
-                LOG.warn("Failed to schedule speculative request for {}, speculativeReadTimeout = {} : ",
-                        new Object[]{requestExecutor, speculativeRequestTimeout, re});
-            }
-        }
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
deleted file mode 100644
index 14615e9..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.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 com.twitter.distributedlog.client.speculative;
-
-import java.util.concurrent.ScheduledExecutorService;
-
-/**
- * Speculative request execution policy.
- */
-public interface SpeculativeRequestExecutionPolicy {
-    /**
-     * Initialize the speculative request execution policy and initiate requests.
-     *
-     * @param scheduler The scheduler service to issue the speculative request
-     * @param requestExecutor The executor is used to issue the actual speculative requests
-     */
-    void initiateSpeculativeRequest(ScheduledExecutorService scheduler,
-                                    SpeculativeRequestExecutor requestExecutor);
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutor.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutor.java
deleted file mode 100644
index de1b0dd..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutor.java
+++ /dev/null
@@ -1,33 +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 com.twitter.distributedlog.client.speculative;
-
-import com.twitter.util.Future;
-
-/**
- * Executor to execute speculative requests.
- */
-public interface SpeculativeRequestExecutor {
-
-    /**
-     * Issues a speculative request and indicates if more speculative requests should be issued.
-     *
-     * @return whether more speculative requests should be issued.
-     */
-    Future<Boolean> issueSpeculativeRequest();
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/package-info.java
deleted file mode 100644
index b299266..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Speculative Mechanism.
- */
-package com.twitter.distributedlog.client.speculative;
\ No newline at end of file
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStats.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStats.java
deleted file mode 100644
index f361892..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStats.java
+++ /dev/null
@@ -1,108 +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 com.twitter.distributedlog.client.stats;
-
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Client Stats.
- */
-public class ClientStats {
-
-    // Region Resolver
-    private final RegionResolver regionResolver;
-
-    // Stats
-    private final StatsReceiver statsReceiver;
-    private final ClientStatsLogger clientStatsLogger;
-    private final boolean enableRegionStats;
-    private final ConcurrentMap<String, ClientStatsLogger> regionClientStatsLoggers;
-    private final ConcurrentMap<String, OpStats> opStatsMap;
-
-    public ClientStats(StatsReceiver statsReceiver,
-                       boolean enableRegionStats,
-                       RegionResolver regionResolver) {
-        this.statsReceiver = statsReceiver;
-        this.clientStatsLogger = new ClientStatsLogger(statsReceiver);
-        this.enableRegionStats = enableRegionStats;
-        this.regionClientStatsLoggers = new ConcurrentHashMap<String, ClientStatsLogger>();
-        this.regionResolver = regionResolver;
-        this.opStatsMap = new ConcurrentHashMap<String, OpStats>();
-    }
-
-    public OpStats getOpStats(String op) {
-        OpStats opStats = opStatsMap.get(op);
-        if (null != opStats) {
-            return opStats;
-        }
-        OpStats newStats = new OpStats(statsReceiver.scope(op),
-                enableRegionStats, regionResolver);
-        OpStats oldStats = opStatsMap.putIfAbsent(op, newStats);
-        if (null == oldStats) {
-            return newStats;
-        } else {
-            return oldStats;
-        }
-    }
-
-    private ClientStatsLogger getRegionClientStatsLogger(SocketAddress address) {
-        String region = regionResolver.resolveRegion(address);
-        return getRegionClientStatsLogger(region);
-    }
-
-    private ClientStatsLogger getRegionClientStatsLogger(String region) {
-        ClientStatsLogger statsLogger = regionClientStatsLoggers.get(region);
-        if (null == statsLogger) {
-            ClientStatsLogger newStatsLogger = new ClientStatsLogger(statsReceiver.scope(region));
-            ClientStatsLogger oldStatsLogger = regionClientStatsLoggers.putIfAbsent(region, newStatsLogger);
-            if (null == oldStatsLogger) {
-                statsLogger = newStatsLogger;
-            } else {
-                statsLogger = oldStatsLogger;
-            }
-        }
-        return statsLogger;
-    }
-
-    public StatsReceiver getFinagleStatsReceiver(SocketAddress addr) {
-        if (enableRegionStats && null != addr) {
-            return getRegionClientStatsLogger(addr).getStatsReceiver();
-        } else {
-            return clientStatsLogger.getStatsReceiver();
-        }
-    }
-
-    public void completeProxyRequest(SocketAddress addr, StatusCode code, long startTimeNanos) {
-        clientStatsLogger.completeProxyRequest(code, startTimeNanos);
-        if (enableRegionStats && null != addr) {
-            getRegionClientStatsLogger(addr).completeProxyRequest(code, startTimeNanos);
-        }
-    }
-
-    public void failProxyRequest(SocketAddress addr, Throwable cause, long startTimeNanos) {
-        clientStatsLogger.failProxyRequest(cause, startTimeNanos);
-        if (enableRegionStats && null != addr) {
-            getRegionClientStatsLogger(addr).failProxyRequest(cause, startTimeNanos);
-        }
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStatsLogger.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStatsLogger.java
deleted file mode 100644
index 0df64cc..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStatsLogger.java
+++ /dev/null
@@ -1,91 +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 com.twitter.distributedlog.client.stats;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.finagle.stats.Counter;
-import com.twitter.finagle.stats.Stat;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Stats Logger to collect client stats.
- */
-public class ClientStatsLogger {
-
-    // Stats
-    private final StatsReceiver statsReceiver;
-    private final StatsReceiver responseStatsReceiver;
-    private final ConcurrentMap<StatusCode, Counter> responseStats =
-            new ConcurrentHashMap<StatusCode, Counter>();
-    private final StatsReceiver exceptionStatsReceiver;
-    private final ConcurrentMap<Class<?>, Counter> exceptionStats =
-            new ConcurrentHashMap<Class<?>, Counter>();
-
-    private final Stat proxySuccessLatencyStat;
-    private final Stat proxyFailureLatencyStat;
-
-    public ClientStatsLogger(StatsReceiver statsReceiver) {
-        this.statsReceiver = statsReceiver;
-        responseStatsReceiver = statsReceiver.scope("responses");
-        exceptionStatsReceiver = statsReceiver.scope("exceptions");
-        StatsReceiver proxyLatencyStatReceiver = statsReceiver.scope("proxy_request_latency");
-        proxySuccessLatencyStat = proxyLatencyStatReceiver.stat0("success");
-        proxyFailureLatencyStat = proxyLatencyStatReceiver.stat0("failure");
-    }
-
-    public StatsReceiver getStatsReceiver() {
-        return statsReceiver;
-    }
-
-    private Counter getResponseCounter(StatusCode code) {
-        Counter counter = responseStats.get(code);
-        if (null == counter) {
-            Counter newCounter = responseStatsReceiver.counter0(code.name());
-            Counter oldCounter = responseStats.putIfAbsent(code, newCounter);
-            counter = null != oldCounter ? oldCounter : newCounter;
-        }
-        return counter;
-    }
-
-    private Counter getExceptionCounter(Class<?> cls) {
-        Counter counter = exceptionStats.get(cls);
-        if (null == counter) {
-            Counter newCounter = exceptionStatsReceiver.counter0(cls.getName());
-            Counter oldCounter = exceptionStats.putIfAbsent(cls, newCounter);
-            counter = null != oldCounter ? oldCounter : newCounter;
-        }
-        return counter;
-    }
-
-    public void completeProxyRequest(StatusCode code, long startTimeNanos) {
-        getResponseCounter(code).incr();
-        proxySuccessLatencyStat.add(elapsedMicroSec(startTimeNanos));
-    }
-
-    public void failProxyRequest(Throwable cause, long startTimeNanos) {
-        getExceptionCounter(cause.getClass()).incr();
-        proxyFailureLatencyStat.add(elapsedMicroSec(startTimeNanos));
-    }
-
-    static long elapsedMicroSec(long startNanoTime) {
-        return TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - startNanoTime);
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStats.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStats.java
deleted file mode 100644
index 26708f3..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStats.java
+++ /dev/null
@@ -1,82 +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 com.twitter.distributedlog.client.stats;
-
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Op Stats.
- */
-public class OpStats {
-
-    // Region Resolver
-    private final RegionResolver regionResolver;
-
-    // Stats
-    private final StatsReceiver statsReceiver;
-    private final OpStatsLogger opStatsLogger;
-    private final boolean enableRegionStats;
-    private final ConcurrentMap<String, OpStatsLogger> regionOpStatsLoggers;
-
-    public OpStats(StatsReceiver statsReceiver,
-                   boolean enableRegionStats,
-                   RegionResolver regionResolver) {
-        this.statsReceiver = statsReceiver;
-        this.opStatsLogger = new OpStatsLogger(statsReceiver);
-        this.enableRegionStats = enableRegionStats;
-        this.regionOpStatsLoggers = new ConcurrentHashMap<String, OpStatsLogger>();
-        this.regionResolver = regionResolver;
-    }
-
-    private OpStatsLogger getRegionOpStatsLogger(SocketAddress address) {
-        String region = regionResolver.resolveRegion(address);
-        return getRegionOpStatsLogger(region);
-    }
-
-    private OpStatsLogger getRegionOpStatsLogger(String region) {
-        OpStatsLogger statsLogger = regionOpStatsLoggers.get(region);
-        if (null == statsLogger) {
-            OpStatsLogger newStatsLogger = new OpStatsLogger(statsReceiver.scope(region));
-            OpStatsLogger oldStatsLogger = regionOpStatsLoggers.putIfAbsent(region, newStatsLogger);
-            if (null == oldStatsLogger) {
-                statsLogger = newStatsLogger;
-            } else {
-                statsLogger = oldStatsLogger;
-            }
-        }
-        return statsLogger;
-    }
-
-    public void completeRequest(SocketAddress addr, long micros, int numTries) {
-        opStatsLogger.completeRequest(micros, numTries);
-        if (enableRegionStats && null != addr) {
-            getRegionOpStatsLogger(addr).completeRequest(micros, numTries);
-        }
-    }
-
-    public void failRequest(SocketAddress addr, long micros, int numTries) {
-        opStatsLogger.failRequest(micros, numTries);
-        if (enableRegionStats && null != addr) {
-            getRegionOpStatsLogger(addr).failRequest(micros, numTries);
-        }
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStatsLogger.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStatsLogger.java
deleted file mode 100644
index 27adda7..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStatsLogger.java
+++ /dev/null
@@ -1,50 +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 com.twitter.distributedlog.client.stats;
-
-import com.twitter.finagle.stats.Stat;
-import com.twitter.finagle.stats.StatsReceiver;
-
-/**
- * Stats Logger per operation type.
- */
-public class OpStatsLogger {
-
-    private final Stat successLatencyStat;
-    private final Stat failureLatencyStat;
-    private final Stat redirectStat;
-
-    public OpStatsLogger(StatsReceiver statsReceiver) {
-        StatsReceiver latencyStatReceiver = statsReceiver.scope("latency");
-        successLatencyStat = latencyStatReceiver.stat0("success");
-        failureLatencyStat = latencyStatReceiver.stat0("failure");
-        StatsReceiver redirectStatReceiver = statsReceiver.scope("redirects");
-        redirectStat = redirectStatReceiver.stat0("times");
-    }
-
-    public void completeRequest(long micros, int numTries) {
-        successLatencyStat.add(micros);
-        redirectStat.add(numTries);
-    }
-
-    public void failRequest(long micros, int numTries) {
-        failureLatencyStat.add(micros);
-        redirectStat.add(numTries);
-    }
-
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OwnershipStatsLogger.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OwnershipStatsLogger.java
deleted file mode 100644
index 7d5a9c9..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OwnershipStatsLogger.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 com.twitter.distributedlog.client.stats;
-
-import com.twitter.finagle.stats.Counter;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Stats Logger for ownerships.
- */
-public class OwnershipStatsLogger {
-
-    /**
-     * Ownership related stats.
-     */
-    public static class OwnershipStat {
-        private final Counter hits;
-        private final Counter misses;
-        private final Counter removes;
-        private final Counter redirects;
-        private final Counter adds;
-
-        OwnershipStat(StatsReceiver ownershipStats) {
-            hits = ownershipStats.counter0("hits");
-            misses = ownershipStats.counter0("misses");
-            adds = ownershipStats.counter0("adds");
-            removes = ownershipStats.counter0("removes");
-            redirects = ownershipStats.counter0("redirects");
-        }
-
-        public void onHit() {
-            hits.incr();
-        }
-
-        public void onMiss() {
-            misses.incr();
-        }
-
-        public void onAdd() {
-            adds.incr();
-        }
-
-        public void onRemove() {
-            removes.incr();
-        }
-
-        public void onRedirect() {
-            redirects.incr();
-        }
-
-    }
-
-    private final OwnershipStat ownershipStat;
-    private final StatsReceiver ownershipStatsReceiver;
-    private final ConcurrentMap<String, OwnershipStat> ownershipStats =
-            new ConcurrentHashMap<String, OwnershipStat>();
-
-    public OwnershipStatsLogger(StatsReceiver statsReceiver,
-                                StatsReceiver streamStatsReceiver) {
-        this.ownershipStat = new OwnershipStat(statsReceiver.scope("ownership"));
-        this.ownershipStatsReceiver = streamStatsReceiver.scope("perstream_ownership");
-    }
-
-    private OwnershipStat getOwnershipStat(String stream) {
-        OwnershipStat stat = ownershipStats.get(stream);
-        if (null == stat) {
-            OwnershipStat newStat = new OwnershipStat(ownershipStatsReceiver.scope(stream));
-            OwnershipStat oldStat = ownershipStats.putIfAbsent(stream, newStat);
-            stat = null != oldStat ? oldStat : newStat;
-        }
-        return stat;
-    }
-
-    public void onMiss(String stream) {
-        ownershipStat.onMiss();
-        getOwnershipStat(stream).onMiss();
-    }
-
-    public void onHit(String stream) {
-        ownershipStat.onHit();
-        getOwnershipStat(stream).onHit();
-    }
-
-    public void onRedirect(String stream) {
-        ownershipStat.onRedirect();
-        getOwnershipStat(stream).onRedirect();
-    }
-
-    public void onRemove(String stream) {
-        ownershipStat.onRemove();
-        getOwnershipStat(stream).onRemove();
-    }
-
-    public void onAdd(String stream) {
-        ownershipStat.onAdd();
-        getOwnershipStat(stream).onAdd();
-    }
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/package-info.java
deleted file mode 100644
index 91d4f39..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Client side stats utils.
- */
-package com.twitter.distributedlog.client.stats;
\ No newline at end of file
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DLSocketAddress.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DLSocketAddress.java
deleted file mode 100644
index 30891c0..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DLSocketAddress.java
+++ /dev/null
@@ -1,161 +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 com.twitter.distributedlog.service;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.UnknownHostException;
-
-/**
- * Socket Address identifier for a DL proxy.
- */
-public class DLSocketAddress {
-
-    private static final int VERSION = 1;
-
-    private static final String COLON = ":";
-    private static final String SEP = ";";
-
-    private final int shard;
-    private final InetSocketAddress socketAddress;
-
-    public DLSocketAddress(int shard, InetSocketAddress socketAddress) {
-        this.shard = shard;
-        this.socketAddress = socketAddress;
-    }
-
-    /**
-     * Shard id for dl write proxy.
-     *
-     * @return shard id for dl write proxy.
-     */
-    public int getShard() {
-        return shard;
-    }
-
-    /**
-     * Socket address for dl write proxy.
-     *
-     * @return socket address for dl write proxy
-     */
-    public InetSocketAddress getSocketAddress() {
-        return socketAddress;
-    }
-
-    /**
-     * Serialize the write proxy identifier to string.
-     *
-     * @return serialized write proxy identifier.
-     */
-    public String serialize() {
-        return toLockId(socketAddress, shard);
-    }
-
-    @Override
-    public int hashCode() {
-        return socketAddress.hashCode();
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (!(obj instanceof DLSocketAddress)) {
-            return false;
-        }
-        DLSocketAddress other = (DLSocketAddress) obj;
-        return shard == other.shard && socketAddress.equals(other.socketAddress);
-    }
-
-    @Override
-    public String toString() {
-        return toLockId(socketAddress, shard);
-    }
-
-    /**
-     * Deserialize proxy address from a string representation.
-     *
-     * @param lockId
-     *          string representation of the proxy address.
-     * @return proxy address.
-     * @throws IOException
-     */
-    public static DLSocketAddress deserialize(String lockId) throws IOException {
-        String parts[] = lockId.split(SEP);
-        if (3 != parts.length) {
-            throw new IOException("Invalid dl socket address " + lockId);
-        }
-        int version;
-        try {
-            version = Integer.parseInt(parts[0]);
-        } catch (NumberFormatException nfe) {
-            throw new IOException("Invalid version found in " + lockId, nfe);
-        }
-        if (VERSION != version) {
-            throw new IOException("Invalid version " + version + " found in " + lockId + ", expected " + VERSION);
-        }
-        int shardId;
-        try {
-            shardId = Integer.parseInt(parts[1]);
-        } catch (NumberFormatException nfe) {
-            throw new IOException("Invalid shard id found in " + lockId, nfe);
-        }
-        InetSocketAddress address = parseSocketAddress(parts[2]);
-        return new DLSocketAddress(shardId, address);
-    }
-
-    /**
-     * Parse the inet socket address from the string representation.
-     *
-     * @param addr
-     *          string representation
-     * @return inet socket address
-     */
-    public static InetSocketAddress parseSocketAddress(String addr) {
-        String[] parts =  addr.split(COLON);
-        checkArgument(parts.length == 2);
-        String hostname = parts[0];
-        int port = Integer.parseInt(parts[1]);
-        return new InetSocketAddress(hostname, port);
-    }
-
-    public static InetSocketAddress getSocketAddress(int port) throws UnknownHostException {
-        return new InetSocketAddress(InetAddress.getLocalHost().getHostAddress(), port);
-    }
-
-    /**
-     * Convert inet socket address to the string representation.
-     *
-     * @param address
-     *          inet socket address.
-     * @return string representation of inet socket address.
-     */
-    public static String toString(InetSocketAddress address) {
-        StringBuilder sb = new StringBuilder();
-        sb.append(address.getHostName()).append(COLON).append(address.getPort());
-        return sb.toString();
-    }
-
-    public static String toLockId(InetSocketAddress address, int shard) {
-        StringBuilder sb = new StringBuilder();
-        sb.append(VERSION).append(SEP).append(shard).append(SEP).append(toString(address));
-        return sb.toString();
-    }
-
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClient.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClient.java
deleted file mode 100644
index a2c5150..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClient.java
+++ /dev/null
@@ -1,108 +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 com.twitter.distributedlog.service;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecordSetBuffer;
-import com.twitter.util.Future;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-/**
- * Interface for distributedlog client.
- */
-public interface DistributedLogClient {
-    /**
-     * Write <i>data</i> to a given <i>stream</i>.
-     *
-     * @param stream
-     *          Stream Name.
-     * @param data
-     *          Data to write.
-     * @return a future representing a sequence id returned for this write.
-     */
-    Future<DLSN> write(String stream, ByteBuffer data);
-
-    /**
-     * Write record set to a given <i>stream</i>.
-     *
-     * <p>The record set is built from {@link com.twitter.distributedlog.LogRecordSet.Writer}
-     *
-     * @param stream stream to write to
-     * @param recordSet record set
-     */
-    Future<DLSN> writeRecordSet(String stream, LogRecordSetBuffer recordSet);
-
-    /**
-     * Write <i>data</i> in bulk to a given <i>stream</i>.
-     *
-     * <p>Return a list of Future dlsns, one for each submitted buffer. In the event of a partial
-     * failure--ex. some specific buffer write fails, all subsequent writes
-     * will also fail.
-     *
-     * @param stream
-     *          Stream Name.
-     * @param data
-     *          Data to write.
-     * @return a list of futures, one for each submitted buffer.
-     */
-    List<Future<DLSN>> writeBulk(String stream, List<ByteBuffer> data);
-
-    /**
-     * Truncate the stream to a given <i>dlsn</i>.
-     *
-     * @param stream
-     *          Stream Name.
-     * @param dlsn
-     *          DLSN to truncate until.
-     * @return a future representing the truncation.
-     */
-    Future<Boolean> truncate(String stream, DLSN dlsn);
-
-    /**
-     * Release the ownership of a stream <i>stream</i>.
-     *
-     * @param stream
-     *          Stream Name to release.
-     * @return a future representing the release operation.
-     */
-    Future<Void> release(String stream);
-
-    /**
-     * Delete a given stream <i>stream</i>.
-     *
-     * @param stream
-     *          Stream Name to delete.
-     * @return a future representing the delete operation.
-     */
-    Future<Void> delete(String stream);
-
-    /**
-     * Create a stream with name <i>stream</i>.
-     *
-     * @param stream
-     *          Stream Name to create.
-     * @return a future representing the create operation.
-     */
-    Future<Void> create(String stream);
-
-    /**
-     * Close the client.
-     */
-    void close();
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClientBuilder.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClientBuilder.java
deleted file mode 100644
index 3f65aff..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClientBuilder.java
+++ /dev/null
@@ -1,608 +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 com.twitter.distributedlog.service;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.client.ClientConfig;
-import com.twitter.distributedlog.client.DistributedLogClientImpl;
-import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
-import com.twitter.distributedlog.client.proxy.ClusterClient;
-import com.twitter.distributedlog.client.resolver.DefaultRegionResolver;
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.distributedlog.client.routing.RegionsRoutingService;
-import com.twitter.distributedlog.client.routing.RoutingService;
-import com.twitter.distributedlog.client.routing.RoutingUtils;
-import com.twitter.distributedlog.thrift.service.DistributedLogService;
-import com.twitter.finagle.Name;
-import com.twitter.finagle.Resolver$;
-import com.twitter.finagle.Service;
-import com.twitter.finagle.ThriftMux;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.finagle.thrift.ClientId;
-import com.twitter.finagle.thrift.ThriftClientFramedCodec;
-import com.twitter.finagle.thrift.ThriftClientRequest;
-import com.twitter.util.Duration;
-import java.net.SocketAddress;
-import java.net.URI;
-import java.util.Random;
-import org.apache.commons.lang.StringUtils;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.Option;
-
-/**
- * Builder to build {@link DistributedLogClient}.
- */
-public final class DistributedLogClientBuilder {
-
-    private static final Logger logger = LoggerFactory.getLogger(DistributedLogClientBuilder.class);
-
-    private static final Random random = new Random(System.currentTimeMillis());
-
-    private String name = null;
-    private ClientId clientId = null;
-    private RoutingService.Builder routingServiceBuilder = null;
-    private ClientBuilder clientBuilder = null;
-    private String serverRoutingServiceFinagleName = null;
-    private StatsReceiver statsReceiver = new NullStatsReceiver();
-    private StatsReceiver streamStatsReceiver = new NullStatsReceiver();
-    private ClientConfig clientConfig = new ClientConfig();
-    private boolean enableRegionStats = false;
-    private final RegionResolver regionResolver = new DefaultRegionResolver();
-
-    /**
-     * Create a client builder.
-     *
-     * @return client builder
-     */
-    public static DistributedLogClientBuilder newBuilder() {
-        return new DistributedLogClientBuilder();
-    }
-
-    /**
-     * Create a new client builder from an existing {@code builder}.
-     *
-     * @param builder the existing builder.
-     * @return a new client builder.
-     */
-    public static DistributedLogClientBuilder newBuilder(DistributedLogClientBuilder builder) {
-        DistributedLogClientBuilder newBuilder = new DistributedLogClientBuilder();
-        newBuilder.name = builder.name;
-        newBuilder.clientId = builder.clientId;
-        newBuilder.clientBuilder = builder.clientBuilder;
-        newBuilder.routingServiceBuilder = builder.routingServiceBuilder;
-        newBuilder.statsReceiver = builder.statsReceiver;
-        newBuilder.streamStatsReceiver = builder.streamStatsReceiver;
-        newBuilder.enableRegionStats = builder.enableRegionStats;
-        newBuilder.serverRoutingServiceFinagleName = builder.serverRoutingServiceFinagleName;
-        newBuilder.clientConfig = ClientConfig.newConfig(builder.clientConfig);
-        return newBuilder;
-    }
-
-    // private constructor
-    private DistributedLogClientBuilder() {}
-
-    /**
-     * Client Name.
-     *
-     * @param name
-     *          client name
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder name(String name) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.name = name;
-        return newBuilder;
-    }
-
-    /**
-     * Client ID.
-     *
-     * @param clientId
-     *          client id
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder clientId(ClientId clientId) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientId = clientId;
-        return newBuilder;
-    }
-
-    /**
-     * Serverset to access proxy services.
-     *
-     * @param serverSet
-     *          server set.
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder serverSet(ServerSet serverSet) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(serverSet);
-        newBuilder.enableRegionStats = false;
-        return newBuilder;
-    }
-
-    /**
-     * Server Sets to access proxy services.
-     *
-     * <p>The <i>local</i> server set will be tried first then <i>remotes</i>.
-     *
-     * @param local local server set.
-     * @param remotes remote server sets.
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder serverSets(ServerSet local, ServerSet...remotes) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        RoutingService.Builder[] builders = new RoutingService.Builder[remotes.length + 1];
-        builders[0] = RoutingUtils.buildRoutingService(local);
-        for (int i = 1; i < builders.length; i++) {
-            builders[i] = RoutingUtils.buildRoutingService(remotes[i - 1]);
-        }
-        newBuilder.routingServiceBuilder = RegionsRoutingService.newBuilder()
-                .resolver(regionResolver)
-                .routingServiceBuilders(builders);
-        newBuilder.enableRegionStats = remotes.length > 0;
-        return newBuilder;
-    }
-
-    /**
-     * Name to access proxy services.
-     *
-     * @param finagleNameStr
-     *          finagle name string.
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder finagleNameStr(String finagleNameStr) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(finagleNameStr);
-        newBuilder.enableRegionStats = false;
-        return newBuilder;
-    }
-
-    /**
-     * Finagle name strs to access proxy services.
-     *
-     * <p>The <i>local</i> finalge name str will be tried first, then <i>remotes</i>.
-     *
-     * @param local local server set.
-     * @param remotes remote server sets.
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder finagleNameStrs(String local, String...remotes) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        RoutingService.Builder[] builders = new RoutingService.Builder[remotes.length + 1];
-        builders[0] = RoutingUtils.buildRoutingService(local);
-        for (int i = 1; i < builders.length; i++) {
-            builders[i] = RoutingUtils.buildRoutingService(remotes[i - 1]);
-        }
-        newBuilder.routingServiceBuilder = RegionsRoutingService.newBuilder()
-                .routingServiceBuilders(builders)
-                .resolver(regionResolver);
-        newBuilder.enableRegionStats = remotes.length > 0;
-        return newBuilder;
-    }
-
-    /**
-     * URI to access proxy services.
-     *
-     * <p>Assuming the write proxies are announced under `.write_proxy` of the provided namespace uri.
-     * The builder will convert the dl uri (e.g. distributedlog://{zkserver}/path/to/namespace) to
-     * zookeeper serverset based finagle name str (`zk!{zkserver}!/path/to/namespace/.write_proxy`)
-     *
-     * @param uri namespace uri to access the serverset of write proxies
-     * @return distributedlog builder
-     */
-    public DistributedLogClientBuilder uri(URI uri) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        String zkServers = uri.getAuthority().replace(";", ",");
-        String[] zkServerList = StringUtils.split(zkServers, ',');
-        String finagleNameStr = String.format(
-                "zk!%s!%s/.write_proxy",
-                zkServerList[random.nextInt(zkServerList.length)], // zk server
-                uri.getPath());
-        newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(finagleNameStr);
-        newBuilder.enableRegionStats = false;
-        return newBuilder;
-    }
-
-    /**
-     * Address of write proxy to connect.
-     *
-     * @param address
-     *          write proxy address.
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder host(SocketAddress address) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(address);
-        newBuilder.enableRegionStats = false;
-        return newBuilder;
-    }
-
-    private DistributedLogClientBuilder routingServiceBuilder(RoutingService.Builder builder) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.routingServiceBuilder = builder;
-        newBuilder.enableRegionStats = false;
-        return newBuilder;
-    }
-
-    /**
-     * Routing Service to access proxy services.
-     *
-     * @param routingService
-     *          routing service
-     * @return client builder.
-     */
-    @VisibleForTesting
-    public DistributedLogClientBuilder routingService(RoutingService routingService) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(routingService);
-        newBuilder.enableRegionStats = false;
-        return newBuilder;
-    }
-
-    /**
-     * Stats receiver to expose client stats.
-     *
-     * @param statsReceiver
-     *          stats receiver.
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder statsReceiver(StatsReceiver statsReceiver) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.statsReceiver = statsReceiver;
-        return newBuilder;
-    }
-
-    /**
-     * Stream Stats Receiver to expose per stream stats.
-     *
-     * @param streamStatsReceiver
-     *          stream stats receiver
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder streamStatsReceiver(StatsReceiver streamStatsReceiver) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.streamStatsReceiver = streamStatsReceiver;
-        return newBuilder;
-    }
-
-    /**
-     * Set underlying finagle client builder.
-     *
-     * @param builder
-     *          finagle client builder.
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder clientBuilder(ClientBuilder builder) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientBuilder = builder;
-        return newBuilder;
-    }
-
-    /**
-     * Backoff time when redirecting to an already retried host.
-     *
-     * @param ms
-     *          backoff time.
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder redirectBackoffStartMs(int ms) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setRedirectBackoffStartMs(ms);
-        return newBuilder;
-    }
-
-    /**
-     * Max backoff time when redirecting to an already retried host.
-     *
-     * @param ms
-     *          backoff time.
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder redirectBackoffMaxMs(int ms) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setRedirectBackoffMaxMs(ms);
-        return newBuilder;
-    }
-
-    /**
-     * Max redirects that is allowed per request.
-     *
-     * <p>If <i>redirects</i> are exhausted, fail the request immediately.
-     *
-     * @param redirects
-     *          max redirects allowed before failing a request.
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder maxRedirects(int redirects) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setMaxRedirects(redirects);
-        return newBuilder;
-    }
-
-    /**
-     * Timeout per request in millis.
-     *
-     * @param timeoutMs
-     *          timeout per request in millis.
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder requestTimeoutMs(int timeoutMs) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setRequestTimeoutMs(timeoutMs);
-        return newBuilder;
-    }
-
-    /**
-     * Set thriftmux enabled.
-     *
-     * @param enabled
-     *          is thriftmux enabled
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder thriftmux(boolean enabled) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setThriftMux(enabled);
-        return newBuilder;
-    }
-
-    /**
-     * Set failfast stream exception handling enabled.
-     *
-     * @param enabled
-     *          is failfast exception handling enabled
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder streamFailfast(boolean enabled) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setStreamFailfast(enabled);
-        return newBuilder;
-    }
-
-    /**
-     * Set the regex to match stream names that the client cares about.
-     *
-     * @param nameRegex
-     *          stream name regex
-     * @return client builder
-     */
-    public DistributedLogClientBuilder streamNameRegex(String nameRegex) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setStreamNameRegex(nameRegex);
-        return newBuilder;
-    }
-
-    /**
-     * Whether to use the new handshake endpoint to exchange ownership cache.
-     *
-     * <p>Enable this when the servers are updated to support handshaking with client info.
-     *
-     * @param enabled
-     *          new handshake endpoint is enabled.
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder handshakeWithClientInfo(boolean enabled) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setHandshakeWithClientInfo(enabled);
-        return newBuilder;
-    }
-
-    /**
-     * Set the periodic handshake interval in milliseconds.
-     *
-     * <p>Every <code>intervalMs</code>, the DL client will handshake with existing proxies again.
-     * If the interval is less than ownership sync interval, the handshake won't sync ownerships. Otherwise, it will.
-     *
-     * @see #periodicOwnershipSyncIntervalMs(long)
-     * @param intervalMs
-     *          handshake interval
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder periodicHandshakeIntervalMs(long intervalMs) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setPeriodicHandshakeIntervalMs(intervalMs);
-        return newBuilder;
-    }
-
-    /**
-     * Set the periodic ownership sync interval in milliseconds.
-     *
-     * <p>If periodic handshake is enabled, the handshake will sync ownership if the elapsed time is larger than
-     * sync interval.
-     *
-     * @see #periodicHandshakeIntervalMs(long)
-     * @param intervalMs
-     *          interval that handshake should sync ownerships.
-     * @return client builder
-     */
-    public DistributedLogClientBuilder periodicOwnershipSyncIntervalMs(long intervalMs) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setPeriodicOwnershipSyncIntervalMs(intervalMs);
-        return newBuilder;
-    }
-
-    /**
-     * Enable/Disable periodic dumping ownership cache.
-     *
-     * @param enabled
-     *          flag to enable/disable periodic dumping ownership cache
-     * @return client builder.
-     */
-    public DistributedLogClientBuilder periodicDumpOwnershipCache(boolean enabled) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setPeriodicDumpOwnershipCacheEnabled(enabled);
-        return newBuilder;
-    }
-
-    /**
-     * Set periodic dumping ownership cache interval.
-     *
-     * @param intervalMs
-     *          interval on dumping ownership cache, in millis.
-     * @return client builder
-     */
-    public DistributedLogClientBuilder periodicDumpOwnershipCacheIntervalMs(long intervalMs) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setPeriodicDumpOwnershipCacheIntervalMs(intervalMs);
-        return newBuilder;
-    }
-
-    /**
-     * Enable handshake tracing.
-     *
-     * @param enabled
-     *          flag to enable/disable handshake tracing
-     * @return client builder
-     */
-    public DistributedLogClientBuilder handshakeTracing(boolean enabled) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setHandshakeTracingEnabled(enabled);
-        return newBuilder;
-    }
-
-    /**
-     * Enable checksum on requests to the proxy.
-     *
-     * @param enabled
-     *          flag to enable/disable checksum
-     * @return client builder
-     */
-    public DistributedLogClientBuilder checksum(boolean enabled) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig.setChecksumEnabled(enabled);
-        return newBuilder;
-    }
-
-    /**
-     * Configure the finagle name string for the server-side routing service.
-     *
-     * @param nameStr name string of the server-side routing service
-     * @return client builder
-     */
-    public DistributedLogClientBuilder serverRoutingServiceFinagleNameStr(String nameStr) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.serverRoutingServiceFinagleName = nameStr;
-        return newBuilder;
-    }
-
-    DistributedLogClientBuilder clientConfig(ClientConfig clientConfig) {
-        DistributedLogClientBuilder newBuilder = newBuilder(this);
-        newBuilder.clientConfig = ClientConfig.newConfig(clientConfig);
-        return newBuilder;
-    }
-
-    /**
-     * Build distributedlog client.
-     *
-     * @return distributedlog client.
-     */
-    public DistributedLogClient build() {
-        return buildClient();
-    }
-
-    /**
-     * Build monitor service client.
-     *
-     * @return monitor service client.
-     */
-    public MonitorServiceClient buildMonitorClient() {
-
-        return buildClient();
-    }
-
-    @SuppressWarnings("unchecked")
-    ClusterClient buildServerRoutingServiceClient(String serverRoutingServiceFinagleName) {
-        ClientBuilder builder = this.clientBuilder;
-        if (null == builder) {
-            builder = ClientBuilder.get()
-                    .tcpConnectTimeout(Duration.fromMilliseconds(200))
-                    .connectTimeout(Duration.fromMilliseconds(200))
-                    .requestTimeout(Duration.fromSeconds(1))
-                    .retries(20);
-            if (!clientConfig.getThriftMux()) {
-                builder = builder.hostConnectionLimit(1);
-            }
-        }
-        if (clientConfig.getThriftMux()) {
-            builder = builder.stack(ThriftMux.client().withClientId(clientId));
-        } else {
-            builder = builder.codec(ThriftClientFramedCodec.apply(Option.apply(clientId)));
-        }
-
-        Name name;
-        try {
-            name = Resolver$.MODULE$.eval(serverRoutingServiceFinagleName);
-        } catch (Exception exc) {
-            logger.error("Exception in Resolver.eval for name {}", serverRoutingServiceFinagleName, exc);
-            throw new RuntimeException(exc);
-        }
-
-        // builder the client
-        Service<ThriftClientRequest, byte[]> client =
-                ClientBuilder.safeBuildFactory(
-                        builder.dest(name).reportTo(statsReceiver.scope("routing"))
-                ).toService();
-        DistributedLogService.ServiceIface service =
-                new DistributedLogService.ServiceToClient(client, new TBinaryProtocol.Factory());
-        return new ClusterClient(client, service);
-    }
-
-    DistributedLogClientImpl buildClient() {
-        checkNotNull(name, "No name provided.");
-        checkNotNull(clientId, "No client id provided.");
-        checkNotNull(routingServiceBuilder, "No routing service builder provided.");
-        checkNotNull(statsReceiver, "No stats receiver provided.");
-        if (null == streamStatsReceiver) {
-            streamStatsReceiver = new NullStatsReceiver();
-        }
-
-        Optional<ClusterClient> serverRoutingServiceClient = Optional.absent();
-        if (null != serverRoutingServiceFinagleName) {
-            serverRoutingServiceClient = Optional.of(
-                    buildServerRoutingServiceClient(serverRoutingServiceFinagleName));
-        }
-
-        RoutingService routingService = routingServiceBuilder
-                .statsReceiver(statsReceiver.scope("routing"))
-                .build();
-        DistributedLogClientImpl clientImpl =
-                new DistributedLogClientImpl(
-                        name,
-                        clientId,
-                        routingService,
-                        clientBuilder,
-                        clientConfig,
-                        serverRoutingServiceClient,
-                        statsReceiver,
-                        streamStatsReceiver,
-                        regionResolver,
-                        enableRegionStats);
-        routingService.startService();
-        clientImpl.handshake();
-        return clientImpl;
-    }
-
-}
diff --git a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/package-info.java b/distributedlog-client/src/main/java/com/twitter/distributedlog/service/package-info.java
deleted file mode 100644
index e6e56c0..0000000
--- a/distributedlog-client/src/main/java/com/twitter/distributedlog/service/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Service Client.
- */
-package com.twitter.distributedlog.service;
\ No newline at end of file
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/ClientConfig.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/ClientConfig.java
new file mode 100644
index 0000000..57e2b5a
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/ClientConfig.java
@@ -0,0 +1,187 @@
+/**
+ * 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.client;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Client Config.
+ */
+public class ClientConfig {
+    int redirectBackoffStartMs = 25;
+    int redirectBackoffMaxMs = 100;
+    int maxRedirects = -1;
+    int requestTimeoutMs = -1;
+    boolean thriftmux = false;
+    boolean streamFailfast = false;
+    String streamNameRegex = ".*";
+    boolean handshakeWithClientInfo = true;
+    long periodicHandshakeIntervalMs = TimeUnit.MINUTES.toMillis(5);
+    long periodicOwnershipSyncIntervalMs = TimeUnit.MINUTES.toMillis(5);
+    boolean periodicDumpOwnershipCacheEnabled = false;
+    long periodicDumpOwnershipCacheIntervalMs = TimeUnit.MINUTES.toMillis(10);
+    boolean enableHandshakeTracing = false;
+    boolean enableChecksum = true;
+
+    public ClientConfig setMaxRedirects(int maxRedirects) {
+        this.maxRedirects = maxRedirects;
+        return this;
+    }
+
+    public int getMaxRedirects() {
+        return this.maxRedirects;
+    }
+
+    public ClientConfig setRequestTimeoutMs(int timeoutInMillis) {
+        this.requestTimeoutMs = timeoutInMillis;
+        return this;
+    }
+
+    public int getRequestTimeoutMs() {
+        return this.requestTimeoutMs;
+    }
+
+    public ClientConfig setRedirectBackoffStartMs(int ms) {
+        this.redirectBackoffStartMs = ms;
+        return this;
+    }
+
+    public int getRedirectBackoffStartMs() {
+        return this.redirectBackoffStartMs;
+    }
+
+    public ClientConfig setRedirectBackoffMaxMs(int ms) {
+        this.redirectBackoffMaxMs = ms;
+        return this;
+    }
+
+    public int getRedirectBackoffMaxMs() {
+        return this.redirectBackoffMaxMs;
+    }
+
+    public ClientConfig setThriftMux(boolean enabled) {
+        this.thriftmux = enabled;
+        return this;
+    }
+
+    public boolean getThriftMux() {
+        return this.thriftmux;
+    }
+
+    public ClientConfig setStreamFailfast(boolean enabled) {
+        this.streamFailfast = enabled;
+        return this;
+    }
+
+    public boolean getStreamFailfast() {
+        return this.streamFailfast;
+    }
+
+    public ClientConfig setStreamNameRegex(String nameRegex) {
+        checkNotNull(nameRegex);
+        this.streamNameRegex = nameRegex;
+        return this;
+    }
+
+    public String getStreamNameRegex() {
+        return this.streamNameRegex;
+    }
+
+    public ClientConfig setHandshakeWithClientInfo(boolean enabled) {
+        this.handshakeWithClientInfo = enabled;
+        return this;
+    }
+
+    public boolean getHandshakeWithClientInfo() {
+        return this.handshakeWithClientInfo;
+    }
+
+    public ClientConfig setPeriodicHandshakeIntervalMs(long intervalMs) {
+        this.periodicHandshakeIntervalMs = intervalMs;
+        return this;
+    }
+
+    public long getPeriodicHandshakeIntervalMs() {
+        return this.periodicHandshakeIntervalMs;
+    }
+
+    public ClientConfig setPeriodicOwnershipSyncIntervalMs(long intervalMs) {
+        this.periodicOwnershipSyncIntervalMs = intervalMs;
+        return this;
+    }
+
+    public long getPeriodicOwnershipSyncIntervalMs() {
+        return this.periodicOwnershipSyncIntervalMs;
+    }
+
+    public ClientConfig setPeriodicDumpOwnershipCacheEnabled(boolean enabled) {
+        this.periodicDumpOwnershipCacheEnabled = enabled;
+        return this;
+    }
+
+    public boolean isPeriodicDumpOwnershipCacheEnabled() {
+        return this.periodicDumpOwnershipCacheEnabled;
+    }
+
+    public ClientConfig setPeriodicDumpOwnershipCacheIntervalMs(long intervalMs) {
+        this.periodicDumpOwnershipCacheIntervalMs = intervalMs;
+        return this;
+    }
+
+    public long getPeriodicDumpOwnershipCacheIntervalMs() {
+        return this.periodicDumpOwnershipCacheIntervalMs;
+    }
+
+    public ClientConfig setHandshakeTracingEnabled(boolean enabled) {
+        this.enableHandshakeTracing = enabled;
+        return this;
+    }
+
+    public boolean isHandshakeTracingEnabled() {
+        return this.enableHandshakeTracing;
+    }
+
+    public ClientConfig setChecksumEnabled(boolean enabled) {
+        this.enableChecksum = enabled;
+        return this;
+    }
+
+    public boolean isChecksumEnabled() {
+        return this.enableChecksum;
+    }
+
+    public static ClientConfig newConfig(ClientConfig config) {
+        ClientConfig newConfig = new ClientConfig();
+        newConfig.setMaxRedirects(config.getMaxRedirects())
+                 .setRequestTimeoutMs(config.getRequestTimeoutMs())
+                 .setRedirectBackoffStartMs(config.getRedirectBackoffStartMs())
+                 .setRedirectBackoffMaxMs(config.getRedirectBackoffMaxMs())
+                 .setThriftMux(config.getThriftMux())
+                 .setStreamFailfast(config.getStreamFailfast())
+                 .setStreamNameRegex(config.getStreamNameRegex())
+                 .setHandshakeWithClientInfo(config.getHandshakeWithClientInfo())
+                 .setPeriodicHandshakeIntervalMs(config.getPeriodicHandshakeIntervalMs())
+                 .setPeriodicDumpOwnershipCacheEnabled(config.isPeriodicDumpOwnershipCacheEnabled())
+                 .setPeriodicDumpOwnershipCacheIntervalMs(config.getPeriodicDumpOwnershipCacheIntervalMs())
+                 .setHandshakeTracingEnabled(config.isHandshakeTracingEnabled())
+                 .setChecksumEnabled(config.isChecksumEnabled());
+        return newConfig;
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogClientImpl.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogClientImpl.java
new file mode 100644
index 0000000..1300187
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogClientImpl.java
@@ -0,0 +1,1200 @@
+/**
+ * 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.client;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Sets;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecordSetBuffer;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.client.ownership.OwnershipCache;
+import org.apache.distributedlog.client.proxy.ClusterClient;
+import org.apache.distributedlog.client.proxy.HostProvider;
+import org.apache.distributedlog.client.proxy.ProxyClient;
+import org.apache.distributedlog.client.proxy.ProxyClientManager;
+import org.apache.distributedlog.client.proxy.ProxyListener;
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.client.routing.RoutingService.RoutingContext;
+import org.apache.distributedlog.client.stats.ClientStats;
+import org.apache.distributedlog.client.stats.OpStats;
+import org.apache.distributedlog.exceptions.DLClientClosedException;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.ServiceUnavailableException;
+import org.apache.distributedlog.exceptions.StreamUnavailableException;
+import org.apache.distributedlog.service.DLSocketAddress;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.thrift.service.BulkWriteResponse;
+import org.apache.distributedlog.thrift.service.HeartbeatOptions;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.thrift.service.ServerInfo;
+import org.apache.distributedlog.thrift.service.ServerStatus;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import org.apache.distributedlog.thrift.service.WriteContext;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import org.apache.distributedlog.util.ProtocolUtils;
+import com.twitter.finagle.CancelledRequestException;
+import com.twitter.finagle.ConnectionFailedException;
+import com.twitter.finagle.Failure;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.RequestTimeoutException;
+import com.twitter.finagle.ServiceException;
+import com.twitter.finagle.ServiceTimeoutException;
+import com.twitter.finagle.WriteException;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.finagle.thrift.ClientId;
+import com.twitter.util.Duration;
+import com.twitter.util.Function;
+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 java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+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.locks.ReentrantReadWriteLock;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+import org.apache.thrift.TApplicationException;
+import org.jboss.netty.channel.ChannelException;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.jboss.netty.util.Timeout;
+import org.jboss.netty.util.TimerTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.collection.Seq;
+import scala.runtime.AbstractFunction1;
+
+
+/**
+ * Implementation of distributedlog client.
+ */
+public class DistributedLogClientImpl implements DistributedLogClient, MonitorServiceClient,
+        RoutingService.RoutingListener, ProxyListener, HostProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(DistributedLogClientImpl.class);
+
+    private final String clientName;
+    private final ClientId clientId;
+    private final ClientConfig clientConfig;
+    private final RoutingService routingService;
+    private final ProxyClient.Builder clientBuilder;
+    private final boolean streamFailfast;
+    private final Pattern streamNameRegexPattern;
+
+    // Timer
+    private final HashedWheelTimer dlTimer;
+
+    // region resolver
+    private final RegionResolver regionResolver;
+
+    // Ownership maintenance
+    private final OwnershipCache ownershipCache;
+    // Channel/Client management
+    private final ProxyClientManager clientManager;
+    // Cluster Client (for routing service)
+    private final Optional<ClusterClient> clusterClient;
+
+    // Close Status
+    private boolean closed = false;
+    private final ReentrantReadWriteLock closeLock =
+            new ReentrantReadWriteLock();
+
+    abstract class StreamOp implements TimerTask {
+        final String stream;
+
+        final AtomicInteger tries = new AtomicInteger(0);
+        final RoutingContext routingContext = RoutingContext.of(regionResolver);
+        final WriteContext ctx = new WriteContext();
+        final Stopwatch stopwatch;
+        final OpStats opStats;
+        SocketAddress nextAddressToSend;
+
+        StreamOp(final String stream, final OpStats opStats) {
+            this.stream = stream;
+            this.stopwatch = Stopwatch.createStarted();
+            this.opStats = opStats;
+        }
+
+        boolean shouldTimeout() {
+            long elapsedMs = stopwatch.elapsed(TimeUnit.MILLISECONDS);
+            return shouldTimeout(elapsedMs);
+        }
+
+        boolean shouldTimeout(long elapsedMs) {
+            return clientConfig.getRequestTimeoutMs() > 0
+                && elapsedMs >= clientConfig.getRequestTimeoutMs();
+        }
+
+        void send(SocketAddress address) {
+            long elapsedMs = stopwatch.elapsed(TimeUnit.MILLISECONDS);
+            if (clientConfig.getMaxRedirects() > 0
+                && tries.get() >= clientConfig.getMaxRedirects()) {
+                fail(address, new RequestTimeoutException(Duration.fromMilliseconds(elapsedMs),
+                        "Exhausted max redirects in " + elapsedMs + " ms"));
+                return;
+            } else if (shouldTimeout(elapsedMs)) {
+                fail(address, new RequestTimeoutException(Duration.fromMilliseconds(elapsedMs),
+                        "Exhausted max request timeout " + clientConfig.getRequestTimeoutMs()
+                                + " in " + elapsedMs + " ms"));
+                return;
+            }
+            synchronized (this) {
+                String addrStr = address.toString();
+                if (ctx.isSetTriedHosts() && ctx.getTriedHosts().contains(addrStr)) {
+                    nextAddressToSend = address;
+                    dlTimer.newTimeout(this,
+                            Math.min(clientConfig.getRedirectBackoffMaxMs(),
+                                    tries.get() * clientConfig.getRedirectBackoffStartMs()),
+                            TimeUnit.MILLISECONDS);
+                } else {
+                    doSend(address);
+                }
+            }
+        }
+
+        abstract Future<ResponseHeader> sendRequest(ProxyClient sc);
+
+        void doSend(SocketAddress address) {
+            ctx.addToTriedHosts(address.toString());
+            if (clientConfig.isChecksumEnabled()) {
+                Long crc32 = computeChecksum();
+                if (null != crc32) {
+                    ctx.setCrc32(crc32);
+                }
+            }
+            tries.incrementAndGet();
+            sendWriteRequest(address, this);
+        }
+
+        void beforeComplete(ProxyClient sc, ResponseHeader responseHeader) {
+            ownershipCache.updateOwner(stream, sc.getAddress());
+        }
+
+        void complete(SocketAddress address) {
+            stopwatch.stop();
+            opStats.completeRequest(address,
+                    stopwatch.elapsed(TimeUnit.MICROSECONDS), tries.get());
+        }
+
+        void fail(SocketAddress address, Throwable t) {
+            stopwatch.stop();
+            opStats.failRequest(address,
+                    stopwatch.elapsed(TimeUnit.MICROSECONDS), tries.get());
+        }
+
+        Long computeChecksum() {
+            return null;
+        }
+
+        @Override
+        public synchronized void run(Timeout timeout) throws Exception {
+            if (!timeout.isCancelled() && null != nextAddressToSend) {
+                doSend(nextAddressToSend);
+            } else {
+                fail(null, new CancelledRequestException());
+            }
+        }
+    }
+
+    class BulkWriteOp extends StreamOp {
+
+        final List<ByteBuffer> data;
+        final ArrayList<Promise<DLSN>> results;
+
+        BulkWriteOp(final String name, final List<ByteBuffer> data) {
+            super(name, clientStats.getOpStats("bulk_write"));
+            this.data = data;
+
+            // This could take a while (relatively speaking) for very large inputs. We probably don't want
+            // to go so large for other reasons though.
+            this.results = new ArrayList<Promise<DLSN>>(data.size());
+            for (int i = 0; i < data.size(); i++) {
+                checkNotNull(data.get(i));
+                this.results.add(new Promise<DLSN>());
+            }
+        }
+
+        @Override
+        Future<ResponseHeader> sendRequest(final ProxyClient sc) {
+            return sc.getService().writeBulkWithContext(stream, data, ctx)
+                .addEventListener(new FutureEventListener<BulkWriteResponse>() {
+                @Override
+                public void onSuccess(BulkWriteResponse response) {
+                    // For non-success case, the ResponseHeader handler (the caller) will handle it.
+                    // Note success in this case means no finagle errors have occurred
+                    // (such as finagle connection issues). In general code != SUCCESS means there's some error
+                    // reported by dlog service. The caller will handle such errors.
+                    if (response.getHeader().getCode() == StatusCode.SUCCESS) {
+                        beforeComplete(sc, response.getHeader());
+                        BulkWriteOp.this.complete(sc.getAddress(), response);
+                        if (response.getWriteResponses().size() == 0 && data.size() > 0) {
+                            logger.error("non-empty bulk write got back empty response without failure for stream {}",
+                                stream);
+                        }
+                    }
+                }
+                @Override
+                public void onFailure(Throwable cause) {
+                    // Handled by the ResponseHeader listener (attached by the caller).
+                }
+            }).map(new AbstractFunction1<BulkWriteResponse, ResponseHeader>() {
+                @Override
+                public ResponseHeader apply(BulkWriteResponse response) {
+                    // We need to return the ResponseHeader to the caller's listener to process DLOG errors.
+                    return response.getHeader();
+                }
+            });
+        }
+
+        void complete(SocketAddress address, BulkWriteResponse bulkWriteResponse) {
+            super.complete(address);
+            Iterator<WriteResponse> writeResponseIterator = bulkWriteResponse.getWriteResponses().iterator();
+            Iterator<Promise<DLSN>> resultIterator = results.iterator();
+
+            // Fill in errors from thrift responses.
+            while (resultIterator.hasNext() && writeResponseIterator.hasNext()) {
+                Promise<DLSN> result = resultIterator.next();
+                WriteResponse writeResponse = writeResponseIterator.next();
+                if (StatusCode.SUCCESS == writeResponse.getHeader().getCode()) {
+                    result.setValue(DLSN.deserialize(writeResponse.getDlsn()));
+                } else {
+                    result.setException(DLException.of(writeResponse.getHeader()));
+                }
+            }
+
+            // Should never happen, but just in case so there's some record.
+            if (bulkWriteResponse.getWriteResponses().size() != data.size()) {
+                logger.error("wrong number of results, response = {} records = {}",
+                    bulkWriteResponse.getWriteResponses().size(), data.size());
+            }
+        }
+
+        @Override
+        void fail(SocketAddress address, Throwable t) {
+
+            // StreamOp.fail is called to fail the overall request. In case of BulkWriteOp we take the request level
+            // exception to apply to the first write. In fact for request level exceptions no request has ever been
+            // attempted, but logically we associate the error with the first write.
+            super.fail(address, t);
+            Iterator<Promise<DLSN>> resultIterator = results.iterator();
+
+            // Fail the first write with the batch level failure.
+            if (resultIterator.hasNext()) {
+                Promise<DLSN> result = resultIterator.next();
+                result.setException(t);
+            }
+
+            // Fail the remaining writes as cancelled requests.
+            while (resultIterator.hasNext()) {
+                Promise<DLSN> result = resultIterator.next();
+                result.setException(new CancelledRequestException());
+            }
+        }
+
+        @SuppressWarnings("unchecked")
+        List<Future<DLSN>> result() {
+            return (List) results;
+        }
+    }
+
+    abstract class AbstractWriteOp extends StreamOp {
+
+        final Promise<WriteResponse> result = new Promise<WriteResponse>();
+        Long crc32 = null;
+
+        AbstractWriteOp(final String name, final OpStats opStats) {
+            super(name, opStats);
+        }
+
+        void complete(SocketAddress address, WriteResponse response) {
+            super.complete(address);
+            result.setValue(response);
+        }
+
+        @Override
+        void fail(SocketAddress address, Throwable t) {
+            super.fail(address, t);
+            result.setException(t);
+        }
+
+        @Override
+        Long computeChecksum() {
+            if (null == crc32) {
+                crc32 = ProtocolUtils.streamOpCRC32(stream);
+            }
+            return crc32;
+        }
+
+        @Override
+        Future<ResponseHeader> sendRequest(final ProxyClient sc) {
+            return this.sendWriteRequest(sc).addEventListener(new FutureEventListener<WriteResponse>() {
+                @Override
+                public void onSuccess(WriteResponse response) {
+                    if (response.getHeader().getCode() == StatusCode.SUCCESS) {
+                        beforeComplete(sc, response.getHeader());
+                        AbstractWriteOp.this.complete(sc.getAddress(), response);
+                    }
+                }
+                @Override
+                public void onFailure(Throwable cause) {
+                    // handled by the ResponseHeader listener
+                }
+            }).map(new AbstractFunction1<WriteResponse, ResponseHeader>() {
+                @Override
+                public ResponseHeader apply(WriteResponse response) {
+                    return response.getHeader();
+                }
+            });
+        }
+
+        abstract Future<WriteResponse> sendWriteRequest(ProxyClient sc);
+    }
+
+    class WriteOp extends AbstractWriteOp {
+        final ByteBuffer data;
+
+        WriteOp(final String name, final ByteBuffer data) {
+            super(name, clientStats.getOpStats("write"));
+            this.data = data;
+        }
+
+        @Override
+        Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
+            return sc.getService().writeWithContext(stream, data, ctx);
+        }
+
+        @Override
+        Long computeChecksum() {
+            if (null == crc32) {
+                byte[] dataBytes = new byte[data.remaining()];
+                data.duplicate().get(dataBytes);
+                crc32 = ProtocolUtils.writeOpCRC32(stream, dataBytes);
+            }
+            return crc32;
+        }
+
+        Future<DLSN> result() {
+            return result.map(new AbstractFunction1<WriteResponse, DLSN>() {
+                @Override
+                public DLSN apply(WriteResponse response) {
+                    return DLSN.deserialize(response.getDlsn());
+                }
+            });
+        }
+    }
+
+    class TruncateOp extends AbstractWriteOp {
+        final DLSN dlsn;
+
+        TruncateOp(String name, DLSN dlsn) {
+            super(name, clientStats.getOpStats("truncate"));
+            this.dlsn = dlsn;
+        }
+
+        @Override
+        Long computeChecksum() {
+            if (null == crc32) {
+                crc32 = ProtocolUtils.truncateOpCRC32(stream, dlsn);
+            }
+            return crc32;
+        }
+
+        @Override
+        Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
+            return sc.getService().truncate(stream, dlsn.serialize(), ctx);
+        }
+
+        Future<Boolean> result() {
+            return result.map(new AbstractFunction1<WriteResponse, Boolean>() {
+                @Override
+                public Boolean apply(WriteResponse response) {
+                    return true;
+                }
+            });
+        }
+    }
+
+    class WriteRecordSetOp extends WriteOp {
+
+        WriteRecordSetOp(String name, LogRecordSetBuffer recordSet) {
+            super(name, recordSet.getBuffer());
+            ctx.setIsRecordSet(true);
+        }
+
+    }
+
+
+    class ReleaseOp extends AbstractWriteOp {
+
+        ReleaseOp(String name) {
+            super(name, clientStats.getOpStats("release"));
+        }
+
+        @Override
+        Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
+            return sc.getService().release(stream, ctx);
+        }
+
+        @Override
+        void beforeComplete(ProxyClient sc, ResponseHeader header) {
+            ownershipCache.removeOwnerFromStream(stream, sc.getAddress(), "Stream Deleted");
+        }
+
+        Future<Void> result() {
+            return result.map(new AbstractFunction1<WriteResponse, Void>() {
+                @Override
+                public Void apply(WriteResponse response) {
+                    return null;
+                }
+            });
+        }
+    }
+
+    class DeleteOp extends AbstractWriteOp {
+
+        DeleteOp(String name) {
+            super(name, clientStats.getOpStats("delete"));
+        }
+
+        @Override
+        Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
+            return sc.getService().delete(stream, ctx);
+        }
+
+        @Override
+        void beforeComplete(ProxyClient sc, ResponseHeader header) {
+            ownershipCache.removeOwnerFromStream(stream, sc.getAddress(), "Stream Deleted");
+        }
+
+        Future<Void> result() {
+            return result.map(new AbstractFunction1<WriteResponse, Void>() {
+                @Override
+                public Void apply(WriteResponse v1) {
+                    return null;
+                }
+            });
+        }
+    }
+
+    class CreateOp extends AbstractWriteOp {
+
+        CreateOp(String name) {
+            super(name, clientStats.getOpStats("create"));
+        }
+
+        @Override
+        Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
+            return sc.getService().create(stream, ctx);
+        }
+
+        @Override
+        void beforeComplete(ProxyClient sc, ResponseHeader header) {
+            ownershipCache.updateOwner(stream, sc.getAddress());
+        }
+
+        Future<Void> result() {
+            return result.map(new AbstractFunction1<WriteResponse, Void>() {
+                @Override
+                public Void apply(WriteResponse v1) {
+                    return null;
+                }
+            }).voided();
+        }
+    }
+
+    class HeartbeatOp extends AbstractWriteOp {
+        HeartbeatOptions options;
+
+        HeartbeatOp(String name, boolean sendReaderHeartBeat) {
+            super(name, clientStats.getOpStats("heartbeat"));
+            options = new HeartbeatOptions();
+            options.setSendHeartBeatToReader(sendReaderHeartBeat);
+        }
+
+        @Override
+        Future<WriteResponse> sendWriteRequest(ProxyClient sc) {
+            return sc.getService().heartbeatWithOptions(stream, ctx, options);
+        }
+
+        Future<Void> result() {
+            return result.map(new AbstractFunction1<WriteResponse, Void>() {
+                @Override
+                public Void apply(WriteResponse response) {
+                    return null;
+                }
+            });
+        }
+    }
+
+    // Stats
+    private final ClientStats clientStats;
+
+    public DistributedLogClientImpl(String name,
+                                    ClientId clientId,
+                                    RoutingService routingService,
+                                    ClientBuilder clientBuilder,
+                                    ClientConfig clientConfig,
+                                    Optional<ClusterClient> clusterClient,
+                                    StatsReceiver statsReceiver,
+                                    StatsReceiver streamStatsReceiver,
+                                    RegionResolver regionResolver,
+                                    boolean enableRegionStats) {
+        this.clientName = name;
+        this.clientId = clientId;
+        this.routingService = routingService;
+        this.clientConfig = clientConfig;
+        this.streamFailfast = clientConfig.getStreamFailfast();
+        this.streamNameRegexPattern = Pattern.compile(clientConfig.getStreamNameRegex());
+        this.regionResolver = regionResolver;
+        // Build the timer
+        this.dlTimer = new HashedWheelTimer(
+                new ThreadFactoryBuilder().setNameFormat("DLClient-" + name + "-timer-%d").build(),
+                this.clientConfig.getRedirectBackoffStartMs(),
+                TimeUnit.MILLISECONDS);
+        // register routing listener
+        this.routingService.registerListener(this);
+        // build the ownership cache
+        this.ownershipCache = new OwnershipCache(this.clientConfig, this.dlTimer, statsReceiver, streamStatsReceiver);
+        // Client Stats
+        this.clientStats = new ClientStats(statsReceiver, enableRegionStats, regionResolver);
+        // Client Manager
+        this.clientBuilder = ProxyClient.newBuilder(clientName, clientId, clientBuilder, clientConfig, clientStats);
+        this.clientManager = new ProxyClientManager(
+                this.clientConfig,  // client config
+                this.clientBuilder, // client builder
+                this.dlTimer,       // timer
+                this,               // host provider
+                clientStats);       // client stats
+        this.clusterClient = clusterClient;
+        this.clientManager.registerProxyListener(this);
+
+        // Cache Stats
+        StatsReceiver cacheStatReceiver = statsReceiver.scope("cache");
+        Seq<String> numCachedStreamsGaugeName =
+                scala.collection.JavaConversions.asScalaBuffer(Arrays.asList("num_streams")).toList();
+        cacheStatReceiver.provideGauge(numCachedStreamsGaugeName, new Function0<Object>() {
+            @Override
+            public Object apply() {
+                return (float) ownershipCache.getNumCachedStreams();
+            }
+        });
+        Seq<String> numCachedHostsGaugeName =
+                scala.collection.JavaConversions.asScalaBuffer(Arrays.asList("num_hosts")).toList();
+        cacheStatReceiver.provideGauge(numCachedHostsGaugeName, new Function0<Object>() {
+            @Override
+            public Object apply() {
+                return (float) clientManager.getNumProxies();
+            }
+        });
+
+        logger.info("Build distributedlog client : name = {}, client_id = {}, routing_service = {},"
+            + " stats_receiver = {}, thriftmux = {}",
+            new Object[] {
+                name,
+                clientId,
+                routingService.getClass(),
+                statsReceiver.getClass(),
+                clientConfig.getThriftMux()
+            });
+    }
+
+    @Override
+    public Set<SocketAddress> getHosts() {
+        Set<SocketAddress> hosts = Sets.newHashSet();
+        // if using server side routing, we only handshake with the hosts in ownership cache.
+        if (!clusterClient.isPresent()) {
+            hosts.addAll(this.routingService.getHosts());
+        }
+        hosts.addAll(this.ownershipCache.getStreamOwnershipDistribution().keySet());
+        return hosts;
+    }
+
+    @Override
+    public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
+        if (null != serverInfo
+            && serverInfo.isSetServerStatus()
+            && ServerStatus.DOWN == serverInfo.getServerStatus()) {
+            logger.info("{} is detected as DOWN during handshaking", address);
+            // server is shutting down
+            handleServiceUnavailable(address, client, Optional.<StreamOp>absent());
+            return;
+        }
+
+        if (null != serverInfo && serverInfo.isSetOwnerships()) {
+            Map<String, String> ownerships = serverInfo.getOwnerships();
+            logger.debug("Handshaked with {} : {} ownerships returned.", address, ownerships.size());
+            for (Map.Entry<String, String> entry : ownerships.entrySet()) {
+                Matcher matcher = streamNameRegexPattern.matcher(entry.getKey());
+                if (!matcher.matches()) {
+                    continue;
+                }
+                updateOwnership(entry.getKey(), entry.getValue());
+            }
+        } else {
+            logger.debug("Handshaked with {} : no ownerships returned", address);
+        }
+    }
+
+    @Override
+    public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
+        cause = showRootCause(Optional.<StreamOp>absent(), cause);
+        handleRequestException(address, client, Optional.<StreamOp>absent(), cause);
+    }
+
+    @VisibleForTesting
+    public void handshake() {
+        clientManager.handshake();
+        logger.info("Handshaked with {} hosts, cached {} streams",
+                clientManager.getNumProxies(), ownershipCache.getNumCachedStreams());
+    }
+
+    @Override
+    public void onServerLeft(SocketAddress address) {
+        onServerLeft(address, null);
+    }
+
+    private void onServerLeft(SocketAddress address, ProxyClient sc) {
+        ownershipCache.removeAllStreamsFromOwner(address);
+        if (null == sc) {
+            clientManager.removeClient(address);
+        } else {
+            clientManager.removeClient(address, sc);
+        }
+    }
+
+    @Override
+    public void onServerJoin(SocketAddress address) {
+        // we only pre-create connection for client-side routing
+        // if it is server side routing, we only know the exact proxy address
+        // when #getOwner.
+        if (!clusterClient.isPresent()) {
+            clientManager.createClient(address);
+        }
+    }
+
+    public void close() {
+        closeLock.writeLock().lock();
+        try {
+            if (closed) {
+                return;
+            }
+            closed = true;
+        } finally {
+            closeLock.writeLock().unlock();
+        }
+        clientManager.close();
+        routingService.unregisterListener(this);
+        routingService.stopService();
+        dlTimer.stop();
+    }
+
+    @Override
+    public Future<Void> check(String stream) {
+        final HeartbeatOp op = new HeartbeatOp(stream, false);
+        sendRequest(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<Void> heartbeat(String stream) {
+        final HeartbeatOp op = new HeartbeatOp(stream, true);
+        sendRequest(op);
+        return op.result();
+    }
+
+    @Override
+    public Map<SocketAddress, Set<String>> getStreamOwnershipDistribution() {
+        return ownershipCache.getStreamOwnershipDistribution();
+    }
+
+    @Override
+    public Future<Void> setAcceptNewStream(boolean enabled) {
+        Map<SocketAddress, ProxyClient> snapshot = clientManager.getAllClients();
+        List<Future<Void>> futures = new ArrayList<Future<Void>>(snapshot.size());
+        for (Map.Entry<SocketAddress, ProxyClient> entry : snapshot.entrySet()) {
+            futures.add(entry.getValue().getService().setAcceptNewStream(enabled));
+        }
+        return Future.collect(futures).map(new Function<List<Void>, Void>() {
+            @Override
+            public Void apply(List<Void> list) {
+                return null;
+            }
+        });
+    }
+
+    @Override
+    public Future<DLSN> write(String stream, ByteBuffer data) {
+        final WriteOp op = new WriteOp(stream, data);
+        sendRequest(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<DLSN> writeRecordSet(String stream, final LogRecordSetBuffer recordSet) {
+        final WriteRecordSetOp op = new WriteRecordSetOp(stream, recordSet);
+        sendRequest(op);
+        return op.result();
+    }
+
+    @Override
+    public List<Future<DLSN>> writeBulk(String stream, List<ByteBuffer> data) {
+        if (data.size() > 0) {
+            final BulkWriteOp op = new BulkWriteOp(stream, data);
+            sendRequest(op);
+            return op.result();
+        } else {
+            return Collections.emptyList();
+        }
+    }
+
+    @Override
+    public Future<Boolean> truncate(String stream, DLSN dlsn) {
+        final TruncateOp op = new TruncateOp(stream, dlsn);
+        sendRequest(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<Void> delete(String stream) {
+        final DeleteOp op = new DeleteOp(stream);
+        sendRequest(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<Void> release(String stream) {
+        final ReleaseOp op = new ReleaseOp(stream);
+        sendRequest(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<Void> create(String stream) {
+        final CreateOp op = new CreateOp(stream);
+        sendRequest(op);
+        return op.result();
+    }
+
+    private void sendRequest(final StreamOp op) {
+        closeLock.readLock().lock();
+        try {
+            if (closed) {
+                op.fail(null, new DLClientClosedException("Client " + clientName + " is closed."));
+            } else {
+                doSend(op, null);
+            }
+        } finally {
+            closeLock.readLock().unlock();
+        }
+    }
+
+    /**
+     * Send the stream operation by routing service, excluding previous address if it is not null.
+     *
+     * @param op
+     *          stream operation.
+     * @param previousAddr
+     *          previous tried address.
+     */
+    private void doSend(final StreamOp op, final SocketAddress previousAddr) {
+        if (null != previousAddr) {
+            op.routingContext.addTriedHost(previousAddr, StatusCode.WRITE_EXCEPTION);
+        }
+        // Get host first
+        final SocketAddress address = ownershipCache.getOwner(op.stream);
+        if (null == address || op.routingContext.isTriedHost(address)) {
+            getOwner(op).addEventListener(new FutureEventListener<SocketAddress>() {
+                @Override
+                public void onFailure(Throwable cause) {
+                    op.fail(null, cause);
+                }
+
+                @Override
+                public void onSuccess(SocketAddress ownerAddr) {
+                    op.send(ownerAddr);
+                }
+            });
+        } else {
+            op.send(address);
+        }
+    }
+
+    private void retryGetOwnerFromResourcePlacementServer(final StreamOp op,
+                                                final Promise<SocketAddress> getOwnerPromise,
+                                                final Throwable cause) {
+        if (op.shouldTimeout()) {
+            op.fail(null, cause);
+            return;
+        }
+        getOwnerFromResourcePlacementServer(op, getOwnerPromise);
+    }
+
+    private void getOwnerFromResourcePlacementServer(final StreamOp op,
+                                                     final Promise<SocketAddress> getOwnerPromise) {
+        clusterClient.get().getService().getOwner(op.stream, op.ctx)
+            .addEventListener(new FutureEventListener<WriteResponse>() {
+                @Override
+                public void onFailure(Throwable cause) {
+                    getOwnerPromise.updateIfEmpty(new Throw<SocketAddress>(cause));
+                }
+
+                @Override
+                public void onSuccess(WriteResponse value) {
+                    if (StatusCode.FOUND == value.getHeader().getCode()
+                          && null != value.getHeader().getLocation()) {
+                        try {
+                            InetSocketAddress addr = DLSocketAddress.deserialize(
+                                value.getHeader().getLocation()
+                            ).getSocketAddress();
+                            getOwnerPromise.updateIfEmpty(new Return<SocketAddress>(addr));
+                        } catch (IOException e) {
+                            // retry from the routing server again
+                            logger.error("ERROR in getOwner", e);
+                            retryGetOwnerFromResourcePlacementServer(op, getOwnerPromise, e);
+                            return;
+                        }
+                    } else {
+                        // retry from the routing server again
+                        retryGetOwnerFromResourcePlacementServer(op, getOwnerPromise,
+                                new StreamUnavailableException("Stream " + op.stream + "'s owner is unknown"));
+                    }
+                }
+            });
+    }
+
+    private Future<SocketAddress> getOwner(final StreamOp op) {
+        if (clusterClient.isPresent()) {
+            final Promise<SocketAddress> getOwnerPromise = new Promise<SocketAddress>();
+            getOwnerFromResourcePlacementServer(op, getOwnerPromise);
+            return getOwnerPromise;
+        }
+        // pickup host by hashing
+        try {
+            return Future.value(routingService.getHost(op.stream, op.routingContext));
+        } catch (NoBrokersAvailableException nbae) {
+            return Future.exception(nbae);
+        }
+    }
+
+    private void sendWriteRequest(final SocketAddress addr, final StreamOp op) {
+        // Get corresponding finagle client
+        final ProxyClient sc = clientManager.getClient(addr);
+        final long startTimeNanos = System.nanoTime();
+        // write the request to that host.
+        op.sendRequest(sc).addEventListener(new FutureEventListener<ResponseHeader>() {
+            @Override
+            public void onSuccess(ResponseHeader header) {
+                if (logger.isDebugEnabled()) {
+                    logger.debug("Received response; header: {}", header);
+                }
+                clientStats.completeProxyRequest(addr, header.getCode(), startTimeNanos);
+                // update routing context
+                op.routingContext.addTriedHost(addr, header.getCode());
+                switch (header.getCode()) {
+                    case SUCCESS:
+                        // success handling is done per stream op
+                        break;
+                    case FOUND:
+                        handleRedirectResponse(header, op, addr);
+                        break;
+                    // for overcapacity, dont report failure since this normally happens quite a bit
+                    case OVER_CAPACITY:
+                        logger.debug("Failed to write request to {} : {}", op.stream, header);
+                        op.fail(addr, DLException.of(header));
+                        break;
+                    // for responses that indicate the requests definitely failed,
+                    // we should fail them immediately (e.g. TOO_LARGE_RECORD, METADATA_EXCEPTION)
+                    case NOT_IMPLEMENTED:
+                    case METADATA_EXCEPTION:
+                    case LOG_EMPTY:
+                    case LOG_NOT_FOUND:
+                    case TRUNCATED_TRANSACTION:
+                    case END_OF_STREAM:
+                    case TRANSACTION_OUT_OF_ORDER:
+                    case INVALID_STREAM_NAME:
+                    case REQUEST_DENIED:
+                    case TOO_LARGE_RECORD:
+                    case CHECKSUM_FAILED:
+                    // status code NOT_READY is returned if failfast is enabled in the server. don't redirect
+                    // since the proxy may still own the stream.
+                    case STREAM_NOT_READY:
+                        op.fail(addr, DLException.of(header));
+                        break;
+                    case SERVICE_UNAVAILABLE:
+                        handleServiceUnavailable(addr, sc, Optional.of(op));
+                        break;
+                    case REGION_UNAVAILABLE:
+                        // region is unavailable, redirect the request to hosts in other region
+                        redirect(op, null);
+                        break;
+                    // Proxy was overloaded and refused to try to acquire the stream. Don't remove ownership, since
+                    // we didn't have it in the first place.
+                    case TOO_MANY_STREAMS:
+                        handleRedirectableError(addr, op, header);
+                        break;
+                    case STREAM_UNAVAILABLE:
+                    case ZOOKEEPER_ERROR:
+                    case LOCKING_EXCEPTION:
+                    case UNEXPECTED:
+                    case INTERRUPTED:
+                    case BK_TRANSMIT_ERROR:
+                    case FLUSH_TIMEOUT:
+                    default:
+                        // when we are receiving these exceptions from proxy, it means proxy or the stream is closed
+                        // redirect the request.
+                        ownershipCache.removeOwnerFromStream(op.stream, addr, header.getCode().name());
+                        handleRedirectableError(addr, op, header);
+                        break;
+                }
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                Optional<StreamOp> opOptional = Optional.of(op);
+                cause = showRootCause(opOptional, cause);
+                clientStats.failProxyRequest(addr, cause, startTimeNanos);
+                handleRequestException(addr, sc, opOptional, cause);
+            }
+        });
+    }
+
+    // Response Handlers
+
+    Throwable showRootCause(Optional<StreamOp> op, Throwable cause) {
+        if (cause instanceof Failure) {
+            Failure failure = (Failure) cause;
+            if (failure.isFlagged(Failure.Wrapped())) {
+                try {
+                    // if it is a wrapped failure, unwrap it first
+                    cause = failure.show();
+                } catch (IllegalArgumentException iae) {
+                    if (op.isPresent()) {
+                        logger.warn("Failed to unwrap finagle failure of stream {} : ", op.get().stream, iae);
+                    } else {
+                        logger.warn("Failed to unwrap finagle failure : ", iae);
+                    }
+                }
+            }
+        }
+        return cause;
+    }
+
+    private void handleRedirectableError(SocketAddress addr,
+                                         StreamOp op,
+                                         ResponseHeader header) {
+        if (streamFailfast) {
+            op.fail(addr, DLException.of(header));
+        } else {
+            redirect(op, null);
+        }
+    }
+
+    void handleServiceUnavailable(SocketAddress addr,
+                                  ProxyClient sc,
+                                  Optional<StreamOp> op) {
+        // service is unavailable, remove it out of routing service
+        routingService.removeHost(addr, new ServiceUnavailableException(addr + " is unavailable now."));
+        onServerLeft(addr);
+        if (op.isPresent()) {
+            ownershipCache.removeOwnerFromStream(op.get().stream, addr, addr + " is unavailable now.");
+            // redirect the request to other host.
+            redirect(op.get(), null);
+        }
+    }
+
+    void handleRequestException(SocketAddress addr,
+                                ProxyClient sc,
+                                Optional<StreamOp> op,
+                                Throwable cause) {
+        boolean resendOp = false;
+        boolean removeOwnerFromStream = false;
+        SocketAddress previousAddr = addr;
+        String reason = cause.getMessage();
+        if (cause instanceof ConnectionFailedException || cause instanceof java.net.ConnectException) {
+            routingService.removeHost(addr, cause);
+            onServerLeft(addr, sc);
+            removeOwnerFromStream = true;
+            // redirect the request to other host.
+            resendOp = true;
+        } else if (cause instanceof ChannelException) {
+            // java.net.ConnectException typically means connection is refused remotely
+            // no process listening on remote address/port.
+            if (cause.getCause() instanceof java.net.ConnectException) {
+                routingService.removeHost(addr, cause.getCause());
+                onServerLeft(addr);
+                reason = cause.getCause().getMessage();
+            } else {
+                routingService.removeHost(addr, cause);
+                reason = cause.getMessage();
+            }
+            removeOwnerFromStream = true;
+            // redirect the request to other host.
+            resendOp = true;
+        } else if (cause instanceof ServiceTimeoutException) {
+            // redirect the request to itself again, which will backoff for a while
+            resendOp = true;
+            previousAddr = null;
+        } else if (cause instanceof WriteException) {
+            // redirect the request to other host.
+            resendOp = true;
+        } else if (cause instanceof ServiceException) {
+            // redirect the request to other host.
+            clientManager.removeClient(addr, sc);
+            resendOp = true;
+        } else if (cause instanceof TApplicationException) {
+            handleTApplicationException(cause, op, addr, sc);
+        } else if (cause instanceof Failure) {
+            handleFinagleFailure((Failure) cause, op, addr);
+        } else {
+            // Default handler
+            handleException(cause, op, addr);
+        }
+
+        if (op.isPresent()) {
+            if (removeOwnerFromStream) {
+                ownershipCache.removeOwnerFromStream(op.get().stream, addr, reason);
+            }
+            if (resendOp) {
+                doSend(op.get(), previousAddr);
+            }
+        }
+    }
+
+    /**
+     * Redirect the request to new proxy <i>newAddr</i>. If <i>newAddr</i> is null,
+     * it would pick up a host from routing service.
+     *
+     * @param op
+     *          stream operation
+     * @param newAddr
+     *          new proxy address
+     */
+    void redirect(StreamOp op, SocketAddress newAddr) {
+        ownershipCache.getOwnershipStatsLogger().onRedirect(op.stream);
+        if (null != newAddr) {
+            logger.debug("Redirect request {} to new owner {}.", op, newAddr);
+            op.send(newAddr);
+        } else {
+            doSend(op, null);
+        }
+    }
+
+    void handleFinagleFailure(Failure failure,
+                              Optional<StreamOp> op,
+                              SocketAddress addr) {
+        if (failure.isFlagged(Failure.Restartable())) {
+            if (op.isPresent()) {
+                // redirect the request to other host
+                doSend(op.get(), addr);
+            }
+        } else {
+            // fail the request if it is other types of failures
+            handleException(failure, op, addr);
+        }
+    }
+
+    void handleException(Throwable cause,
+                         Optional<StreamOp> op,
+                         SocketAddress addr) {
+        // RequestTimeoutException: fail it and let client decide whether to retry or not.
+
+        // FailedFastException:
+        // We don't actually know when FailedFastException will be thrown
+        // so properly we just throw it back to application to let application
+        // handle it.
+
+        // Other Exceptions: as we don't know how to handle them properly so throw them to client
+        if (op.isPresent()) {
+            logger.error("Failed to write request to {} @ {} : {}",
+                    new Object[]{op.get().stream, addr, cause.toString()});
+            op.get().fail(addr, cause);
+        }
+    }
+
+    void handleTApplicationException(Throwable cause,
+                                     Optional<StreamOp> op,
+                                     SocketAddress addr,
+                                     ProxyClient sc) {
+        TApplicationException ex = (TApplicationException) cause;
+        if (ex.getType() == TApplicationException.UNKNOWN_METHOD) {
+            // if we encountered unknown method exception on thrift server, it means this proxy
+            // has problem. we should remove it from routing service, clean up ownerships
+            routingService.removeHost(addr, cause);
+            onServerLeft(addr, sc);
+            if (op.isPresent()) {
+                ownershipCache.removeOwnerFromStream(op.get().stream, addr, cause.getMessage());
+                doSend(op.get(), addr);
+            }
+        } else {
+            handleException(cause, op, addr);
+        }
+    }
+
+    void handleRedirectResponse(ResponseHeader header, StreamOp op, SocketAddress curAddr) {
+        SocketAddress ownerAddr = null;
+        if (header.isSetLocation()) {
+            String owner = header.getLocation();
+            try {
+                ownerAddr = DLSocketAddress.deserialize(owner).getSocketAddress();
+                // if we are receiving a direct request to same host, we won't try the same host.
+                // as the proxy will shut itself down if it redirects client to itself.
+                if (curAddr.equals(ownerAddr)) {
+                    logger.warn("Request to stream {} is redirected to same server {}!", op.stream, curAddr);
+                    ownerAddr = null;
+                } else {
+                    // update ownership when redirects.
+                    ownershipCache.updateOwner(op.stream, ownerAddr);
+                }
+            } catch (IOException e) {
+                ownerAddr = null;
+            }
+        }
+        redirect(op, ownerAddr);
+    }
+
+    void updateOwnership(String stream, String location) {
+        try {
+            SocketAddress ownerAddr = DLSocketAddress.deserialize(location).getSocketAddress();
+            // update ownership
+            ownershipCache.updateOwner(stream, ownerAddr);
+        } catch (IOException e) {
+            logger.warn("Invalid ownership {} found for stream {} : ",
+                new Object[] { location, stream, e });
+        }
+    }
+
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java
new file mode 100644
index 0000000..b3f3368
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java
@@ -0,0 +1,486 @@
+/**
+ * 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.client;
+
+import static com.google.common.base.Preconditions.checkArgument;
+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 com.google.common.base.Stopwatch;
+import com.google.common.base.Ticker;
+import com.google.common.collect.Lists;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecordSet;
+import org.apache.distributedlog.LogRecordSetBuffer;
+import org.apache.distributedlog.client.speculative.DefaultSpeculativeRequestExecutionPolicy;
+import org.apache.distributedlog.client.speculative.SpeculativeRequestExecutionPolicy;
+import org.apache.distributedlog.client.speculative.SpeculativeRequestExecutor;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.service.DistributedLogClient;
+import com.twitter.finagle.IndividualRequestTimeoutException;
+import com.twitter.util.Duration;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Write to multiple streams.
+ */
+public class DistributedLogMultiStreamWriter implements Runnable {
+
+    /**
+     * Create a new builder to create a multi stream writer.
+     *
+     * @return a new builder to create a multi stream writer.
+     */
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    /**
+     * Builder for the multi stream writer.
+     */
+    public static class Builder {
+
+        private DistributedLogClient client = null;
+        private List<String> streams = null;
+        private int bufferSize = 16 * 1024; // 16k
+        private long flushIntervalMicros = 2000; // 2ms
+        private CompressionCodec.Type codec = CompressionCodec.Type.NONE;
+        private ScheduledExecutorService executorService = null;
+        private long requestTimeoutMs = 500; // 500ms
+        private int firstSpeculativeTimeoutMs = 50; // 50ms
+        private int maxSpeculativeTimeoutMs = 200; // 200ms
+        private float speculativeBackoffMultiplier = 2;
+        private Ticker ticker = Ticker.systemTicker();
+
+        private Builder() {}
+
+        /**
+         * Set the distributedlog client used for multi stream writer.
+         *
+         * @param client
+         *          distributedlog client
+         * @return builder
+         */
+        public Builder client(DistributedLogClient client) {
+            this.client = client;
+            return this;
+        }
+
+        /**
+         * Set the list of streams to write to.
+         *
+         * @param streams
+         *          list of streams to write
+         * @return builder
+         */
+        public Builder streams(List<String> streams) {
+            this.streams = streams;
+            return this;
+        }
+
+        /**
+         * Set the output buffer size.
+         *
+         * <p>If output buffer size is 0, the writes will be transmitted to
+         * wire immediately.
+         *
+         * @param bufferSize
+         *          output buffer size
+         * @return builder
+         */
+        public Builder bufferSize(int bufferSize) {
+            this.bufferSize = bufferSize;
+            return this;
+        }
+
+        /**
+         * Set the flush interval in milliseconds.
+         *
+         * @param flushIntervalMs
+         *          flush interval in milliseconds.
+         * @return builder
+         */
+        public Builder flushIntervalMs(int flushIntervalMs) {
+            this.flushIntervalMicros = TimeUnit.MILLISECONDS.toMicros(flushIntervalMs);
+            return this;
+        }
+
+        /**
+         * Set the flush interval in microseconds.
+         *
+         * @param flushIntervalMicros
+         *          flush interval in microseconds.
+         * @return builder
+         */
+        public Builder flushIntervalMicros(int flushIntervalMicros) {
+            this.flushIntervalMicros = flushIntervalMicros;
+            return this;
+        }
+
+        /**
+         * Set compression codec.
+         *
+         * @param codec compression codec.
+         * @return builder
+         */
+        public Builder compressionCodec(CompressionCodec.Type codec) {
+            this.codec = codec;
+            return this;
+        }
+
+        /**
+         * Set the scheduler to flush output buffers.
+         *
+         * @param executorService
+         *          executor service to flush output buffers.
+         * @return builder
+         */
+        public Builder scheduler(ScheduledExecutorService executorService) {
+            this.executorService = executorService;
+            return this;
+        }
+
+        /**
+         * Set request timeout in milliseconds.
+         *
+         * @param requestTimeoutMs
+         *          request timeout in milliseconds.
+         * @return builder
+         */
+        public Builder requestTimeoutMs(long requestTimeoutMs) {
+            this.requestTimeoutMs = requestTimeoutMs;
+            return this;
+        }
+
+        /**
+         * Set the first speculative timeout in milliseconds.
+         *
+         * <p>The multi-streams writer does speculative writes on streams.
+         * The write issues first write request to a stream, if the write request
+         * doesn't respond within speculative timeout. it issues next write request
+         * to a different stream. It does such speculative retries until receive
+         * a success or request timeout ({@link #requestTimeoutMs(long)}).
+         *
+         * <p>This setting is to configure the first speculative timeout, in milliseconds.
+         *
+         * @param timeoutMs
+         *          timeout in milliseconds
+         * @return builder
+         */
+        public Builder firstSpeculativeTimeoutMs(int timeoutMs) {
+            this.firstSpeculativeTimeoutMs = timeoutMs;
+            return this;
+        }
+
+        /**
+         * Set the max speculative timeout in milliseconds.
+         *
+         * <p>The multi-streams writer does speculative writes on streams.
+         * The write issues first write request to a stream, if the write request
+         * doesn't respond within speculative timeout. it issues next write request
+         * to a different stream. It does such speculative retries until receive
+         * a success or request timeout ({@link #requestTimeoutMs(long)}).
+         *
+         * <p>This setting is to configure the max speculative timeout, in milliseconds.
+         *
+         * @param timeoutMs
+         *          timeout in milliseconds
+         * @return builder
+         */
+        public Builder maxSpeculativeTimeoutMs(int timeoutMs) {
+            this.maxSpeculativeTimeoutMs = timeoutMs;
+            return this;
+        }
+
+        /**
+         * Set the speculative timeout backoff multiplier.
+         *
+         * <p>The multi-streams writer does speculative writes on streams.
+         * The write issues first write request to a stream, if the write request
+         * doesn't respond within speculative timeout. it issues next write request
+         * to a different stream. It does such speculative retries until receive
+         * a success or request timeout ({@link #requestTimeoutMs(long)}).
+         *
+         * <p>This setting is to configure the speculative timeout backoff multiplier.
+         *
+         * @param multiplier
+         *          backoff multiplier
+         * @return builder
+         */
+        public Builder speculativeBackoffMultiplier(float multiplier) {
+            this.speculativeBackoffMultiplier = multiplier;
+            return this;
+        }
+
+        /**
+         * Ticker for timing.
+         *
+         * @param ticker
+         *          ticker
+         * @return builder
+         * @see Ticker
+         */
+        public Builder clockTicker(Ticker ticker) {
+            this.ticker = ticker;
+            return this;
+        }
+
+        /**
+         * Build the multi stream writer.
+         *
+         * @return the multi stream writer.
+         */
+        public DistributedLogMultiStreamWriter build() {
+            checkArgument((null != streams && !streams.isEmpty()),
+                    "No streams provided");
+            checkNotNull(client,
+                    "No distributedlog client provided");
+            checkNotNull(codec,
+                    "No compression codec provided");
+            checkArgument(firstSpeculativeTimeoutMs > 0
+                    && firstSpeculativeTimeoutMs <= maxSpeculativeTimeoutMs
+                    && speculativeBackoffMultiplier > 0
+                    && maxSpeculativeTimeoutMs < requestTimeoutMs,
+                    "Invalid speculative timeout settings");
+            return new DistributedLogMultiStreamWriter(
+                streams,
+                client,
+                Math.min(bufferSize, MAX_LOGRECORDSET_SIZE),
+                flushIntervalMicros,
+                requestTimeoutMs,
+                firstSpeculativeTimeoutMs,
+                maxSpeculativeTimeoutMs,
+                speculativeBackoffMultiplier,
+                codec,
+                ticker,
+                executorService);
+        }
+    }
+
+    /**
+     * Pending Write Request.
+     */
+    class PendingWriteRequest implements FutureEventListener<DLSN>,
+            SpeculativeRequestExecutor {
+
+        private final LogRecordSetBuffer recordSet;
+        private AtomicBoolean complete = new AtomicBoolean(false);
+        private final Stopwatch stopwatch = Stopwatch.createStarted(clockTicker);
+        private int nextStream;
+        private int numTriedStreams = 0;
+
+        PendingWriteRequest(LogRecordSetBuffer recordSet) {
+            this.recordSet = recordSet;
+            this.nextStream = Math.abs(nextStreamId.incrementAndGet()) % numStreams;
+        }
+
+        synchronized String sendNextWrite() {
+            long elapsedMs = stopwatch.elapsed(TimeUnit.MILLISECONDS);
+            if (elapsedMs > requestTimeoutMs || numTriedStreams >= numStreams) {
+                fail(new IndividualRequestTimeoutException(Duration.fromMilliseconds(elapsedMs)));
+                return null;
+            }
+            try {
+                return sendWriteToStream(nextStream);
+            } finally {
+                nextStream = (nextStream + 1) % numStreams;
+                ++numTriedStreams;
+            }
+        }
+
+        synchronized String sendWriteToStream(int streamId) {
+            String stream = getStream(streamId);
+            client.writeRecordSet(stream, recordSet)
+                    .addEventListener(this);
+            return stream;
+        }
+
+        @Override
+        public void onSuccess(DLSN dlsn) {
+            if (!complete.compareAndSet(false, true)) {
+                return;
+            }
+            recordSet.completeTransmit(
+                    dlsn.getLogSegmentSequenceNo(),
+                    dlsn.getEntryId(),
+                    dlsn.getSlotId());
+        }
+
+        @Override
+        public void onFailure(Throwable cause) {
+            sendNextWrite();
+        }
+
+        private void fail(Throwable cause) {
+            if (!complete.compareAndSet(false, true)) {
+                return;
+            }
+            recordSet.abortTransmit(cause);
+        }
+
+        @Override
+        public Future<Boolean> issueSpeculativeRequest() {
+            return Future.value(!complete.get() && null != sendNextWrite());
+        }
+    }
+
+    private final int numStreams;
+    private final List<String> streams;
+    private final DistributedLogClient client;
+    private final int bufferSize;
+    private final long requestTimeoutMs;
+    private final SpeculativeRequestExecutionPolicy speculativePolicy;
+    private final Ticker clockTicker;
+    private final CompressionCodec.Type codec;
+    private final ScheduledExecutorService scheduler;
+    private final boolean ownScheduler;
+    private final AtomicInteger nextStreamId;
+    private LogRecordSet.Writer recordSetWriter;
+
+    private DistributedLogMultiStreamWriter(List<String> streams,
+                                            DistributedLogClient client,
+                                            int bufferSize,
+                                            long flushIntervalMicros,
+                                            long requestTimeoutMs,
+                                            int firstSpecultiveTimeoutMs,
+                                            int maxSpeculativeTimeoutMs,
+                                            float speculativeBackoffMultiplier,
+                                            CompressionCodec.Type codec,
+                                            Ticker clockTicker,
+                                            ScheduledExecutorService scheduler) {
+        this.streams = Lists.newArrayList(streams);
+        this.numStreams = this.streams.size();
+        this.client = client;
+        this.bufferSize = bufferSize;
+        this.requestTimeoutMs = requestTimeoutMs;
+        this.codec = codec;
+        this.clockTicker = clockTicker;
+        if (null == scheduler) {
+            this.scheduler = Executors.newSingleThreadScheduledExecutor(
+                    new ThreadFactoryBuilder()
+                            .setDaemon(true)
+                            .setNameFormat("MultiStreamWriterFlushThread-%d")
+                            .build());
+            this.ownScheduler = true;
+        } else {
+            this.scheduler = scheduler;
+            this.ownScheduler = false;
+        }
+        this.speculativePolicy = new DefaultSpeculativeRequestExecutionPolicy(
+                firstSpecultiveTimeoutMs,
+                maxSpeculativeTimeoutMs,
+                speculativeBackoffMultiplier);
+        // shuffle the streams
+        Collections.shuffle(this.streams);
+        this.nextStreamId = new AtomicInteger(0);
+        this.recordSetWriter = newRecordSetWriter();
+
+        if (flushIntervalMicros > 0) {
+            this.scheduler.scheduleAtFixedRate(
+                    this,
+                    flushIntervalMicros,
+                    flushIntervalMicros,
+                    TimeUnit.MICROSECONDS);
+        }
+    }
+
+    String getStream(int streamId) {
+        return streams.get(streamId);
+    }
+
+    synchronized LogRecordSet.Writer getLogRecordSetWriter() {
+        return recordSetWriter;
+    }
+
+    private LogRecordSet.Writer newRecordSetWriter() {
+        return LogRecordSet.newWriter(
+                bufferSize,
+                codec);
+    }
+
+    public synchronized Future<DLSN> write(ByteBuffer buffer) {
+        int logRecordSize = buffer.remaining();
+        if (logRecordSize > MAX_LOGRECORD_SIZE) {
+            return Future.exception(new LogRecordTooLongException(
+                    "Log record of size " + logRecordSize + " written when only "
+                            + MAX_LOGRECORD_SIZE + " is allowed"));
+        }
+        // if exceed max number of bytes
+        if ((recordSetWriter.getNumBytes() + logRecordSize) > MAX_LOGRECORDSET_SIZE) {
+            flush();
+        }
+        Promise<DLSN> writePromise = new Promise<DLSN>();
+        try {
+            recordSetWriter.writeRecord(buffer, writePromise);
+        } catch (LogRecordTooLongException e) {
+            return Future.exception(e);
+        } catch (WriteException e) {
+            recordSetWriter.abortTransmit(e);
+            recordSetWriter = newRecordSetWriter();
+            return Future.exception(e);
+        }
+        if (recordSetWriter.getNumBytes() >= bufferSize) {
+            flush();
+        }
+        return writePromise;
+    }
+
+    @Override
+    public void run() {
+        flush();
+    }
+
+    private void flush() {
+        LogRecordSet.Writer recordSetToFlush;
+        synchronized (this) {
+            if (recordSetWriter.getNumRecords() == 0) {
+                return;
+            }
+            recordSetToFlush = recordSetWriter;
+            recordSetWriter = newRecordSetWriter();
+        }
+        transmit(recordSetToFlush);
+    }
+
+    private void transmit(LogRecordSet.Writer recordSetToFlush) {
+        PendingWriteRequest writeRequest =
+                new PendingWriteRequest(recordSetToFlush);
+        this.speculativePolicy.initiateSpeculativeRequest(scheduler, writeRequest);
+    }
+
+    public void close() {
+        if (ownScheduler) {
+            this.scheduler.shutdown();
+        }
+    }
+
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/MonitorServiceClient.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/MonitorServiceClient.java
new file mode 100644
index 0000000..ed6269b
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/MonitorServiceClient.java
@@ -0,0 +1,68 @@
+/**
+ * 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.client.monitor;
+
+import com.twitter.util.Future;
+import java.net.SocketAddress;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Interface for distributedlog monitor service.
+ */
+public interface MonitorServiceClient {
+
+    /**
+     * Check a given stream.
+     *
+     * @param stream
+     *          stream.
+     * @return check result.
+     */
+    Future<Void> check(String stream);
+
+    /**
+     * Send heartbeat to the stream and its readers.
+     *
+     * @param stream
+     *          stream.
+     * @return check result.
+     */
+    Future<Void> heartbeat(String stream);
+
+    /**
+     * Get current ownership distribution from current monitor service view.
+     *
+     * @return current ownership distribution
+     */
+    Map<SocketAddress, Set<String>> getStreamOwnershipDistribution();
+
+    /**
+     * Enable/Disable accepting new stream on a given proxy.
+     *
+     * @param enabled
+     *          flag to enable/disable accepting new streams on a given proxy
+     * @return void
+     */
+    Future<Void> setAcceptNewStream(boolean enabled);
+
+    /**
+     * Close the client.
+     */
+    void close();
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/package-info.java
new file mode 100644
index 0000000..d7e2c94
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * DistributedLog Monitor Client.
+ */
+package org.apache.distributedlog.client.monitor;
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/OwnershipCache.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/OwnershipCache.java
new file mode 100644
index 0000000..f3c24ca
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/OwnershipCache.java
@@ -0,0 +1,235 @@
+/**
+ * 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.client.ownership;
+
+import com.google.common.collect.ImmutableMap;
+import org.apache.distributedlog.client.ClientConfig;
+import org.apache.distributedlog.client.stats.OwnershipStatsLogger;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.jboss.netty.util.Timeout;
+import org.jboss.netty.util.TimerTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Client Side Ownership Cache.
+ */
+public class OwnershipCache implements TimerTask {
+
+    private static final Logger logger = LoggerFactory.getLogger(OwnershipCache.class);
+
+    private final ConcurrentHashMap<String, SocketAddress> stream2Addresses =
+            new ConcurrentHashMap<String, SocketAddress>();
+    private final ConcurrentHashMap<SocketAddress, Set<String>> address2Streams =
+            new ConcurrentHashMap<SocketAddress, Set<String>>();
+    private final ClientConfig clientConfig;
+    private final HashedWheelTimer timer;
+
+    // Stats
+    private final OwnershipStatsLogger ownershipStatsLogger;
+
+    public OwnershipCache(ClientConfig clientConfig,
+                          HashedWheelTimer timer,
+                          StatsReceiver statsReceiver,
+                          StatsReceiver streamStatsReceiver) {
+        this.clientConfig = clientConfig;
+        this.timer = timer;
+        this.ownershipStatsLogger = new OwnershipStatsLogger(statsReceiver, streamStatsReceiver);
+        scheduleDumpOwnershipCache();
+    }
+
+    private void scheduleDumpOwnershipCache() {
+        if (clientConfig.isPeriodicDumpOwnershipCacheEnabled()
+            && clientConfig.getPeriodicDumpOwnershipCacheIntervalMs() > 0) {
+            timer.newTimeout(this, clientConfig.getPeriodicDumpOwnershipCacheIntervalMs(),
+                    TimeUnit.MILLISECONDS);
+        }
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        if (timeout.isCancelled()) {
+            return;
+        }
+        logger.info("Ownership cache : {} streams cached, {} hosts cached",
+                stream2Addresses.size(), address2Streams.size());
+        logger.info("Cached streams : {}", stream2Addresses);
+        scheduleDumpOwnershipCache();
+    }
+
+    public OwnershipStatsLogger getOwnershipStatsLogger() {
+        return ownershipStatsLogger;
+    }
+
+    /**
+     * Update ownership of <i>stream</i> to <i>addr</i>.
+     *
+     * @param stream
+     *          Stream Name.
+     * @param addr
+     *          Owner Address.
+     * @return true if owner is updated
+     */
+    public boolean updateOwner(String stream, SocketAddress addr) {
+        // update ownership
+        SocketAddress oldAddr = stream2Addresses.putIfAbsent(stream, addr);
+        if (null != oldAddr && oldAddr.equals(addr)) {
+            return true;
+        }
+        if (null != oldAddr) {
+            if (stream2Addresses.replace(stream, oldAddr, addr)) {
+                // Store the relevant mappings for this topic and host combination
+                logger.info("Storing ownership for stream : {}, old host : {}, new host : {}.",
+                        new Object[] { stream, oldAddr, addr });
+                StringBuilder sb = new StringBuilder();
+                sb.append("Ownership changed '")
+                  .append(oldAddr).append("' -> '").append(addr).append("'");
+                removeOwnerFromStream(stream, oldAddr, sb.toString());
+
+                // update stats
+                ownershipStatsLogger.onRemove(stream);
+                ownershipStatsLogger.onAdd(stream);
+            } else {
+                logger.warn("Ownership of stream : {} has been changed from {} to {} when storing host : {}.",
+                        new Object[] { stream, oldAddr, stream2Addresses.get(stream), addr });
+                return false;
+            }
+        } else {
+            logger.info("Storing ownership for stream : {}, host : {}.", stream, addr);
+            // update stats
+            ownershipStatsLogger.onAdd(stream);
+        }
+
+        Set<String> streamsForHost = address2Streams.get(addr);
+        if (null == streamsForHost) {
+            Set<String> newStreamsForHost = new HashSet<String>();
+            streamsForHost = address2Streams.putIfAbsent(addr, newStreamsForHost);
+            if (null == streamsForHost) {
+                streamsForHost = newStreamsForHost;
+            }
+        }
+        synchronized (streamsForHost) {
+            // check whether the ownership changed, since it might happend after replace succeed
+            if (addr.equals(stream2Addresses.get(stream))) {
+                streamsForHost.add(stream);
+            }
+        }
+        return true;
+    }
+
+    /**
+     * Get the cached owner for stream <code>stream</code>.
+     *
+     * @param stream
+     *          stream to lookup ownership
+     * @return owner's address
+     */
+    public SocketAddress getOwner(String stream) {
+        SocketAddress address = stream2Addresses.get(stream);
+        if (null == address) {
+            ownershipStatsLogger.onMiss(stream);
+        } else {
+            ownershipStatsLogger.onHit(stream);
+        }
+        return address;
+    }
+
+    /**
+     * Remove the owner <code>addr</code> from <code>stream</code> for a given <code>reason</code>.
+     *
+     * @param stream stream name
+     * @param addr owner address
+     * @param reason reason to remove ownership
+     */
+    public void removeOwnerFromStream(String stream, SocketAddress addr, String reason) {
+        if (stream2Addresses.remove(stream, addr)) {
+            logger.info("Removed stream to host mapping for (stream: {} -> host: {}) : reason = '{}'.",
+                    new Object[] { stream, addr, reason });
+        }
+        Set<String> streamsForHost = address2Streams.get(addr);
+        if (null != streamsForHost) {
+            synchronized (streamsForHost) {
+                if (streamsForHost.remove(stream)) {
+                    logger.info("Removed stream ({}) from host {} : reason = '{}'.",
+                            new Object[] { stream, addr, reason });
+                    if (streamsForHost.isEmpty()) {
+                        address2Streams.remove(addr, streamsForHost);
+                    }
+                    ownershipStatsLogger.onRemove(stream);
+                }
+            }
+        }
+    }
+
+    /**
+     * Remove all streams from host <code>addr</code>.
+     *
+     * @param addr
+     *          host to remove ownerships
+     */
+    public void removeAllStreamsFromOwner(SocketAddress addr) {
+        logger.info("Remove streams mapping for host {}", addr);
+        Set<String> streamsForHost = address2Streams.get(addr);
+        if (null != streamsForHost) {
+            synchronized (streamsForHost) {
+                for (String s : streamsForHost) {
+                    if (stream2Addresses.remove(s, addr)) {
+                        logger.info("Removing mapping for stream : {} from host : {}", s, addr);
+                        ownershipStatsLogger.onRemove(s);
+                    }
+                }
+                address2Streams.remove(addr, streamsForHost);
+            }
+        }
+    }
+
+    /**
+     * Get the number cached streams.
+     *
+     * @return number cached streams.
+     */
+    public int getNumCachedStreams() {
+        return stream2Addresses.size();
+    }
+
+    /**
+     * Get the stream ownership distribution across proxies.
+     *
+     * @return stream ownership distribution
+     */
+    public Map<SocketAddress, Set<String>> getStreamOwnershipDistribution() {
+        return ImmutableMap.copyOf(address2Streams);
+    }
+
+    /**
+     * Get the stream ownership mapping.
+     *
+     * @return stream ownership mapping.
+     */
+    public Map<String, SocketAddress> getStreamOwnerMapping() {
+        return stream2Addresses;
+    }
+
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/package-info.java
new file mode 100644
index 0000000..486bd6f
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Utils for managing ownership at client side.
+ */
+package org.apache.distributedlog.client.ownership;
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/package-info.java
new file mode 100644
index 0000000..d22b0da
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * DistributedLog Client.
+ */
+package org.apache.distributedlog.client;
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ClusterClient.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ClusterClient.java
new file mode 100644
index 0000000..9b5c7f6
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ClusterClient.java
@@ -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.
+ */
+package org.apache.distributedlog.client.proxy;
+
+import org.apache.distributedlog.thrift.service.DistributedLogService;
+import com.twitter.finagle.Service;
+import com.twitter.finagle.thrift.ThriftClientRequest;
+import com.twitter.util.Future;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Cluster client.
+ */
+public class ClusterClient {
+
+    private final Service<ThriftClientRequest, byte[]> client;
+    private final DistributedLogService.ServiceIface service;
+
+    public ClusterClient(Service<ThriftClientRequest, byte[]> client,
+                         DistributedLogService.ServiceIface service) {
+        this.client = client;
+        this.service = service;
+    }
+
+    public Service<ThriftClientRequest, byte[]> getClient() {
+        return client;
+    }
+
+    public DistributedLogService.ServiceIface getService() {
+        return service;
+    }
+
+    public Future<BoxedUnit> close() {
+        return client.close();
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/HostProvider.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/HostProvider.java
new file mode 100644
index 0000000..769cca8
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/HostProvider.java
@@ -0,0 +1,35 @@
+/**
+ * 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.client.proxy;
+
+import java.net.SocketAddress;
+import java.util.Set;
+
+/**
+ * Provider to provider list of hosts for handshaking.
+ */
+public interface HostProvider {
+
+    /**
+     * Get the list of hosts for handshaking.
+     *
+     * @return list of hosts for handshaking.
+     */
+    Set<SocketAddress> getHosts();
+
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClient.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClient.java
new file mode 100644
index 0000000..6ef1d8e
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClient.java
@@ -0,0 +1,165 @@
+/**
+ * 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.client.proxy;
+
+import org.apache.distributedlog.client.ClientConfig;
+import org.apache.distributedlog.client.stats.ClientStats;
+import org.apache.distributedlog.thrift.service.DistributedLogService;
+import com.twitter.finagle.Service;
+import com.twitter.finagle.ThriftMux;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.thrift.ClientId;
+import com.twitter.finagle.thrift.ThriftClientFramedCodec;
+import com.twitter.finagle.thrift.ThriftClientRequest;
+import com.twitter.util.Duration;
+import com.twitter.util.Future;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import scala.Option;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Client talks to a single proxy.
+ */
+public class ProxyClient {
+
+  /**
+   * Builder to build a proxy client talking to given host <code>address</code>.
+   */
+  public interface Builder {
+        /**
+         * Build a proxy client to <code>address</code>.
+         *
+         * @param address
+         *          proxy address
+         * @return proxy client
+         */
+        ProxyClient build(SocketAddress address);
+    }
+
+    public static Builder newBuilder(String clientName,
+                                     ClientId clientId,
+                                     ClientBuilder clientBuilder,
+                                     ClientConfig clientConfig,
+                                     ClientStats clientStats) {
+        return new DefaultBuilder(clientName, clientId, clientBuilder, clientConfig, clientStats);
+    }
+
+    /**
+     * Default Builder for {@link ProxyClient}.
+     */
+    public static class DefaultBuilder implements Builder {
+
+        private final String clientName;
+        private final ClientId clientId;
+        private final ClientBuilder clientBuilder;
+        private final ClientStats clientStats;
+
+        private DefaultBuilder(String clientName,
+                               ClientId clientId,
+                               ClientBuilder clientBuilder,
+                               ClientConfig clientConfig,
+                               ClientStats clientStats) {
+            this.clientName = clientName;
+            this.clientId = clientId;
+            this.clientStats = clientStats;
+            // client builder
+            ClientBuilder builder = setDefaultSettings(
+                    null == clientBuilder ? getDefaultClientBuilder(clientConfig) : clientBuilder);
+            this.clientBuilder = configureThriftMux(builder, clientId, clientConfig);
+        }
+
+        @SuppressWarnings("unchecked")
+        private ClientBuilder configureThriftMux(ClientBuilder builder,
+                                                 ClientId clientId,
+                                                 ClientConfig clientConfig) {
+            if (clientConfig.getThriftMux()) {
+                return builder.stack(ThriftMux.client().withClientId(clientId));
+            } else {
+                return builder.codec(ThriftClientFramedCodec.apply(Option.apply(clientId)));
+            }
+        }
+
+        private ClientBuilder getDefaultClientBuilder(ClientConfig clientConfig) {
+            ClientBuilder builder = ClientBuilder.get()
+                .tcpConnectTimeout(Duration.fromMilliseconds(200))
+                .connectTimeout(Duration.fromMilliseconds(200))
+                .requestTimeout(Duration.fromSeconds(1));
+            if (!clientConfig.getThriftMux()) {
+                builder = builder.hostConnectionLimit(1);
+            }
+            return builder;
+        }
+
+        @SuppressWarnings("unchecked")
+        private ClientBuilder setDefaultSettings(ClientBuilder builder) {
+            return builder.name(clientName)
+                   .failFast(false)
+                   .noFailureAccrual()
+                   // disable retries on finagle client builder, as there is only one host per finagle client
+                   // we should throw exception immediately on first failure, so DL client could quickly detect
+                   // failures and retry other proxies.
+                   .retries(1)
+                   .keepAlive(true);
+        }
+
+        @Override
+        @SuppressWarnings("unchecked")
+        public ProxyClient build(SocketAddress address) {
+            Service<ThriftClientRequest, byte[]> client =
+                ClientBuilder.safeBuildFactory(
+                        clientBuilder
+                                .hosts((InetSocketAddress) address)
+                                .reportTo(clientStats.getFinagleStatsReceiver(address))
+                ).toService();
+            DistributedLogService.ServiceIface service =
+                    new DistributedLogService.ServiceToClient(client, new TBinaryProtocol.Factory());
+            return new ProxyClient(address, client, service);
+        }
+
+    }
+
+    private final SocketAddress address;
+    private final Service<ThriftClientRequest, byte[]> client;
+    private final DistributedLogService.ServiceIface service;
+
+    protected ProxyClient(SocketAddress address,
+                          Service<ThriftClientRequest, byte[]> client,
+                          DistributedLogService.ServiceIface service) {
+        this.address = address;
+        this.client  = client;
+        this.service = service;
+    }
+
+    public SocketAddress getAddress() {
+        return address;
+    }
+
+    public Service<ThriftClientRequest, byte[]> getClient() {
+        return client;
+    }
+
+    public DistributedLogService.ServiceIface getService() {
+        return service;
+    }
+
+    public Future<BoxedUnit> close() {
+        return client.close();
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClientManager.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClientManager.java
new file mode 100644
index 0000000..17b70be
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClientManager.java
@@ -0,0 +1,362 @@
+/**
+ * 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.client.proxy;
+
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.ImmutableMap;
+import org.apache.distributedlog.client.ClientConfig;
+import org.apache.distributedlog.client.stats.ClientStats;
+import org.apache.distributedlog.client.stats.OpStats;
+import org.apache.distributedlog.thrift.service.ClientInfo;
+import org.apache.distributedlog.thrift.service.ServerInfo;
+import com.twitter.util.FutureEventListener;
+import java.net.SocketAddress;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.jboss.netty.util.Timeout;
+import org.jboss.netty.util.TimerTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Manager manages clients (channels) to proxies.
+ */
+public class ProxyClientManager implements TimerTask {
+
+    private static final Logger logger = LoggerFactory.getLogger(ProxyClientManager.class);
+
+    private final ClientConfig clientConfig;
+    private final ProxyClient.Builder clientBuilder;
+    private final HashedWheelTimer timer;
+    private final HostProvider hostProvider;
+    private volatile Timeout periodicHandshakeTask;
+    private final ConcurrentHashMap<SocketAddress, ProxyClient> address2Services =
+            new ConcurrentHashMap<SocketAddress, ProxyClient>();
+    private final CopyOnWriteArraySet<ProxyListener> proxyListeners =
+            new CopyOnWriteArraySet<ProxyListener>();
+    private volatile boolean closed = false;
+    private volatile boolean periodicHandshakeEnabled = true;
+    private final Stopwatch lastOwnershipSyncStopwatch;
+
+    private final OpStats handshakeStats;
+
+    public ProxyClientManager(ClientConfig clientConfig,
+                              ProxyClient.Builder clientBuilder,
+                              HashedWheelTimer timer,
+                              HostProvider hostProvider,
+                              ClientStats clientStats) {
+        this.clientConfig = clientConfig;
+        this.clientBuilder = clientBuilder;
+        this.timer = timer;
+        this.hostProvider = hostProvider;
+        this.handshakeStats = clientStats.getOpStats("handshake");
+        scheduleHandshake();
+        this.lastOwnershipSyncStopwatch = Stopwatch.createStarted();
+    }
+
+    private void scheduleHandshake() {
+        if (clientConfig.getPeriodicHandshakeIntervalMs() > 0) {
+            periodicHandshakeTask = timer.newTimeout(this,
+                    clientConfig.getPeriodicHandshakeIntervalMs(), TimeUnit.MILLISECONDS);
+        }
+    }
+
+    void setPeriodicHandshakeEnabled(boolean enabled) {
+        this.periodicHandshakeEnabled = enabled;
+    }
+
+    @Override
+    public void run(Timeout timeout) throws Exception {
+        if (timeout.isCancelled() || closed) {
+            return;
+        }
+        if (periodicHandshakeEnabled) {
+            final boolean syncOwnerships = lastOwnershipSyncStopwatch.elapsed(TimeUnit.MILLISECONDS)
+                >= clientConfig.getPeriodicOwnershipSyncIntervalMs();
+
+            final Set<SocketAddress> hostsSnapshot = hostProvider.getHosts();
+            final AtomicInteger numHosts = new AtomicInteger(hostsSnapshot.size());
+            final AtomicInteger numStreams = new AtomicInteger(0);
+            final AtomicInteger numSuccesses = new AtomicInteger(0);
+            final AtomicInteger numFailures = new AtomicInteger(0);
+            final ConcurrentMap<SocketAddress, Integer> streamDistributions =
+                    new ConcurrentHashMap<SocketAddress, Integer>();
+            final Stopwatch stopwatch = Stopwatch.createStarted();
+            for (SocketAddress host : hostsSnapshot) {
+                final SocketAddress address = host;
+                final ProxyClient client = getClient(address);
+                handshake(address, client, new FutureEventListener<ServerInfo>() {
+                    @Override
+                    public void onSuccess(ServerInfo serverInfo) {
+                        numStreams.addAndGet(serverInfo.getOwnershipsSize());
+                        numSuccesses.incrementAndGet();
+                        notifyHandshakeSuccess(address, client, serverInfo, false, stopwatch);
+                        if (clientConfig.isHandshakeTracingEnabled()) {
+                            streamDistributions.putIfAbsent(address, serverInfo.getOwnershipsSize());
+                        }
+                        complete();
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        numFailures.incrementAndGet();
+                        notifyHandshakeFailure(address, client, cause, stopwatch);
+                        complete();
+                    }
+
+                    private void complete() {
+                        if (0 == numHosts.decrementAndGet()) {
+                            if (syncOwnerships) {
+                                logger.info("Periodic handshaked with {} hosts : {} streams returned,"
+                                    + " {} hosts succeeded, {} hosts failed",
+                                    new Object[] {
+                                        hostsSnapshot.size(),
+                                        numStreams.get(),
+                                        numSuccesses.get(),
+                                        numFailures.get()});
+                                if (clientConfig.isHandshakeTracingEnabled()) {
+                                    logger.info("Periodic handshaked stream distribution : {}", streamDistributions);
+                                }
+                            }
+                        }
+                    }
+                }, false, syncOwnerships);
+            }
+
+            if (syncOwnerships) {
+                lastOwnershipSyncStopwatch.reset().start();
+            }
+        }
+        scheduleHandshake();
+    }
+
+    /**
+     * Register a proxy <code>listener</code> on proxy related changes.
+     *
+     * @param listener
+     *          proxy listener
+     */
+    public void registerProxyListener(ProxyListener listener) {
+        proxyListeners.add(listener);
+    }
+
+    private void notifyHandshakeSuccess(SocketAddress address,
+                                        ProxyClient client,
+                                        ServerInfo serverInfo,
+                                        boolean logging,
+                                        Stopwatch stopwatch) {
+        if (logging) {
+            if (null != serverInfo && serverInfo.isSetOwnerships()) {
+                logger.info("Handshaked with {} : {} ownerships returned.",
+                        address, serverInfo.getOwnerships().size());
+            } else {
+                logger.info("Handshaked with {} : no ownerships returned", address);
+            }
+        }
+        handshakeStats.completeRequest(address, stopwatch.elapsed(TimeUnit.MICROSECONDS), 1);
+        for (ProxyListener listener : proxyListeners) {
+            listener.onHandshakeSuccess(address, client, serverInfo);
+        }
+    }
+
+    private void notifyHandshakeFailure(SocketAddress address,
+                                        ProxyClient client,
+                                        Throwable cause,
+                                        Stopwatch stopwatch) {
+        handshakeStats.failRequest(address, stopwatch.elapsed(TimeUnit.MICROSECONDS), 1);
+        for (ProxyListener listener : proxyListeners) {
+            listener.onHandshakeFailure(address, client, cause);
+        }
+    }
+
+    /**
+     * Retrieve a client to proxy <code>address</code>.
+     *
+     * @param address
+     *          proxy address
+     * @return proxy client
+     */
+    public ProxyClient getClient(final SocketAddress address) {
+        ProxyClient sc = address2Services.get(address);
+        if (null != sc) {
+            return sc;
+        }
+        return createClient(address);
+    }
+
+    /**
+     * Remove the client to proxy <code>address</code>.
+     *
+     * @param address
+     *          proxy address
+     */
+    public void removeClient(SocketAddress address) {
+        ProxyClient sc = address2Services.remove(address);
+        if (null != sc) {
+            logger.info("Removed host {}.", address);
+            sc.close();
+        }
+    }
+
+    /**
+     * Remove the client <code>sc</code> to proxy <code>address</code>.
+     *
+     * @param address
+     *          proxy address
+     * @param sc
+     *          proxy client
+     */
+    public void removeClient(SocketAddress address, ProxyClient sc) {
+        if (address2Services.remove(address, sc)) {
+            logger.info("Remove client {} to host {}.", sc, address);
+            sc.close();
+        }
+    }
+
+    /**
+     * Create a client to proxy <code>address</code>.
+     *
+     * @param address
+     *          proxy address
+     * @return proxy client
+     */
+    public ProxyClient createClient(final SocketAddress address) {
+        final ProxyClient sc = clientBuilder.build(address);
+        ProxyClient oldSC = address2Services.putIfAbsent(address, sc);
+        if (null != oldSC) {
+            sc.close();
+            return oldSC;
+        } else {
+            final Stopwatch stopwatch = Stopwatch.createStarted();
+            FutureEventListener<ServerInfo> listener = new FutureEventListener<ServerInfo>() {
+                @Override
+                public void onSuccess(ServerInfo serverInfo) {
+                    notifyHandshakeSuccess(address, sc, serverInfo, true, stopwatch);
+                }
+                @Override
+                public void onFailure(Throwable cause) {
+                    notifyHandshakeFailure(address, sc, cause, stopwatch);
+                }
+            };
+            // send a ping messaging after creating connections.
+            handshake(address, sc, listener, true, true);
+            return sc;
+        }
+    }
+
+    /**
+     * Handshake with a given proxy.
+     *
+     * @param address
+     *          proxy address
+     * @param sc
+     *          proxy client
+     * @param listener
+     *          listener on handshake result
+     */
+    private void handshake(SocketAddress address,
+                           ProxyClient sc,
+                           FutureEventListener<ServerInfo> listener,
+                           boolean logging,
+                           boolean getOwnerships) {
+        if (clientConfig.getHandshakeWithClientInfo()) {
+            ClientInfo clientInfo = new ClientInfo();
+            clientInfo.setGetOwnerships(getOwnerships);
+            clientInfo.setStreamNameRegex(clientConfig.getStreamNameRegex());
+            if (logging) {
+                logger.info("Handshaking with {} : {}", address, clientInfo);
+            }
+            sc.getService().handshakeWithClientInfo(clientInfo)
+                    .addEventListener(listener);
+        } else {
+            if (logging) {
+                logger.info("Handshaking with {}", address);
+            }
+            sc.getService().handshake().addEventListener(listener);
+        }
+    }
+
+    /**
+     * Handshake with all proxies.
+     *
+     * <p>NOTE: this is a synchronous call.
+     */
+    public void handshake() {
+        Set<SocketAddress> hostsSnapshot = hostProvider.getHosts();
+        logger.info("Handshaking with {} hosts.", hostsSnapshot.size());
+        final CountDownLatch latch = new CountDownLatch(hostsSnapshot.size());
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+        for (SocketAddress host: hostsSnapshot) {
+            final SocketAddress address = host;
+            final ProxyClient client = getClient(address);
+            handshake(address, client, new FutureEventListener<ServerInfo>() {
+                @Override
+                public void onSuccess(ServerInfo serverInfo) {
+                    notifyHandshakeSuccess(address, client, serverInfo, true, stopwatch);
+                    latch.countDown();
+                }
+                @Override
+                public void onFailure(Throwable cause) {
+                    notifyHandshakeFailure(address, client, cause, stopwatch);
+                    latch.countDown();
+                }
+            }, true, true);
+        }
+        try {
+            latch.await(1, TimeUnit.MINUTES);
+        } catch (InterruptedException e) {
+            logger.warn("Interrupted on handshaking with servers : ", e);
+        }
+    }
+
+    /**
+     * Return number of proxies managed by client manager.
+     *
+     * @return number of proxies managed by client manager.
+     */
+    public int getNumProxies() {
+        return address2Services.size();
+    }
+
+    /**
+     * Return all clients.
+     *
+     * @return all clients.
+     */
+    public Map<SocketAddress, ProxyClient> getAllClients() {
+        return ImmutableMap.copyOf(address2Services);
+    }
+
+    public void close() {
+        closed = true;
+        Timeout task = periodicHandshakeTask;
+        if (null != task) {
+            task.cancel();
+        }
+        for (ProxyClient sc : address2Services.values()) {
+            sc.close();
+        }
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyListener.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyListener.java
new file mode 100644
index 0000000..0a6b076
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyListener.java
@@ -0,0 +1,50 @@
+/**
+ * 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.client.proxy;
+
+import org.apache.distributedlog.thrift.service.ServerInfo;
+import java.net.SocketAddress;
+
+/**
+ * Listener on server changes.
+ */
+public interface ProxyListener {
+    /**
+     * When a proxy's server info changed, it would be notified.
+     *
+     * @param address
+     *          proxy address
+     * @param client
+     *          proxy client that executes handshaking
+     * @param serverInfo
+     *          proxy's server info
+     */
+    void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo);
+
+    /**
+     * Failed to handshake with a proxy.
+     *
+     * @param address
+     *          proxy address
+     * @param client
+     *          proxy client
+     * @param cause
+     *          failure reason
+     */
+    void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause);
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/package-info.java
new file mode 100644
index 0000000..4161afb
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Clients that interact with individual proxies.
+ */
+package org.apache.distributedlog.client.proxy;
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/DefaultRegionResolver.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/DefaultRegionResolver.java
new file mode 100644
index 0000000..2ac5be3
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/DefaultRegionResolver.java
@@ -0,0 +1,85 @@
+/**
+ * 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.client.resolver;
+
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Default implementation of {@link RegionResolver}.
+ */
+public class DefaultRegionResolver implements RegionResolver {
+
+    private static final String DEFAULT_REGION = "default-region";
+
+    private final Map<SocketAddress, String> regionOverrides =
+            new HashMap<SocketAddress, String>();
+    private final ConcurrentMap<SocketAddress, String> regionMap =
+            new ConcurrentHashMap<SocketAddress, String>();
+
+    public DefaultRegionResolver() {
+    }
+
+    public DefaultRegionResolver(Map<SocketAddress, String> regionOverrides) {
+        this.regionOverrides.putAll(regionOverrides);
+    }
+
+    @Override
+    public String resolveRegion(SocketAddress address) {
+        String region = regionMap.get(address);
+        if (null == region) {
+            region = doResolveRegion(address);
+            regionMap.put(address, region);
+        }
+        return region;
+    }
+
+    private String doResolveRegion(SocketAddress address) {
+        String region = regionOverrides.get(address);
+        if (null != region) {
+            return region;
+        }
+
+        String domainName;
+        if (address instanceof InetSocketAddress) {
+            InetSocketAddress iAddr = (InetSocketAddress) address;
+            domainName = iAddr.getHostName();
+        } else {
+            domainName = address.toString();
+        }
+        String[] parts = domainName.split("\\.");
+        if (parts.length <= 0) {
+            return DEFAULT_REGION;
+        }
+        String hostName = parts[0];
+        String[] labels = hostName.split("-");
+        if (labels.length != 4) {
+            return DEFAULT_REGION;
+        }
+        return labels[0];
+    }
+
+    @Override
+    public void removeCachedHost(SocketAddress address) {
+        regionMap.remove(address);
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/RegionResolver.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/RegionResolver.java
new file mode 100644
index 0000000..023799c
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/RegionResolver.java
@@ -0,0 +1,43 @@
+/**
+ * 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.client.resolver;
+
+import java.net.SocketAddress;
+
+/**
+ * Resolve address to region.
+ */
+public interface RegionResolver {
+
+    /**
+     * Resolve address to region.
+     *
+     * @param address
+     *          socket address
+     * @return region
+     */
+    String resolveRegion(SocketAddress address);
+
+    /**
+     * Remove cached host.
+     *
+     * @param address
+     *          socket address.
+     */
+    void removeCachedHost(SocketAddress address);
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/package-info.java
new file mode 100644
index 0000000..81cda2f
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Resolver to resolve network addresses.
+ */
+package org.apache.distributedlog.client.resolver;
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ConsistentHashRoutingService.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ConsistentHashRoutingService.java
new file mode 100644
index 0000000..666fa31
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ConsistentHashRoutingService.java
@@ -0,0 +1,500 @@
+/**
+ * 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.client.routing;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.MapDifference;
+import com.google.common.collect.Maps;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.service.DLSocketAddress;
+import com.twitter.finagle.ChannelException;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.Counter;
+import com.twitter.finagle.stats.Gauge;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.util.Function0;
+import java.net.SocketAddress;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.commons.lang3.tuple.Pair;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.jboss.netty.util.Timeout;
+import org.jboss.netty.util.TimerTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.collection.Seq;
+
+/**
+ * Consistent Hashing Based {@link RoutingService}.
+ */
+public class ConsistentHashRoutingService extends ServerSetRoutingService {
+
+    private static final Logger logger = LoggerFactory.getLogger(ConsistentHashRoutingService.class);
+
+    @Deprecated
+    public static ConsistentHashRoutingService of(ServerSetWatcher serverSetWatcher, int numReplicas) {
+        return new ConsistentHashRoutingService(serverSetWatcher, numReplicas, 300, NullStatsReceiver.get());
+    }
+
+    /**
+     * Builder helper class to build a consistent hash bashed {@link RoutingService}.
+     *
+     * @return builder to build a consistent hash based {@link RoutingService}.
+     */
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    /**
+     * Builder for building consistent hash based routing service.
+     */
+    public static class Builder implements RoutingService.Builder {
+
+        private ServerSet serverSet;
+        private boolean resolveFromName = false;
+        private int numReplicas;
+        private int blackoutSeconds = 300;
+        private StatsReceiver statsReceiver = NullStatsReceiver.get();
+
+        private Builder() {}
+
+        public Builder serverSet(ServerSet serverSet) {
+            this.serverSet = serverSet;
+            return this;
+        }
+
+        public Builder resolveFromName(boolean enabled) {
+            this.resolveFromName = enabled;
+            return this;
+        }
+
+        public Builder numReplicas(int numReplicas) {
+            this.numReplicas = numReplicas;
+            return this;
+        }
+
+        public Builder blackoutSeconds(int seconds) {
+            this.blackoutSeconds = seconds;
+            return this;
+        }
+
+        public Builder statsReceiver(StatsReceiver statsReceiver) {
+            this.statsReceiver = statsReceiver;
+            return this;
+        }
+
+        @Override
+        public RoutingService build() {
+            checkNotNull(serverSet, "No serverset provided.");
+            checkNotNull(statsReceiver, "No stats receiver provided.");
+            checkArgument(numReplicas > 0, "Invalid number of replicas : " + numReplicas);
+            return new ConsistentHashRoutingService(new TwitterServerSetWatcher(serverSet, resolveFromName),
+                numReplicas, blackoutSeconds, statsReceiver);
+        }
+    }
+
+    static class ConsistentHash {
+        private final HashFunction hashFunction;
+        private final int numOfReplicas;
+        private final SortedMap<Long, SocketAddress> circle;
+
+        // Stats
+        protected final Counter hostAddedCounter;
+        protected final Counter hostRemovedCounter;
+
+        ConsistentHash(HashFunction hashFunction,
+                       int numOfReplicas,
+                       StatsReceiver statsReceiver) {
+            this.hashFunction = hashFunction;
+            this.numOfReplicas = numOfReplicas;
+            this.circle = new TreeMap<Long, SocketAddress>();
+
+            this.hostAddedCounter = statsReceiver.counter0("adds");
+            this.hostRemovedCounter = statsReceiver.counter0("removes");
+        }
+
+        private String replicaName(int shardId, int replica, String address) {
+            if (shardId < 0) {
+                shardId = UNKNOWN_SHARD_ID;
+            }
+
+            StringBuilder sb = new StringBuilder(100);
+            sb.append("shard-");
+            sb.append(shardId);
+            sb.append('-');
+            sb.append(replica);
+            sb.append('-');
+            sb.append(address);
+
+            return sb.toString();
+        }
+
+        private Long replicaHash(int shardId, int replica, String address) {
+            return hashFunction.hashUnencodedChars(replicaName(shardId, replica, address)).asLong();
+        }
+
+        private Long replicaHash(int shardId, int replica, SocketAddress address) {
+            return replicaHash(shardId, replica, address.toString());
+        }
+
+        public synchronized void add(int shardId, SocketAddress address) {
+            String addressStr = address.toString();
+            for (int i = 0; i < numOfReplicas; i++) {
+                Long hash = replicaHash(shardId, i, addressStr);
+                circle.put(hash, address);
+            }
+            hostAddedCounter.incr();
+        }
+
+        public synchronized void remove(int shardId, SocketAddress address) {
+            for (int i = 0; i < numOfReplicas; i++) {
+                long hash = replicaHash(shardId, i, address);
+                SocketAddress oldAddress = circle.get(hash);
+                if (null != oldAddress && oldAddress.equals(address)) {
+                    circle.remove(hash);
+                }
+            }
+            hostRemovedCounter.incr();
+        }
+
+        public SocketAddress get(String key, RoutingContext rContext) {
+            long hash = hashFunction.hashUnencodedChars(key).asLong();
+            return find(hash, rContext);
+        }
+
+        private synchronized SocketAddress find(long hash, RoutingContext rContext) {
+            if (circle.isEmpty()) {
+                return null;
+            }
+
+            Iterator<Map.Entry<Long, SocketAddress>> iterator =
+                    circle.tailMap(hash).entrySet().iterator();
+            while (iterator.hasNext()) {
+                Map.Entry<Long, SocketAddress> entry = iterator.next();
+                if (!rContext.isTriedHost(entry.getValue())) {
+                    return entry.getValue();
+                }
+            }
+            // the tail map has been checked
+            iterator = circle.headMap(hash).entrySet().iterator();
+            while (iterator.hasNext()) {
+                Map.Entry<Long, SocketAddress> entry = iterator.next();
+                if (!rContext.isTriedHost(entry.getValue())) {
+                    return entry.getValue();
+                }
+            }
+
+            return null;
+        }
+
+        private synchronized Pair<Long, SocketAddress> get(long hash) {
+            if (circle.isEmpty()) {
+                return null;
+            }
+
+            if (!circle.containsKey(hash)) {
+                SortedMap<Long, SocketAddress> tailMap = circle.tailMap(hash);
+                hash = tailMap.isEmpty() ? circle.firstKey() : tailMap.firstKey();
+            }
+            return Pair.of(hash, circle.get(hash));
+        }
+
+        synchronized void dumpHashRing() {
+            for (Map.Entry<Long, SocketAddress> entry : circle.entrySet()) {
+                logger.info(entry.getKey() + " : " + entry.getValue());
+            }
+        }
+
+    }
+
+    class BlackoutHost implements TimerTask {
+        final int shardId;
+        final SocketAddress address;
+
+        BlackoutHost(int shardId, SocketAddress address) {
+            this.shardId = shardId;
+            this.address = address;
+            numBlackoutHosts.incrementAndGet();
+        }
+
+        @Override
+        public void run(Timeout timeout) throws Exception {
+            numBlackoutHosts.decrementAndGet();
+            if (!timeout.isExpired()) {
+                return;
+            }
+            Set<SocketAddress> removedList = new HashSet<SocketAddress>();
+            boolean joined;
+            // add the shard back
+            synchronized (shardId2Address) {
+                SocketAddress curHost = shardId2Address.get(shardId);
+                if (null != curHost) {
+                    // there is already new shard joint, so drop the host.
+                    logger.info("Blackout Shard {} ({}) was already replaced by {} permanently.",
+                            new Object[] { shardId, address, curHost });
+                    joined = false;
+                } else {
+                    join(shardId, address, removedList);
+                    joined = true;
+                }
+            }
+            if (joined) {
+                for (RoutingListener listener : listeners) {
+                    listener.onServerJoin(address);
+                }
+            } else {
+                for (RoutingListener listener : listeners) {
+                    listener.onServerLeft(address);
+                }
+            }
+        }
+    }
+
+    protected final HashedWheelTimer hashedWheelTimer;
+    protected final HashFunction hashFunction = Hashing.md5();
+    protected final ConsistentHash circle;
+    protected final Map<Integer, SocketAddress> shardId2Address =
+            new HashMap<Integer, SocketAddress>();
+    protected final Map<SocketAddress, Integer> address2ShardId =
+            new HashMap<SocketAddress, Integer>();
+
+    // blackout period
+    protected final int blackoutSeconds;
+
+    // stats
+    protected final StatsReceiver statsReceiver;
+    protected final AtomicInteger numBlackoutHosts;
+    protected final Gauge numBlackoutHostsGauge;
+    protected final Gauge numHostsGauge;
+
+    private static final int UNKNOWN_SHARD_ID = -1;
+
+    ConsistentHashRoutingService(ServerSetWatcher serverSetWatcher,
+                                 int numReplicas,
+                                 int blackoutSeconds,
+                                 StatsReceiver statsReceiver) {
+        super(serverSetWatcher);
+        this.circle = new ConsistentHash(hashFunction, numReplicas, statsReceiver.scope("ring"));
+        this.hashedWheelTimer = new HashedWheelTimer(new ThreadFactoryBuilder()
+                .setNameFormat("ConsistentHashRoutingService-Timer-%d").build());
+        this.blackoutSeconds = blackoutSeconds;
+        // stats
+        this.statsReceiver = statsReceiver;
+        this.numBlackoutHosts = new AtomicInteger(0);
+        this.numBlackoutHostsGauge = this.statsReceiver.addGauge(gaugeName("num_blackout_hosts"),
+                new Function0<Object>() {
+                    @Override
+                    public Object apply() {
+                        return (float) numBlackoutHosts.get();
+                    }
+                });
+        this.numHostsGauge = this.statsReceiver.addGauge(gaugeName("num_hosts"),
+                new Function0<Object>() {
+                    @Override
+                    public Object apply() {
+                        return (float) address2ShardId.size();
+                    }
+                });
+    }
+
+    private static Seq<String> gaugeName(String name) {
+        return scala.collection.JavaConversions.asScalaBuffer(Arrays.asList(name)).toList();
+    }
+
+    @Override
+    public void startService() {
+        super.startService();
+        this.hashedWheelTimer.start();
+    }
+
+    @Override
+    public void stopService() {
+        this.hashedWheelTimer.stop();
+        super.stopService();
+    }
+
+    @Override
+    public Set<SocketAddress> getHosts() {
+        synchronized (shardId2Address) {
+            return ImmutableSet.copyOf(address2ShardId.keySet());
+        }
+    }
+
+    @Override
+    public SocketAddress getHost(String key, RoutingContext rContext)
+            throws NoBrokersAvailableException {
+        SocketAddress host = circle.get(key, rContext);
+        if (null != host) {
+            return host;
+        }
+        throw new NoBrokersAvailableException("No host found for " + key + ", routing context : " + rContext);
+    }
+
+    @Override
+    public void removeHost(SocketAddress host, Throwable reason) {
+        removeHostInternal(host, Optional.of(reason));
+    }
+
+    private void removeHostInternal(SocketAddress host, Optional<Throwable> reason) {
+        synchronized (shardId2Address) {
+            Integer shardId = address2ShardId.remove(host);
+            if (null != shardId) {
+                SocketAddress curHost = shardId2Address.get(shardId);
+                if (null != curHost && curHost.equals(host)) {
+                    shardId2Address.remove(shardId);
+                }
+                circle.remove(shardId, host);
+                if (reason.isPresent()) {
+                    if (reason.get() instanceof ChannelException) {
+                        logger.info("Shard {} ({}) left due to ChannelException, black it out for {} seconds"
+                            + " (message = {})",
+                            new Object[] { shardId, host, blackoutSeconds, reason.get().toString() });
+                        BlackoutHost blackoutHost = new BlackoutHost(shardId, host);
+                        hashedWheelTimer.newTimeout(blackoutHost, blackoutSeconds, TimeUnit.SECONDS);
+                    } else {
+                        logger.info("Shard {} ({}) left due to exception {}",
+                                new Object[] { shardId, host, reason.get().toString() });
+                    }
+                } else {
+                    logger.info("Shard {} ({}) left after server set change",
+                                shardId, host);
+                }
+            } else if (reason.isPresent()) {
+                logger.info("Node {} left due to exception {}", host, reason.get().toString());
+            } else {
+                logger.info("Node {} left after server set change", host);
+            }
+        }
+    }
+
+    /**
+     * The caller should synchronize on <i>shardId2Address</i>.
+     * @param shardId
+     *          Shard id of new host joined.
+     * @param newHost
+     *          New host joined.
+     * @param removedList
+     *          Old hosts to remove
+     */
+    private void join(int shardId, SocketAddress newHost, Set<SocketAddress> removedList) {
+        SocketAddress oldHost = shardId2Address.put(shardId, newHost);
+        if (null != oldHost) {
+            // remove the old host only when a new shard is kicked in to replace it.
+            address2ShardId.remove(oldHost);
+            circle.remove(shardId, oldHost);
+            removedList.add(oldHost);
+            logger.info("Shard {} ({}) left permanently.", shardId, oldHost);
+        }
+        address2ShardId.put(newHost, shardId);
+        circle.add(shardId, newHost);
+        logger.info("Shard {} ({}) joined to replace ({}).",
+                    new Object[] { shardId, newHost, oldHost });
+    }
+
+    @Override
+    protected synchronized void performServerSetChange(ImmutableSet<DLSocketAddress> serviceInstances) {
+        Set<SocketAddress> joinedList = new HashSet<SocketAddress>();
+        Set<SocketAddress> removedList = new HashSet<SocketAddress>();
+
+        Map<Integer, SocketAddress> newMap = new HashMap<Integer, SocketAddress>();
+        synchronized (shardId2Address) {
+            for (DLSocketAddress serviceInstance : serviceInstances) {
+                if (serviceInstance.getShard() >= 0) {
+                    newMap.put(serviceInstance.getShard(), serviceInstance.getSocketAddress());
+                } else {
+                    Integer shard = address2ShardId.get(serviceInstance.getSocketAddress());
+                    if (null == shard) {
+                        // Assign a random negative shardId
+                        int shardId;
+                        do {
+                            shardId = Math.min(-1 , (int) (Math.random() * Integer.MIN_VALUE));
+                        } while (null != shardId2Address.get(shardId));
+                        shard = shardId;
+                    }
+                    newMap.put(shard, serviceInstance.getSocketAddress());
+                }
+            }
+        }
+
+        Map<Integer, SocketAddress> left;
+        synchronized (shardId2Address) {
+            MapDifference<Integer, SocketAddress> difference =
+                    Maps.difference(shardId2Address, newMap);
+            left = difference.entriesOnlyOnLeft();
+            for (Map.Entry<Integer, SocketAddress> shardEntry : left.entrySet()) {
+                int shard = shardEntry.getKey();
+                if (shard >= 0) {
+                    SocketAddress host = shardId2Address.get(shard);
+                    if (null != host) {
+                        // we don't remove those hosts that just disappered on serverset proactively,
+                        // since it might be just because serverset become flaky
+                        // address2ShardId.remove(host);
+                        // circle.remove(shard, host);
+                        logger.info("Shard {} ({}) left temporarily.", shard, host);
+                    }
+                } else {
+                    // shard id is negative - they are resolved from finagle name, which instances don't have shard id
+                    // in this case, if they are removed from serverset, we removed them directly
+                    SocketAddress host = shardEntry.getValue();
+                    if (null != host) {
+                        removeHostInternal(host, Optional.<Throwable>absent());
+                        removedList.add(host);
+                    }
+                }
+            }
+            // we need to find if any shards are replacing old shards
+            for (Map.Entry<Integer, SocketAddress> shard : newMap.entrySet()) {
+                SocketAddress oldHost = shardId2Address.get(shard.getKey());
+                SocketAddress newHost = shard.getValue();
+                if (!newHost.equals(oldHost)) {
+                    join(shard.getKey(), newHost, removedList);
+                    joinedList.add(newHost);
+                }
+            }
+        }
+
+        for (SocketAddress addr : removedList) {
+            for (RoutingListener listener : listeners) {
+                listener.onServerLeft(addr);
+            }
+        }
+
+        for (SocketAddress addr : joinedList) {
+            for (RoutingListener listener : listeners) {
+                listener.onServerJoin(addr);
+            }
+        }
+    }
+
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/NameServerSet.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/NameServerSet.java
new file mode 100644
index 0000000..e51eb1e
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/NameServerSet.java
@@ -0,0 +1,263 @@
+/**
+ * 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.client.routing;
+
+import com.google.common.collect.ImmutableSet;
+import com.twitter.common.base.Command;
+import com.twitter.common.base.Commands;
+import com.twitter.common.zookeeper.Group;
+import com.twitter.common.zookeeper.ServerSet;
+import com.twitter.finagle.Addr;
+import com.twitter.finagle.Address;
+import com.twitter.finagle.Name;
+import com.twitter.finagle.Resolver$;
+import com.twitter.thrift.Endpoint;
+import com.twitter.thrift.ServiceInstance;
+import com.twitter.thrift.Status;
+import java.net.InetSocketAddress;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Finagle Name based {@link ServerSet} implementation.
+ */
+class NameServerSet implements ServerSet {
+
+    private static final Logger logger = LoggerFactory.getLogger(NameServerSet.class);
+
+    private volatile Set<HostChangeMonitor<ServiceInstance>> watchers =
+        new HashSet<HostChangeMonitor<ServiceInstance>>();
+    private volatile ImmutableSet<ServiceInstance> hostSet = ImmutableSet.of();
+    private AtomicBoolean resolutionPending = new AtomicBoolean(true);
+
+    public NameServerSet(String nameStr) {
+        Name name;
+        try {
+            name = Resolver$.MODULE$.eval(nameStr);
+        } catch (Exception exc) {
+            logger.error("Exception in Resolver.eval for name {}", nameStr, exc);
+            // Since this is called from various places that dont handle specific exceptions,
+            // we have no option than to throw a runtime exception to halt the control flow
+            // This should only happen in case of incorrect configuration. Having a log message
+            // would help identify the problem during tests
+            throw new RuntimeException(exc);
+        }
+        initialize(name);
+    }
+
+    public NameServerSet(Name name) {
+        initialize(name);
+    }
+
+    private void initialize(Name name) {
+        if (name instanceof TestName) {
+            ((TestName) name).changes(new AbstractFunction1<Addr, BoxedUnit>() {
+                @Override
+                public BoxedUnit apply(Addr varAddr) {
+                    return NameServerSet.this.respondToChanges(varAddr);
+                }
+            });
+        } else if (name instanceof Name.Bound) {
+            ((Name.Bound) name).addr().changes().respond(new AbstractFunction1<Addr, BoxedUnit>() {
+                @Override
+                public BoxedUnit apply(Addr varAddr) {
+                    return NameServerSet.this.respondToChanges(varAddr);
+                }
+            });
+        } else {
+            logger.error("NameServerSet only supports Name.Bound. While the resolved name {} was {}",
+                name, name.getClass());
+            throw new UnsupportedOperationException("NameServerSet only supports Name.Bound");
+        }
+    }
+
+    private ServiceInstance endpointAddressToServiceInstance(Address endpointAddress) {
+        if (endpointAddress instanceof Address.Inet) {
+            InetSocketAddress inetSocketAddress = ((Address.Inet) endpointAddress).addr();
+            Endpoint endpoint = new Endpoint(inetSocketAddress.getHostString(), inetSocketAddress.getPort());
+            HashMap<String, Endpoint> map = new HashMap<String, Endpoint>();
+            map.put("thrift", endpoint);
+            return new ServiceInstance(
+                endpoint,
+                map,
+                Status.ALIVE);
+        } else {
+            logger.error("We expect InetSocketAddress while the resolved address {} was {}",
+                        endpointAddress, endpointAddress.getClass());
+            throw new UnsupportedOperationException("invalid endpoint address: " + endpointAddress);
+        }
+    }
+
+
+    private BoxedUnit respondToChanges(Addr addr) {
+        ImmutableSet<ServiceInstance> oldHostSet = ImmutableSet.copyOf(hostSet);
+
+        ImmutableSet<ServiceInstance> newHostSet = oldHostSet;
+
+        if (addr instanceof Addr.Bound) {
+            scala.collection.immutable.Set<Address> endpointAddresses = ((Addr.Bound) addr).addrs();
+            scala.collection.Iterator<Address> endpointAddressesIterator = endpointAddresses.toIterator();
+            HashSet<ServiceInstance> serviceInstances = new HashSet<ServiceInstance>();
+            while (endpointAddressesIterator.hasNext()) {
+                serviceInstances.add(endpointAddressToServiceInstance(endpointAddressesIterator.next()));
+            }
+            newHostSet = ImmutableSet.copyOf(serviceInstances);
+
+        } else if (addr instanceof Addr.Failed) {
+            logger.error("Name resolution failed", ((Addr.Failed) addr).cause());
+            newHostSet = ImmutableSet.of();
+        } else if (addr.toString().equals("Pending")) {
+            logger.info("Name resolution pending");
+            newHostSet = oldHostSet;
+        } else if (addr.toString().equals("Neg")) {
+            newHostSet = ImmutableSet.of();
+        } else {
+            logger.error("Invalid Addr type: {}", addr.getClass().getName());
+            throw new UnsupportedOperationException("Invalid Addr type:" + addr.getClass().getName());
+        }
+
+        // Reference comparison is valid as the sets are immutable
+        if (oldHostSet != newHostSet) {
+            logger.info("NameServerSet updated: {} -> {}", hostSetToString(oldHostSet), hostSetToString(newHostSet));
+            resolutionPending.set(false);
+            hostSet = newHostSet;
+            synchronized (watchers) {
+                for (HostChangeMonitor<ServiceInstance> watcher: watchers) {
+                    watcher.onChange(newHostSet);
+                }
+            }
+
+        }
+
+        return BoxedUnit.UNIT;
+    }
+
+
+    private String hostSetToString(ImmutableSet<ServiceInstance> hostSet) {
+        StringBuilder result = new StringBuilder();
+        result.append("(");
+        for (ServiceInstance serviceInstance : hostSet) {
+            Endpoint endpoint = serviceInstance.getServiceEndpoint();
+            result.append(String.format(" %s:%d", endpoint.getHost(), endpoint.getPort()));
+        }
+        result.append(" )");
+
+        return result.toString();
+    }
+
+
+    /**
+     * Attempts to join a server set for this logical service group.
+     *
+     * @param endpoint the primary service endpoint
+     * @param additionalEndpoints and additional endpoints keyed by their logical name
+     * @param status the current service status
+     * @return an EndpointStatus object that allows the endpoint to adjust its status
+     * @throws Group.JoinException if there was a problem joining the server set
+     * @throws InterruptedException if interrupted while waiting to join the server set
+     * @deprecated The status field is deprecated. Please use {@link #join(java.net.InetSocketAddress, java.util.Map)}
+     */
+    @Override
+    public EndpointStatus join(InetSocketAddress endpoint,
+                               Map<String, InetSocketAddress> additionalEndpoints,
+                               Status status)
+            throws Group.JoinException, InterruptedException {
+        throw new UnsupportedOperationException("NameServerSet does not support join");
+    }
+
+    /**
+     * Attempts to join a server set for this logical service group.
+     *
+     * @param endpoint the primary service endpoint
+     * @param additionalEndpoints and additional endpoints keyed by their logical name
+     * @return an EndpointStatus object that allows the endpoint to adjust its status
+     * @throws Group.JoinException if there was a problem joining the server set
+     * @throws InterruptedException if interrupted while waiting to join the server set
+     */
+    @Override
+    public EndpointStatus join(InetSocketAddress endpoint, Map<String, InetSocketAddress> additionalEndpoints)
+            throws Group.JoinException, InterruptedException {
+        throw new UnsupportedOperationException("NameServerSet does not support join");
+    }
+
+    /**
+     * Attempts to join a server set for this logical service group.
+     *
+     * @param endpoint the primary service endpoint
+     * @param additionalEndpoints and additional endpoints keyed by their logical name
+     * @param shardId Unique shard identifier for this member of the service.
+     * @return an EndpointStatus object that allows the endpoint to adjust its status
+     * @throws Group.JoinException if there was a problem joining the server set
+     * @throws InterruptedException if interrupted while waiting to join the server set
+     */
+    @Override
+    public EndpointStatus join(InetSocketAddress endpoint,
+                               Map<String, InetSocketAddress> additionalEndpoints,
+                               int shardId)
+            throws Group.JoinException, InterruptedException {
+        throw new UnsupportedOperationException("NameServerSet does not support join");
+    }
+
+    /**
+     * Registers a monitor to receive change notices for this server set as long as this jvm process
+     * is alive.  Blocks until the initial server set can be gathered and delivered to the monitor.
+     * The monitor will be notified if the membership set or parameters of existing members have
+     * changed.
+     *
+     * @param monitor the server set monitor to call back when the host set changes
+     * @throws com.twitter.common.net.pool.DynamicHostSet.MonitorException if there is a problem monitoring the host set
+     * @deprecated Deprecated in favor of {@link #watch(com.twitter.common.net.pool.DynamicHostSet.HostChangeMonitor)}
+     */
+    @Deprecated
+    @Override
+    public void monitor(HostChangeMonitor<ServiceInstance> monitor) throws MonitorException {
+        throw new UnsupportedOperationException("NameServerSet does not support monitor");
+    }
+
+    /**
+     * Registers a monitor to receive change notices for this server set as long as this jvm process
+     * is alive.  Blocks until the initial server set can be gathered and delivered to the monitor.
+     * The monitor will be notified if the membership set or parameters of existing members have
+     * changed.
+     *
+     * @param monitor the server set monitor to call back when the host set changes
+     * @return A command which, when executed, will stop monitoring the host set.
+     * @throws com.twitter.common.net.pool.DynamicHostSet.MonitorException if there is a problem monitoring the host set
+     */
+    @Override
+    public Command watch(HostChangeMonitor<ServiceInstance> monitor) throws MonitorException {
+        // First add the monitor to the watchers so that it does not miss any changes and invoke
+        // the onChange method
+        synchronized (watchers) {
+            watchers.add(monitor);
+        }
+
+        if (resolutionPending.compareAndSet(false, false)) {
+            monitor.onChange(hostSet);
+        }
+
+        return Commands.NOOP; // Return value is not used
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RegionsRoutingService.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RegionsRoutingService.java
new file mode 100644
index 0000000..d71cee3
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RegionsRoutingService.java
@@ -0,0 +1,192 @@
+/**
+ * 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.client.routing;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.Set;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Chain multiple routing services.
+ */
+public class RegionsRoutingService implements RoutingService {
+
+    private static final Logger logger = LoggerFactory.getLogger(RegionsRoutingService.class);
+
+    /**
+     * Create a multiple regions routing services based on a list of region routing {@code services}.
+     *
+     * <p>It is deprecated. Please use {@link Builder} to build multiple regions routing service.
+     *
+     * @param regionResolver region resolver
+     * @param services a list of region routing services.
+     * @return multiple regions routing service
+     * @see Builder
+     */
+    @Deprecated
+    public static RegionsRoutingService of(RegionResolver regionResolver,
+                                         RoutingService...services) {
+        return new RegionsRoutingService(regionResolver, services);
+    }
+
+    /**
+     * Create a builder to build a multiple-regions routing service.
+     *
+     * @return builder to build a multiple-regions routing service.
+     */
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    /**
+     * Builder to build a multiple-regions routing service.
+     */
+    public static class Builder implements RoutingService.Builder {
+
+        private RegionResolver resolver;
+        private RoutingService.Builder[] routingServiceBuilders;
+        private StatsReceiver statsReceiver = NullStatsReceiver.get();
+
+        private Builder() {}
+
+        public Builder routingServiceBuilders(RoutingService.Builder...builders) {
+            this.routingServiceBuilders = builders;
+            return this;
+        }
+
+        public Builder resolver(RegionResolver regionResolver) {
+            this.resolver = regionResolver;
+            return this;
+        }
+
+        @Override
+        public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
+            this.statsReceiver = statsReceiver;
+            return this;
+        }
+
+        @Override
+        public RegionsRoutingService build() {
+            checkNotNull(routingServiceBuilders, "No routing service builder provided.");
+            checkNotNull(resolver, "No region resolver provided.");
+            checkNotNull(statsReceiver, "No stats receiver provided");
+            RoutingService[] services = new RoutingService[routingServiceBuilders.length];
+            for (int i = 0; i < services.length; i++) {
+                String statsScope;
+                if (0 == i) {
+                    statsScope = "local";
+                } else {
+                    statsScope = "remote_" + i;
+                }
+                services[i] = routingServiceBuilders[i]
+                        .statsReceiver(statsReceiver.scope(statsScope))
+                        .build();
+            }
+            return new RegionsRoutingService(resolver, services);
+        }
+    }
+
+    protected final RegionResolver regionResolver;
+    protected final RoutingService[] routingServices;
+
+    private RegionsRoutingService(RegionResolver resolver,
+                                  RoutingService[] routingServices) {
+        this.regionResolver = resolver;
+        this.routingServices = routingServices;
+    }
+
+    @Override
+    public Set<SocketAddress> getHosts() {
+        Set<SocketAddress> hosts = Sets.newHashSet();
+        for (RoutingService rs : routingServices) {
+            hosts.addAll(rs.getHosts());
+        }
+        return hosts;
+    }
+
+    @Override
+    public void startService() {
+        for (RoutingService service : routingServices) {
+            service.startService();
+        }
+        logger.info("Regions Routing Service Started");
+    }
+
+    @Override
+    public void stopService() {
+        for (RoutingService service : routingServices) {
+            service.stopService();
+        }
+        logger.info("Regions Routing Service Stopped");
+    }
+
+    @Override
+    public RoutingService registerListener(RoutingListener listener) {
+        for (RoutingService service : routingServices) {
+            service.registerListener(listener);
+        }
+        return this;
+    }
+
+    @Override
+    public RoutingService unregisterListener(RoutingListener listener) {
+        for (RoutingService service : routingServices) {
+            service.registerListener(listener);
+        }
+        return this;
+    }
+
+    @Override
+    public SocketAddress getHost(String key, RoutingContext routingContext)
+            throws NoBrokersAvailableException {
+        for (RoutingService service : routingServices) {
+            try {
+                SocketAddress addr = service.getHost(key, routingContext);
+                if (routingContext.hasUnavailableRegions()) {
+                    // current region is unavailable
+                    String region = regionResolver.resolveRegion(addr);
+                    if (routingContext.isUnavailableRegion(region)) {
+                        continue;
+                    }
+                }
+                if (!routingContext.isTriedHost(addr)) {
+                    return addr;
+                }
+            } catch (NoBrokersAvailableException nbae) {
+                // if there isn't broker available in current service, try next service.
+                logger.debug("No brokers available in region {} : ", service, nbae);
+            }
+        }
+        throw new NoBrokersAvailableException("No host found for " + key + ", routing context : " + routingContext);
+    }
+
+    @Override
+    public void removeHost(SocketAddress address, Throwable reason) {
+        for (RoutingService service : routingServices) {
+            service.removeHost(address, reason);
+        }
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingService.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingService.java
new file mode 100644
index 0000000..ad73c17
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingService.java
@@ -0,0 +1,206 @@
+/**
+ * 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.client.routing;
+
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * Routing Service provides mechanism how to route requests.
+ */
+public interface RoutingService {
+
+    /**
+     * Builder to build routing service.
+     */
+    interface Builder {
+
+        /**
+         * Build routing service with stats receiver.
+         *
+         * @param statsReceiver
+         *          stats receiver
+         * @return built routing service
+         */
+        Builder statsReceiver(StatsReceiver statsReceiver);
+
+        /**
+         * Build the routing service.
+         *
+         * @return built routing service
+         */
+        RoutingService build();
+
+    }
+
+    /**
+     * Listener for server changes on routing service.
+     */
+    interface RoutingListener {
+        /**
+         * Trigger when server left.
+         *
+         * @param address left server.
+         */
+        void onServerLeft(SocketAddress address);
+
+        /**
+         * Trigger when server joint.
+         *
+         * @param address joint server.
+         */
+        void onServerJoin(SocketAddress address);
+    }
+
+    /**
+     * Routing Context of a request.
+     */
+    class RoutingContext {
+
+        public static RoutingContext of(RegionResolver resolver) {
+            return new RoutingContext(resolver);
+        }
+
+        final RegionResolver regionResolver;
+        final Map<SocketAddress, StatusCode> triedHosts;
+        final Set<String> unavailableRegions;
+
+        private RoutingContext(RegionResolver regionResolver) {
+            this.regionResolver = regionResolver;
+            this.triedHosts = new HashMap<SocketAddress, StatusCode>();
+            this.unavailableRegions = new HashSet<String>();
+        }
+
+        @Override
+        public synchronized String toString() {
+            return "(tried hosts=" + triedHosts + ")";
+        }
+
+        /**
+         * Add tried host to routing context.
+         *
+         * @param socketAddress
+         *          socket address of tried host.
+         * @param code
+         *          status code returned from tried host.
+         * @return routing context.
+         */
+        public synchronized RoutingContext addTriedHost(SocketAddress socketAddress, StatusCode code) {
+            this.triedHosts.put(socketAddress, code);
+            if (StatusCode.REGION_UNAVAILABLE == code) {
+                unavailableRegions.add(regionResolver.resolveRegion(socketAddress));
+            }
+            return this;
+        }
+
+        /**
+         * Is the host <i>address</i> already tried.
+         *
+         * @param address
+         *          socket address to check
+         * @return true if the address is already tried, otherwise false.
+         */
+        public synchronized boolean isTriedHost(SocketAddress address) {
+            return this.triedHosts.containsKey(address);
+        }
+
+        /**
+         * Whether encountered unavailable regions.
+         *
+         * @return true if encountered unavailable regions, otherwise false.
+         */
+        public synchronized boolean hasUnavailableRegions() {
+            return !unavailableRegions.isEmpty();
+        }
+
+        /**
+         * Whether the <i>region</i> is unavailable.
+         *
+         * @param region
+         *          region
+         * @return true if the region is unavailable, otherwise false.
+         */
+        public synchronized boolean isUnavailableRegion(String region) {
+            return unavailableRegions.contains(region);
+        }
+
+    }
+
+    /**
+     * Start routing service.
+     */
+    void startService();
+
+    /**
+     * Stop routing service.
+     */
+    void stopService();
+
+    /**
+     * Register routing listener.
+     *
+     * @param listener routing listener.
+     * @return routing service.
+     */
+    RoutingService registerListener(RoutingListener listener);
+
+    /**
+     * Unregister routing listener.
+     *
+     * @param listener routing listener.
+     * @return routing service.
+     */
+    RoutingService unregisterListener(RoutingListener listener);
+
+    /**
+     * Get all the hosts that available in routing service.
+     *
+     * @return all the hosts
+     */
+    Set<SocketAddress> getHosts();
+
+    /**
+     * Get the host to route the request by <i>key</i>.
+     *
+     * @param key
+     *          key to route the request.
+     * @param rContext
+     *          routing context.
+     * @return host to route the request
+     * @throws NoBrokersAvailableException
+     */
+    SocketAddress getHost(String key, RoutingContext rContext)
+            throws NoBrokersAvailableException;
+
+    /**
+     * Remove the host <i>address</i> for a specific <i>reason</i>.
+     *
+     * @param address
+     *          host address to remove
+     * @param reason
+     *          reason to remove the host
+     */
+    void removeHost(SocketAddress address, Throwable reason);
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingServiceProvider.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingServiceProvider.java
new file mode 100644
index 0000000..4ac22ce
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingServiceProvider.java
@@ -0,0 +1,39 @@
+/**
+ * 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.client.routing;
+
+import com.twitter.finagle.stats.StatsReceiver;
+
+class RoutingServiceProvider implements RoutingService.Builder {
+
+    final RoutingService routingService;
+
+    RoutingServiceProvider(RoutingService routingService) {
+        this.routingService = routingService;
+    }
+
+    @Override
+    public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
+        return this;
+    }
+
+    @Override
+    public RoutingService build() {
+        return routingService;
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingUtils.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingUtils.java
new file mode 100644
index 0000000..8e8edd3
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingUtils.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.client.routing;
+
+import com.twitter.common.zookeeper.ServerSet;
+import java.net.SocketAddress;
+
+/**
+ * Utils for routing services.
+ */
+public class RoutingUtils {
+
+    private static final int NUM_CONSISTENT_HASH_REPLICAS = 997;
+
+    /**
+     * Building routing service from <code>finagleNameStr</code>.
+     *
+     * @param finagleNameStr
+     *          finagle name str of a service
+     * @return routing service builder
+     */
+    public static RoutingService.Builder buildRoutingService(String finagleNameStr) {
+        if (!finagleNameStr.startsWith("serverset!")
+                && !finagleNameStr.startsWith("inet!")
+                && !finagleNameStr.startsWith("zk!")) {
+            // We only support serverset based names at the moment
+            throw new UnsupportedOperationException("Finagle Name format not supported for name: " + finagleNameStr);
+        }
+        return buildRoutingService(new NameServerSet(finagleNameStr), true);
+    }
+
+    /**
+     * Building routing service from <code>serverSet</code>.
+     *
+     * @param serverSet
+     *          server set of a service
+     * @return routing service builder
+     */
+    public static RoutingService.Builder buildRoutingService(ServerSet serverSet) {
+        return buildRoutingService(serverSet, false);
+    }
+
+    /**
+     * Building routing service from <code>address</code>.
+     *
+     * @param address
+     *          host to route the requests
+     * @return routing service builder
+     */
+    public static RoutingService.Builder buildRoutingService(SocketAddress address) {
+        return SingleHostRoutingService.newBuilder().address(address);
+    }
+
+    /**
+     * Build routing service builder of a routing service <code>routingService</code>.
+     *
+     * @param routingService
+     *          routing service to provide
+     * @return routing service builder
+     */
+    public static RoutingService.Builder buildRoutingService(RoutingService routingService) {
+        return new RoutingServiceProvider(routingService);
+    }
+
+    private static RoutingService.Builder buildRoutingService(ServerSet serverSet,
+                                                              boolean resolveFromName) {
+        return ConsistentHashRoutingService.newBuilder()
+                .serverSet(serverSet)
+                .resolveFromName(resolveFromName)
+                .numReplicas(NUM_CONSISTENT_HASH_REPLICAS);
+    }
+
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetRoutingService.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetRoutingService.java
new file mode 100644
index 0000000..4fe8141
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetRoutingService.java
@@ -0,0 +1,274 @@
+/**
+ * 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.client.routing;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+import org.apache.distributedlog.service.DLSocketAddress;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Routing Service based on a given {@link com.twitter.common.zookeeper.ServerSet}.
+ */
+class ServerSetRoutingService extends Thread implements RoutingService {
+
+    private static final Logger logger = LoggerFactory.getLogger(ServerSetRoutingService.class);
+
+    static ServerSetRoutingServiceBuilder newServerSetRoutingServiceBuilder() {
+        return new ServerSetRoutingServiceBuilder();
+    }
+
+    /**
+     * Builder to build {@link com.twitter.common.zookeeper.ServerSet} based routing service.
+     */
+    static class ServerSetRoutingServiceBuilder implements RoutingService.Builder {
+
+        private ServerSetWatcher serverSetWatcher;
+
+        private ServerSetRoutingServiceBuilder() {}
+
+        public ServerSetRoutingServiceBuilder serverSetWatcher(ServerSetWatcher serverSetWatcher) {
+            this.serverSetWatcher = serverSetWatcher;
+            return this;
+        }
+
+        @Override
+        public Builder statsReceiver(StatsReceiver statsReceiver) {
+            return this;
+        }
+
+        @Override
+        public RoutingService build() {
+            checkNotNull(serverSetWatcher, "No serverset watcher provided.");
+            return new ServerSetRoutingService(this.serverSetWatcher);
+        }
+    }
+
+    private static class HostComparator implements Comparator<SocketAddress> {
+
+        private static final HostComparator INSTANCE = new HostComparator();
+
+        @Override
+        public int compare(SocketAddress o1, SocketAddress o2) {
+            return o1.toString().compareTo(o2.toString());
+        }
+    }
+
+    private final ServerSetWatcher serverSetWatcher;
+
+    private final Set<SocketAddress> hostSet = new HashSet<SocketAddress>();
+    private List<SocketAddress> hostList = new ArrayList<SocketAddress>();
+    private final HashFunction hasher = Hashing.md5();
+
+    // Server Set Changes
+    private final AtomicReference<ImmutableSet<DLSocketAddress>> serverSetChange =
+            new AtomicReference<ImmutableSet<DLSocketAddress>>(null);
+    private final CountDownLatch changeLatch = new CountDownLatch(1);
+
+    // Listeners
+    protected final CopyOnWriteArraySet<RoutingListener> listeners =
+            new CopyOnWriteArraySet<RoutingListener>();
+
+    ServerSetRoutingService(ServerSetWatcher serverSetWatcher) {
+        super("ServerSetRoutingService");
+        this.serverSetWatcher = serverSetWatcher;
+    }
+
+    @Override
+    public Set<SocketAddress> getHosts() {
+        synchronized (hostSet) {
+            return ImmutableSet.copyOf(hostSet);
+        }
+    }
+
+    @Override
+    public void startService() {
+        start();
+        try {
+            if (!changeLatch.await(1, TimeUnit.MINUTES)) {
+                logger.warn("No serverset change received in 1 minute.");
+            }
+        } catch (InterruptedException e) {
+            logger.warn("Interrupted waiting first serverset change : ", e);
+        }
+        logger.info("{} Routing Service Started.", getClass().getSimpleName());
+    }
+
+    @Override
+    public void stopService() {
+        Thread.currentThread().interrupt();
+        try {
+            join();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            logger.warn("Interrupted on waiting serverset routing service to finish : ", e);
+        }
+        logger.info("{} Routing Service Stopped.", getClass().getSimpleName());
+    }
+
+    @Override
+    public RoutingService registerListener(RoutingListener listener) {
+        listeners.add(listener);
+        return this;
+    }
+
+    @Override
+    public RoutingService unregisterListener(RoutingListener listener) {
+        listeners.remove(listener);
+        return this;
+    }
+
+    @Override
+    public SocketAddress getHost(String key, RoutingContext rContext)
+            throws NoBrokersAvailableException {
+        SocketAddress address = null;
+        synchronized (hostSet) {
+            if (0 != hostList.size()) {
+                int hashCode = hasher.hashUnencodedChars(key).asInt();
+                int hostId = signSafeMod(hashCode, hostList.size());
+                address = hostList.get(hostId);
+                if (rContext.isTriedHost(address)) {
+                    ArrayList<SocketAddress> newList = new ArrayList<SocketAddress>(hostList);
+                    newList.remove(hostId);
+                    // pickup a new host by rehashing it.
+                    hostId = signSafeMod(hashCode, newList.size());
+                    address = newList.get(hostId);
+                    int i = hostId;
+                    while (rContext.isTriedHost(address)) {
+                        i = (i + 1) % newList.size();
+                        if (i == hostId) {
+                            address = null;
+                            break;
+                        }
+                        address = newList.get(i);
+                    }
+                }
+            }
+        }
+        if (null == address) {
+            throw new NoBrokersAvailableException("No host is available.");
+        }
+        return address;
+    }
+
+    @Override
+    public void removeHost(SocketAddress host, Throwable reason) {
+        synchronized (hostSet) {
+            if (hostSet.remove(host)) {
+                logger.info("Node {} left due to : ", host, reason);
+            }
+            hostList = new ArrayList<SocketAddress>(hostSet);
+            Collections.sort(hostList, HostComparator.INSTANCE);
+            logger.info("Host list becomes : {}.", hostList);
+        }
+    }
+
+    @Override
+    public void run() {
+        try {
+            serverSetWatcher.watch(new ServerSetWatcher.ServerSetMonitor() {
+                @Override
+                public void onChange(ImmutableSet<DLSocketAddress> serviceInstances) {
+                    ImmutableSet<DLSocketAddress> lastValue = serverSetChange.getAndSet(serviceInstances);
+                    if (null == lastValue) {
+                        ImmutableSet<DLSocketAddress> mostRecentValue;
+                        do {
+                            mostRecentValue = serverSetChange.get();
+                            performServerSetChange(mostRecentValue);
+                            changeLatch.countDown();
+                        } while (!serverSetChange.compareAndSet(mostRecentValue, null));
+                    }
+                }
+            });
+        } catch (Exception e) {
+            logger.error("Fail to monitor server set : ", e);
+            Runtime.getRuntime().exit(-1);
+        }
+    }
+
+    protected synchronized void performServerSetChange(ImmutableSet<DLSocketAddress> serverSet) {
+        Set<SocketAddress> newSet = new HashSet<SocketAddress>();
+        for (DLSocketAddress serviceInstance : serverSet) {
+            newSet.add(serviceInstance.getSocketAddress());
+        }
+
+        Set<SocketAddress> removed;
+        Set<SocketAddress> added;
+        synchronized (hostSet) {
+            removed = Sets.difference(hostSet, newSet).immutableCopy();
+            added = Sets.difference(newSet, hostSet).immutableCopy();
+            for (SocketAddress node: removed) {
+                if (hostSet.remove(node)) {
+                    logger.info("Node {} left.", node);
+                }
+            }
+            for (SocketAddress node: added) {
+                if (hostSet.add(node)) {
+                    logger.info("Node {} joined.", node);
+                }
+            }
+        }
+
+        for (SocketAddress addr : removed) {
+            for (RoutingListener listener : listeners) {
+                listener.onServerLeft(addr);
+            }
+        }
+
+        for (SocketAddress addr : added) {
+            for (RoutingListener listener : listeners) {
+                listener.onServerJoin(addr);
+            }
+        }
+
+        synchronized (hostSet) {
+            hostList = new ArrayList<SocketAddress>(hostSet);
+            Collections.sort(hostList, HostComparator.INSTANCE);
+            logger.info("Host list becomes : {}.", hostList);
+        }
+
+    }
+
+    static int signSafeMod(long dividend, int divisor) {
+        int mod = (int) (dividend % divisor);
+
+        if (mod < 0) {
+            mod += divisor;
+        }
+
+        return mod;
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetWatcher.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetWatcher.java
new file mode 100644
index 0000000..77b7beb
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetWatcher.java
@@ -0,0 +1,71 @@
+/**
+ * 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.client.routing;
+
+import com.google.common.collect.ImmutableSet;
+import org.apache.distributedlog.service.DLSocketAddress;
+
+/**
+ * Watch on server set changes.
+ */
+public interface ServerSetWatcher {
+
+    /**
+     * Exception thrown when failed to monitor serverset.
+     */
+    class MonitorException extends Exception {
+
+        private static final long serialVersionUID = 392751505154339548L;
+
+        public MonitorException(String msg) {
+            super(msg);
+        }
+
+        public MonitorException(String msg, Throwable cause) {
+            super(msg, cause);
+        }
+    }
+
+    /**
+     * An interface to an object that is interested in receiving notification whenever the host set changes.
+     */
+    interface ServerSetMonitor {
+
+        /**
+         * Called when either the available set of services changes.
+         *
+         * <p>It happens either when a service dies or a new INSTANCE comes on-line or
+         * when an existing service advertises a status or health change.
+         *
+         * @param hostSet the current set of available ServiceInstances
+         */
+        void onChange(ImmutableSet<DLSocketAddress> hostSet);
+    }
+
+    /**
+     * Registers a monitor to receive change notices for this server set as long as this jvm process is alive.
+     *
+     * <p>Blocks until the initial server set can be gathered and delivered to the monitor.
+     * The monitor will be notified if the membership set or parameters of existing members have
+     * changed.
+     *
+     * @param monitor the server set monitor to call back when the host set changes
+     * @throws MonitorException if there is a problem monitoring the host set
+     */
+    void watch(final ServerSetMonitor monitor) throws MonitorException;
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/SingleHostRoutingService.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/SingleHostRoutingService.java
new file mode 100644
index 0000000..753a1af
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/SingleHostRoutingService.java
@@ -0,0 +1,128 @@
+/**
+ * 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.client.routing;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.collect.Sets;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+
+/**
+ * Single Host Routing Service.
+ */
+public class SingleHostRoutingService implements RoutingService {
+
+    public static SingleHostRoutingService of(SocketAddress address) {
+        return new SingleHostRoutingService(address);
+    }
+
+    /**
+     * Builder to build single host based routing service.
+     *
+     * @return builder to build single host based routing service.
+     */
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    /**
+     * Builder to build single host based routing service.
+     */
+    public static class Builder implements RoutingService.Builder {
+
+        private SocketAddress address;
+
+        private Builder() {}
+
+        public Builder address(SocketAddress address) {
+            this.address = address;
+            return this;
+        }
+
+        @Override
+        public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
+            return this;
+        }
+
+        @Override
+        public RoutingService build() {
+            checkNotNull(address, "Host is null");
+            return new SingleHostRoutingService(address);
+        }
+    }
+
+    private SocketAddress address;
+    private final CopyOnWriteArraySet<RoutingListener> listeners =
+            new CopyOnWriteArraySet<RoutingListener>();
+
+    SingleHostRoutingService(SocketAddress address) {
+        this.address = address;
+    }
+
+    public void setAddress(SocketAddress address) {
+        this.address = address;
+    }
+
+    @Override
+    public Set<SocketAddress> getHosts() {
+        return Sets.newHashSet(address);
+    }
+
+    @Override
+    public void startService() {
+        // no-op
+        for (RoutingListener listener : listeners) {
+            listener.onServerJoin(address);
+        }
+    }
+
+    @Override
+    public void stopService() {
+        // no-op
+    }
+
+    @Override
+    public RoutingService registerListener(RoutingListener listener) {
+        listeners.add(listener);
+        return this;
+    }
+
+    @Override
+    public RoutingService unregisterListener(RoutingListener listener) {
+        listeners.remove(listener);
+        return null;
+    }
+
+    @Override
+    public SocketAddress getHost(String key, RoutingContext rContext)
+            throws NoBrokersAvailableException {
+        if (rContext.isTriedHost(address)) {
+            throw new NoBrokersAvailableException("No hosts is available : routing context = " + rContext);
+        }
+        return address;
+    }
+
+    @Override
+    public void removeHost(SocketAddress address, Throwable reason) {
+        // no-op
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TestName.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TestName.java
new file mode 100644
index 0000000..2fc8de0
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TestName.java
@@ -0,0 +1,49 @@
+/**
+ * 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.client.routing;
+
+import com.twitter.finagle.Addr;
+import com.twitter.finagle.Address;
+import com.twitter.finagle.Addrs;
+import com.twitter.finagle.Name;
+import java.util.List;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A {@link Name} implementation for testing purpose.
+ */
+public class TestName implements Name {
+
+    private static final Logger LOG = LoggerFactory.getLogger(TestName.class);
+
+    private AbstractFunction1<Addr, BoxedUnit> callback = null;
+
+    public void changes(AbstractFunction1<Addr, BoxedUnit> callback) {
+        this.callback = callback;
+    }
+
+    public void changeAddrs(List<Address> addresses) {
+        if (null != callback) {
+            LOG.info("Sending a callback {}", addresses);
+            callback.apply(Addrs.newBoundAddr(addresses));
+        }
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TwitterServerSetWatcher.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TwitterServerSetWatcher.java
new file mode 100644
index 0000000..1ff7c93
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TwitterServerSetWatcher.java
@@ -0,0 +1,83 @@
+/**
+ * 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.client.routing;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Sets;
+import com.twitter.common.net.pool.DynamicHostSet;
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.service.DLSocketAddress;
+import com.twitter.thrift.Endpoint;
+import com.twitter.thrift.ServiceInstance;
+import java.net.InetSocketAddress;
+import java.util.Set;
+
+/**
+ * Twitter {@link ServerSet} based watcher.
+ */
+public class TwitterServerSetWatcher implements ServerSetWatcher {
+
+    private final ServerSet serverSet;
+    private final boolean resolvedFromName;
+
+    /**
+     * Construct a {@link ServerSet} based watcher.
+     *
+     * @param serverSet server set.
+     * @param resolvedFromName whether to resolve hosts from {@link com.twitter.finagle.Name}.
+     */
+    public TwitterServerSetWatcher(ServerSet serverSet,
+                                   boolean resolvedFromName) {
+        this.serverSet = serverSet;
+        this.resolvedFromName = resolvedFromName;
+    }
+
+    /**
+     * Registers a monitor to receive change notices for this server set as long as this jvm process is alive.
+     *
+     * <p>Blocks until the initial server set can be gathered and delivered to the monitor.
+     * The monitor will be notified if the membership set or parameters of existing members have
+     * changed.
+     *
+     * @param monitor the server set monitor to call back when the host set changes
+     * @throws MonitorException if there is a problem monitoring the host set
+     */
+    public void watch(final ServerSetMonitor monitor)
+            throws MonitorException {
+        try {
+            serverSet.watch(new DynamicHostSet.HostChangeMonitor<ServiceInstance>() {
+                @Override
+                public void onChange(ImmutableSet<ServiceInstance> serviceInstances) {
+                    Set<DLSocketAddress> dlServers = Sets.newHashSet();
+                    for (ServiceInstance serviceInstance : serviceInstances) {
+                        Endpoint endpoint = serviceInstance.getAdditionalEndpoints().get("thrift");
+                        InetSocketAddress inetAddr =
+                                new InetSocketAddress(endpoint.getHost(), endpoint.getPort());
+                        int shardId = resolvedFromName ? -1 : serviceInstance.getShard();
+                        DLSocketAddress address = new DLSocketAddress(shardId, inetAddr);
+                        dlServers.add(address);
+                    }
+                    monitor.onChange(ImmutableSet.copyOf(dlServers));
+                }
+            });
+        } catch (DynamicHostSet.MonitorException me) {
+            throw new MonitorException("Failed to monitor server set : ", me);
+        }
+    }
+
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/package-info.java
new file mode 100644
index 0000000..352d755
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Routing Mechanisms to route the traffic to the owner of streams.
+ */
+package org.apache.distributedlog.client.routing;
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/DLZkServerSet.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/DLZkServerSet.java
new file mode 100644
index 0000000..93cdf7a
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/DLZkServerSet.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.client.serverset;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.net.HostAndPort;
+import com.twitter.common.quantity.Amount;
+import com.twitter.common.quantity.Time;
+import com.twitter.common.zookeeper.ServerSet;
+import com.twitter.common.zookeeper.ServerSets;
+import com.twitter.common.zookeeper.ZooKeeperClient;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import org.apache.commons.lang.StringUtils;
+import org.apache.zookeeper.ZooDefs;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A wrapper over zookeeper client and its server set.
+ */
+public class DLZkServerSet {
+
+    private static final Logger logger = LoggerFactory.getLogger(DLZkServerSet.class);
+
+    static final String ZNODE_WRITE_PROXY = ".write_proxy";
+
+    private static String getZKServersFromDLUri(URI uri) {
+        return uri.getAuthority().replace(";", ",");
+    }
+
+    private static Iterable<InetSocketAddress> getZkAddresses(URI uri) {
+        String zkServers = getZKServersFromDLUri(uri);
+        String[] zkServerList = StringUtils.split(zkServers, ',');
+        ImmutableList.Builder<InetSocketAddress> builder = ImmutableList.builder();
+        for (String zkServer : zkServerList) {
+            HostAndPort hostAndPort = HostAndPort.fromString(zkServer).withDefaultPort(2181);
+            builder.add(InetSocketAddress.createUnresolved(
+                    hostAndPort.getHostText(),
+                    hostAndPort.getPort()));
+        }
+        return builder.build();
+    }
+
+    public static DLZkServerSet of(URI uri,
+                                   int zkSessionTimeoutMs) {
+        // Create zookeeper and server set
+        String zkPath = uri.getPath() + "/" + ZNODE_WRITE_PROXY;
+        Iterable<InetSocketAddress> zkAddresses = getZkAddresses(uri);
+        ZooKeeperClient zkClient =
+                new ZooKeeperClient(Amount.of(zkSessionTimeoutMs, Time.MILLISECONDS), zkAddresses);
+        ServerSet serverSet = ServerSets.create(zkClient, ZooDefs.Ids.OPEN_ACL_UNSAFE, zkPath);
+        return new DLZkServerSet(zkClient, serverSet);
+    }
+
+    private final ZooKeeperClient zkClient;
+    private final ServerSet zkServerSet;
+
+    public DLZkServerSet(ZooKeeperClient zkClient,
+                         ServerSet zkServerSet) {
+        this.zkClient = zkClient;
+        this.zkServerSet = zkServerSet;
+    }
+
+    public ZooKeeperClient getZkClient() {
+        return zkClient;
+    }
+
+    public ServerSet getServerSet() {
+        return zkServerSet;
+    }
+
+    public void close() {
+        zkClient.close();
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/package-info.java
new file mode 100644
index 0000000..38a7544
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Utils related to server set.
+ */
+package org.apache.distributedlog.client.serverset;
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
new file mode 100644
index 0000000..f1da33c
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.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.client.speculative;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Default implementation of {@link SpeculativeRequestExecutionPolicy}.
+ */
+public class DefaultSpeculativeRequestExecutionPolicy implements SpeculativeRequestExecutionPolicy {
+
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultSpeculativeRequestExecutionPolicy.class);
+    final int firstSpeculativeRequestTimeout;
+    final int maxSpeculativeRequestTimeout;
+    final float backoffMultiplier;
+    int nextSpeculativeRequestTimeout;
+
+    public DefaultSpeculativeRequestExecutionPolicy(int firstSpeculativeRequestTimeout,
+                                                    int maxSpeculativeRequestTimeout,
+                                                    float backoffMultiplier) {
+        this.firstSpeculativeRequestTimeout = firstSpeculativeRequestTimeout;
+        this.maxSpeculativeRequestTimeout = maxSpeculativeRequestTimeout;
+        this.backoffMultiplier = backoffMultiplier;
+        this.nextSpeculativeRequestTimeout = firstSpeculativeRequestTimeout;
+
+        if (backoffMultiplier <= 0) {
+            throw new IllegalArgumentException("Invalid value provided for backoffMultiplier");
+        }
+
+        // Prevent potential over flow
+        if (Math.round((double) maxSpeculativeRequestTimeout * (double) backoffMultiplier) > Integer.MAX_VALUE) {
+            throw new IllegalArgumentException("Invalid values for maxSpeculativeRequestTimeout and backoffMultiplier");
+        }
+    }
+
+    @VisibleForTesting
+    int getNextSpeculativeRequestTimeout() {
+        return nextSpeculativeRequestTimeout;
+    }
+
+    /**
+     * Initialize the speculative request execution policy.
+     *
+     * @param scheduler The scheduler service to issue the speculative request
+     * @param requestExecutor The executor is used to issue the actual speculative requests
+     */
+    @Override
+    public void initiateSpeculativeRequest(final ScheduledExecutorService scheduler,
+                                           final SpeculativeRequestExecutor requestExecutor) {
+        issueSpeculativeRequest(scheduler, requestExecutor);
+    }
+
+    private void issueSpeculativeRequest(final ScheduledExecutorService scheduler,
+                                         final SpeculativeRequestExecutor requestExecutor) {
+        Future<Boolean> issueNextRequest = requestExecutor.issueSpeculativeRequest();
+        issueNextRequest.addEventListener(new FutureEventListener<Boolean>() {
+            // we want this handler to run immediately after we push the big red button!
+            @Override
+            public void onSuccess(Boolean issueNextRequest) {
+                if (issueNextRequest) {
+                    scheduleSpeculativeRequest(scheduler, requestExecutor, nextSpeculativeRequestTimeout);
+                    nextSpeculativeRequestTimeout = Math.min(maxSpeculativeRequestTimeout,
+                            (int) (nextSpeculativeRequestTimeout * backoffMultiplier));
+                } else {
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("Stopped issuing speculative requests for {}, "
+                            + "speculativeReadTimeout = {}", requestExecutor, nextSpeculativeRequestTimeout);
+                    }
+                }
+            }
+
+            @Override
+            public void onFailure(Throwable thrown) {
+                LOG.warn("Failed to issue speculative request for {}, speculativeReadTimeout = {} : ",
+                        new Object[] { requestExecutor, nextSpeculativeRequestTimeout, thrown });
+            }
+        });
+    }
+
+    private void scheduleSpeculativeRequest(final ScheduledExecutorService scheduler,
+                                            final SpeculativeRequestExecutor requestExecutor,
+                                            final int speculativeRequestTimeout) {
+        try {
+            scheduler.schedule(new Runnable() {
+                @Override
+                public void run() {
+                    issueSpeculativeRequest(scheduler, requestExecutor);
+                }
+            }, speculativeRequestTimeout, TimeUnit.MILLISECONDS);
+        } catch (RejectedExecutionException re) {
+            if (!scheduler.isShutdown()) {
+                LOG.warn("Failed to schedule speculative request for {}, speculativeReadTimeout = {} : ",
+                        new Object[]{requestExecutor, speculativeRequestTimeout, re});
+            }
+        }
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
new file mode 100644
index 0000000..faf45c2
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
@@ -0,0 +1,34 @@
+/**
+ * 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.client.speculative;
+
+import java.util.concurrent.ScheduledExecutorService;
+
+/**
+ * Speculative request execution policy.
+ */
+public interface SpeculativeRequestExecutionPolicy {
+    /**
+     * Initialize the speculative request execution policy and initiate requests.
+     *
+     * @param scheduler The scheduler service to issue the speculative request
+     * @param requestExecutor The executor is used to issue the actual speculative requests
+     */
+    void initiateSpeculativeRequest(ScheduledExecutorService scheduler,
+                                    SpeculativeRequestExecutor requestExecutor);
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutor.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutor.java
new file mode 100644
index 0000000..68fe8b0
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutor.java
@@ -0,0 +1,33 @@
+/**
+ * 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.client.speculative;
+
+import com.twitter.util.Future;
+
+/**
+ * Executor to execute speculative requests.
+ */
+public interface SpeculativeRequestExecutor {
+
+    /**
+     * Issues a speculative request and indicates if more speculative requests should be issued.
+     *
+     * @return whether more speculative requests should be issued.
+     */
+    Future<Boolean> issueSpeculativeRequest();
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/package-info.java
new file mode 100644
index 0000000..4bdd4b1
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Speculative Mechanism.
+ */
+package org.apache.distributedlog.client.speculative;
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStats.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStats.java
new file mode 100644
index 0000000..c2dcddd
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStats.java
@@ -0,0 +1,108 @@
+/**
+ * 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.client.stats;
+
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Client Stats.
+ */
+public class ClientStats {
+
+    // Region Resolver
+    private final RegionResolver regionResolver;
+
+    // Stats
+    private final StatsReceiver statsReceiver;
+    private final ClientStatsLogger clientStatsLogger;
+    private final boolean enableRegionStats;
+    private final ConcurrentMap<String, ClientStatsLogger> regionClientStatsLoggers;
+    private final ConcurrentMap<String, OpStats> opStatsMap;
+
+    public ClientStats(StatsReceiver statsReceiver,
+                       boolean enableRegionStats,
+                       RegionResolver regionResolver) {
+        this.statsReceiver = statsReceiver;
+        this.clientStatsLogger = new ClientStatsLogger(statsReceiver);
+        this.enableRegionStats = enableRegionStats;
+        this.regionClientStatsLoggers = new ConcurrentHashMap<String, ClientStatsLogger>();
+        this.regionResolver = regionResolver;
+        this.opStatsMap = new ConcurrentHashMap<String, OpStats>();
+    }
+
+    public OpStats getOpStats(String op) {
+        OpStats opStats = opStatsMap.get(op);
+        if (null != opStats) {
+            return opStats;
+        }
+        OpStats newStats = new OpStats(statsReceiver.scope(op),
+                enableRegionStats, regionResolver);
+        OpStats oldStats = opStatsMap.putIfAbsent(op, newStats);
+        if (null == oldStats) {
+            return newStats;
+        } else {
+            return oldStats;
+        }
+    }
+
+    private ClientStatsLogger getRegionClientStatsLogger(SocketAddress address) {
+        String region = regionResolver.resolveRegion(address);
+        return getRegionClientStatsLogger(region);
+    }
+
+    private ClientStatsLogger getRegionClientStatsLogger(String region) {
+        ClientStatsLogger statsLogger = regionClientStatsLoggers.get(region);
+        if (null == statsLogger) {
+            ClientStatsLogger newStatsLogger = new ClientStatsLogger(statsReceiver.scope(region));
+            ClientStatsLogger oldStatsLogger = regionClientStatsLoggers.putIfAbsent(region, newStatsLogger);
+            if (null == oldStatsLogger) {
+                statsLogger = newStatsLogger;
+            } else {
+                statsLogger = oldStatsLogger;
+            }
+        }
+        return statsLogger;
+    }
+
+    public StatsReceiver getFinagleStatsReceiver(SocketAddress addr) {
+        if (enableRegionStats && null != addr) {
+            return getRegionClientStatsLogger(addr).getStatsReceiver();
+        } else {
+            return clientStatsLogger.getStatsReceiver();
+        }
+    }
+
+    public void completeProxyRequest(SocketAddress addr, StatusCode code, long startTimeNanos) {
+        clientStatsLogger.completeProxyRequest(code, startTimeNanos);
+        if (enableRegionStats && null != addr) {
+            getRegionClientStatsLogger(addr).completeProxyRequest(code, startTimeNanos);
+        }
+    }
+
+    public void failProxyRequest(SocketAddress addr, Throwable cause, long startTimeNanos) {
+        clientStatsLogger.failProxyRequest(cause, startTimeNanos);
+        if (enableRegionStats && null != addr) {
+            getRegionClientStatsLogger(addr).failProxyRequest(cause, startTimeNanos);
+        }
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStatsLogger.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStatsLogger.java
new file mode 100644
index 0000000..530c632
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStatsLogger.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.client.stats;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+import com.twitter.finagle.stats.Counter;
+import com.twitter.finagle.stats.Stat;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Stats Logger to collect client stats.
+ */
+public class ClientStatsLogger {
+
+    // Stats
+    private final StatsReceiver statsReceiver;
+    private final StatsReceiver responseStatsReceiver;
+    private final ConcurrentMap<StatusCode, Counter> responseStats =
+            new ConcurrentHashMap<StatusCode, Counter>();
+    private final StatsReceiver exceptionStatsReceiver;
+    private final ConcurrentMap<Class<?>, Counter> exceptionStats =
+            new ConcurrentHashMap<Class<?>, Counter>();
+
+    private final Stat proxySuccessLatencyStat;
+    private final Stat proxyFailureLatencyStat;
+
+    public ClientStatsLogger(StatsReceiver statsReceiver) {
+        this.statsReceiver = statsReceiver;
+        responseStatsReceiver = statsReceiver.scope("responses");
+        exceptionStatsReceiver = statsReceiver.scope("exceptions");
+        StatsReceiver proxyLatencyStatReceiver = statsReceiver.scope("proxy_request_latency");
+        proxySuccessLatencyStat = proxyLatencyStatReceiver.stat0("success");
+        proxyFailureLatencyStat = proxyLatencyStatReceiver.stat0("failure");
+    }
+
+    public StatsReceiver getStatsReceiver() {
+        return statsReceiver;
+    }
+
+    private Counter getResponseCounter(StatusCode code) {
+        Counter counter = responseStats.get(code);
+        if (null == counter) {
+            Counter newCounter = responseStatsReceiver.counter0(code.name());
+            Counter oldCounter = responseStats.putIfAbsent(code, newCounter);
+            counter = null != oldCounter ? oldCounter : newCounter;
+        }
+        return counter;
+    }
+
+    private Counter getExceptionCounter(Class<?> cls) {
+        Counter counter = exceptionStats.get(cls);
+        if (null == counter) {
+            Counter newCounter = exceptionStatsReceiver.counter0(cls.getName());
+            Counter oldCounter = exceptionStats.putIfAbsent(cls, newCounter);
+            counter = null != oldCounter ? oldCounter : newCounter;
+        }
+        return counter;
+    }
+
+    public void completeProxyRequest(StatusCode code, long startTimeNanos) {
+        getResponseCounter(code).incr();
+        proxySuccessLatencyStat.add(elapsedMicroSec(startTimeNanos));
+    }
+
+    public void failProxyRequest(Throwable cause, long startTimeNanos) {
+        getExceptionCounter(cause.getClass()).incr();
+        proxyFailureLatencyStat.add(elapsedMicroSec(startTimeNanos));
+    }
+
+    static long elapsedMicroSec(long startNanoTime) {
+        return TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - startNanoTime);
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStats.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStats.java
new file mode 100644
index 0000000..7a49faa
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStats.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.client.stats;
+
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Op Stats.
+ */
+public class OpStats {
+
+    // Region Resolver
+    private final RegionResolver regionResolver;
+
+    // Stats
+    private final StatsReceiver statsReceiver;
+    private final OpStatsLogger opStatsLogger;
+    private final boolean enableRegionStats;
+    private final ConcurrentMap<String, OpStatsLogger> regionOpStatsLoggers;
+
+    public OpStats(StatsReceiver statsReceiver,
+                   boolean enableRegionStats,
+                   RegionResolver regionResolver) {
+        this.statsReceiver = statsReceiver;
+        this.opStatsLogger = new OpStatsLogger(statsReceiver);
+        this.enableRegionStats = enableRegionStats;
+        this.regionOpStatsLoggers = new ConcurrentHashMap<String, OpStatsLogger>();
+        this.regionResolver = regionResolver;
+    }
+
+    private OpStatsLogger getRegionOpStatsLogger(SocketAddress address) {
+        String region = regionResolver.resolveRegion(address);
+        return getRegionOpStatsLogger(region);
+    }
+
+    private OpStatsLogger getRegionOpStatsLogger(String region) {
+        OpStatsLogger statsLogger = regionOpStatsLoggers.get(region);
+        if (null == statsLogger) {
+            OpStatsLogger newStatsLogger = new OpStatsLogger(statsReceiver.scope(region));
+            OpStatsLogger oldStatsLogger = regionOpStatsLoggers.putIfAbsent(region, newStatsLogger);
+            if (null == oldStatsLogger) {
+                statsLogger = newStatsLogger;
+            } else {
+                statsLogger = oldStatsLogger;
+            }
+        }
+        return statsLogger;
+    }
+
+    public void completeRequest(SocketAddress addr, long micros, int numTries) {
+        opStatsLogger.completeRequest(micros, numTries);
+        if (enableRegionStats && null != addr) {
+            getRegionOpStatsLogger(addr).completeRequest(micros, numTries);
+        }
+    }
+
+    public void failRequest(SocketAddress addr, long micros, int numTries) {
+        opStatsLogger.failRequest(micros, numTries);
+        if (enableRegionStats && null != addr) {
+            getRegionOpStatsLogger(addr).failRequest(micros, numTries);
+        }
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStatsLogger.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStatsLogger.java
new file mode 100644
index 0000000..b94b4be
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStatsLogger.java
@@ -0,0 +1,50 @@
+/**
+ * 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.client.stats;
+
+import com.twitter.finagle.stats.Stat;
+import com.twitter.finagle.stats.StatsReceiver;
+
+/**
+ * Stats Logger per operation type.
+ */
+public class OpStatsLogger {
+
+    private final Stat successLatencyStat;
+    private final Stat failureLatencyStat;
+    private final Stat redirectStat;
+
+    public OpStatsLogger(StatsReceiver statsReceiver) {
+        StatsReceiver latencyStatReceiver = statsReceiver.scope("latency");
+        successLatencyStat = latencyStatReceiver.stat0("success");
+        failureLatencyStat = latencyStatReceiver.stat0("failure");
+        StatsReceiver redirectStatReceiver = statsReceiver.scope("redirects");
+        redirectStat = redirectStatReceiver.stat0("times");
+    }
+
+    public void completeRequest(long micros, int numTries) {
+        successLatencyStat.add(micros);
+        redirectStat.add(numTries);
+    }
+
+    public void failRequest(long micros, int numTries) {
+        failureLatencyStat.add(micros);
+        redirectStat.add(numTries);
+    }
+
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OwnershipStatsLogger.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OwnershipStatsLogger.java
new file mode 100644
index 0000000..110e99a
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OwnershipStatsLogger.java
@@ -0,0 +1,115 @@
+/**
+ * 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.client.stats;
+
+import com.twitter.finagle.stats.Counter;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Stats Logger for ownerships.
+ */
+public class OwnershipStatsLogger {
+
+    /**
+     * Ownership related stats.
+     */
+    public static class OwnershipStat {
+        private final Counter hits;
+        private final Counter misses;
+        private final Counter removes;
+        private final Counter redirects;
+        private final Counter adds;
+
+        OwnershipStat(StatsReceiver ownershipStats) {
+            hits = ownershipStats.counter0("hits");
+            misses = ownershipStats.counter0("misses");
+            adds = ownershipStats.counter0("adds");
+            removes = ownershipStats.counter0("removes");
+            redirects = ownershipStats.counter0("redirects");
+        }
+
+        public void onHit() {
+            hits.incr();
+        }
+
+        public void onMiss() {
+            misses.incr();
+        }
+
+        public void onAdd() {
+            adds.incr();
+        }
+
+        public void onRemove() {
+            removes.incr();
+        }
+
+        public void onRedirect() {
+            redirects.incr();
+        }
+
+    }
+
+    private final OwnershipStat ownershipStat;
+    private final StatsReceiver ownershipStatsReceiver;
+    private final ConcurrentMap<String, OwnershipStat> ownershipStats =
+            new ConcurrentHashMap<String, OwnershipStat>();
+
+    public OwnershipStatsLogger(StatsReceiver statsReceiver,
+                                StatsReceiver streamStatsReceiver) {
+        this.ownershipStat = new OwnershipStat(statsReceiver.scope("ownership"));
+        this.ownershipStatsReceiver = streamStatsReceiver.scope("perstream_ownership");
+    }
+
+    private OwnershipStat getOwnershipStat(String stream) {
+        OwnershipStat stat = ownershipStats.get(stream);
+        if (null == stat) {
+            OwnershipStat newStat = new OwnershipStat(ownershipStatsReceiver.scope(stream));
+            OwnershipStat oldStat = ownershipStats.putIfAbsent(stream, newStat);
+            stat = null != oldStat ? oldStat : newStat;
+        }
+        return stat;
+    }
+
+    public void onMiss(String stream) {
+        ownershipStat.onMiss();
+        getOwnershipStat(stream).onMiss();
+    }
+
+    public void onHit(String stream) {
+        ownershipStat.onHit();
+        getOwnershipStat(stream).onHit();
+    }
+
+    public void onRedirect(String stream) {
+        ownershipStat.onRedirect();
+        getOwnershipStat(stream).onRedirect();
+    }
+
+    public void onRemove(String stream) {
+        ownershipStat.onRemove();
+        getOwnershipStat(stream).onRemove();
+    }
+
+    public void onAdd(String stream) {
+        ownershipStat.onAdd();
+        getOwnershipStat(stream).onAdd();
+    }
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/package-info.java
new file mode 100644
index 0000000..106d3fc
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Client side stats utils.
+ */
+package org.apache.distributedlog.client.stats;
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/service/DLSocketAddress.java b/distributedlog-client/src/main/java/org/apache/distributedlog/service/DLSocketAddress.java
new file mode 100644
index 0000000..68e6825
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/service/DLSocketAddress.java
@@ -0,0 +1,161 @@
+/**
+ * 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.service;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.UnknownHostException;
+
+/**
+ * Socket Address identifier for a DL proxy.
+ */
+public class DLSocketAddress {
+
+    private static final int VERSION = 1;
+
+    private static final String COLON = ":";
+    private static final String SEP = ";";
+
+    private final int shard;
+    private final InetSocketAddress socketAddress;
+
+    public DLSocketAddress(int shard, InetSocketAddress socketAddress) {
+        this.shard = shard;
+        this.socketAddress = socketAddress;
+    }
+
+    /**
+     * Shard id for dl write proxy.
+     *
+     * @return shard id for dl write proxy.
+     */
+    public int getShard() {
+        return shard;
+    }
+
+    /**
+     * Socket address for dl write proxy.
+     *
+     * @return socket address for dl write proxy
+     */
+    public InetSocketAddress getSocketAddress() {
+        return socketAddress;
+    }
+
+    /**
+     * Serialize the write proxy identifier to string.
+     *
+     * @return serialized write proxy identifier.
+     */
+    public String serialize() {
+        return toLockId(socketAddress, shard);
+    }
+
+    @Override
+    public int hashCode() {
+        return socketAddress.hashCode();
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof DLSocketAddress)) {
+            return false;
+        }
+        DLSocketAddress other = (DLSocketAddress) obj;
+        return shard == other.shard && socketAddress.equals(other.socketAddress);
+    }
+
+    @Override
+    public String toString() {
+        return toLockId(socketAddress, shard);
+    }
+
+    /**
+     * Deserialize proxy address from a string representation.
+     *
+     * @param lockId
+     *          string representation of the proxy address.
+     * @return proxy address.
+     * @throws IOException
+     */
+    public static DLSocketAddress deserialize(String lockId) throws IOException {
+        String parts[] = lockId.split(SEP);
+        if (3 != parts.length) {
+            throw new IOException("Invalid dl socket address " + lockId);
+        }
+        int version;
+        try {
+            version = Integer.parseInt(parts[0]);
+        } catch (NumberFormatException nfe) {
+            throw new IOException("Invalid version found in " + lockId, nfe);
+        }
+        if (VERSION != version) {
+            throw new IOException("Invalid version " + version + " found in " + lockId + ", expected " + VERSION);
+        }
+        int shardId;
+        try {
+            shardId = Integer.parseInt(parts[1]);
+        } catch (NumberFormatException nfe) {
+            throw new IOException("Invalid shard id found in " + lockId, nfe);
+        }
+        InetSocketAddress address = parseSocketAddress(parts[2]);
+        return new DLSocketAddress(shardId, address);
+    }
+
+    /**
+     * Parse the inet socket address from the string representation.
+     *
+     * @param addr
+     *          string representation
+     * @return inet socket address
+     */
+    public static InetSocketAddress parseSocketAddress(String addr) {
+        String[] parts =  addr.split(COLON);
+        checkArgument(parts.length == 2);
+        String hostname = parts[0];
+        int port = Integer.parseInt(parts[1]);
+        return new InetSocketAddress(hostname, port);
+    }
+
+    public static InetSocketAddress getSocketAddress(int port) throws UnknownHostException {
+        return new InetSocketAddress(InetAddress.getLocalHost().getHostAddress(), port);
+    }
+
+    /**
+     * Convert inet socket address to the string representation.
+     *
+     * @param address
+     *          inet socket address.
+     * @return string representation of inet socket address.
+     */
+    public static String toString(InetSocketAddress address) {
+        StringBuilder sb = new StringBuilder();
+        sb.append(address.getHostName()).append(COLON).append(address.getPort());
+        return sb.toString();
+    }
+
+    public static String toLockId(InetSocketAddress address, int shard) {
+        StringBuilder sb = new StringBuilder();
+        sb.append(VERSION).append(SEP).append(shard).append(SEP).append(toString(address));
+        return sb.toString();
+    }
+
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClient.java b/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClient.java
new file mode 100644
index 0000000..9f30815
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClient.java
@@ -0,0 +1,108 @@
+/**
+ * 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.service;
+
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecordSetBuffer;
+import com.twitter.util.Future;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Interface for distributedlog client.
+ */
+public interface DistributedLogClient {
+    /**
+     * Write <i>data</i> to a given <i>stream</i>.
+     *
+     * @param stream
+     *          Stream Name.
+     * @param data
+     *          Data to write.
+     * @return a future representing a sequence id returned for this write.
+     */
+    Future<DLSN> write(String stream, ByteBuffer data);
+
+    /**
+     * Write record set to a given <i>stream</i>.
+     *
+     * <p>The record set is built from {@link org.apache.distributedlog.LogRecordSet.Writer}
+     *
+     * @param stream stream to write to
+     * @param recordSet record set
+     */
+    Future<DLSN> writeRecordSet(String stream, LogRecordSetBuffer recordSet);
+
+    /**
+     * Write <i>data</i> in bulk to a given <i>stream</i>.
+     *
+     * <p>Return a list of Future dlsns, one for each submitted buffer. In the event of a partial
+     * failure--ex. some specific buffer write fails, all subsequent writes
+     * will also fail.
+     *
+     * @param stream
+     *          Stream Name.
+     * @param data
+     *          Data to write.
+     * @return a list of futures, one for each submitted buffer.
+     */
+    List<Future<DLSN>> writeBulk(String stream, List<ByteBuffer> data);
+
+    /**
+     * Truncate the stream to a given <i>dlsn</i>.
+     *
+     * @param stream
+     *          Stream Name.
+     * @param dlsn
+     *          DLSN to truncate until.
+     * @return a future representing the truncation.
+     */
+    Future<Boolean> truncate(String stream, DLSN dlsn);
+
+    /**
+     * Release the ownership of a stream <i>stream</i>.
+     *
+     * @param stream
+     *          Stream Name to release.
+     * @return a future representing the release operation.
+     */
+    Future<Void> release(String stream);
+
+    /**
+     * Delete a given stream <i>stream</i>.
+     *
+     * @param stream
+     *          Stream Name to delete.
+     * @return a future representing the delete operation.
+     */
+    Future<Void> delete(String stream);
+
+    /**
+     * Create a stream with name <i>stream</i>.
+     *
+     * @param stream
+     *          Stream Name to create.
+     * @return a future representing the create operation.
+     */
+    Future<Void> create(String stream);
+
+    /**
+     * Close the client.
+     */
+    void close();
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClientBuilder.java b/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClientBuilder.java
new file mode 100644
index 0000000..0e2a152
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClientBuilder.java
@@ -0,0 +1,608 @@
+/**
+ * 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.service;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.client.ClientConfig;
+import org.apache.distributedlog.client.DistributedLogClientImpl;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.client.proxy.ClusterClient;
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import org.apache.distributedlog.client.routing.RegionsRoutingService;
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.client.routing.RoutingUtils;
+import org.apache.distributedlog.thrift.service.DistributedLogService;
+import com.twitter.finagle.Name;
+import com.twitter.finagle.Resolver$;
+import com.twitter.finagle.Service;
+import com.twitter.finagle.ThriftMux;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.finagle.thrift.ClientId;
+import com.twitter.finagle.thrift.ThriftClientFramedCodec;
+import com.twitter.finagle.thrift.ThriftClientRequest;
+import com.twitter.util.Duration;
+import java.net.SocketAddress;
+import java.net.URI;
+import java.util.Random;
+import org.apache.commons.lang.StringUtils;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * Builder to build {@link DistributedLogClient}.
+ */
+public final class DistributedLogClientBuilder {
+
+    private static final Logger logger = LoggerFactory.getLogger(DistributedLogClientBuilder.class);
+
+    private static final Random random = new Random(System.currentTimeMillis());
+
+    private String name = null;
+    private ClientId clientId = null;
+    private RoutingService.Builder routingServiceBuilder = null;
+    private ClientBuilder clientBuilder = null;
+    private String serverRoutingServiceFinagleName = null;
+    private StatsReceiver statsReceiver = new NullStatsReceiver();
+    private StatsReceiver streamStatsReceiver = new NullStatsReceiver();
+    private ClientConfig clientConfig = new ClientConfig();
+    private boolean enableRegionStats = false;
+    private final RegionResolver regionResolver = new DefaultRegionResolver();
+
+    /**
+     * Create a client builder.
+     *
+     * @return client builder
+     */
+    public static DistributedLogClientBuilder newBuilder() {
+        return new DistributedLogClientBuilder();
+    }
+
+    /**
+     * Create a new client builder from an existing {@code builder}.
+     *
+     * @param builder the existing builder.
+     * @return a new client builder.
+     */
+    public static DistributedLogClientBuilder newBuilder(DistributedLogClientBuilder builder) {
+        DistributedLogClientBuilder newBuilder = new DistributedLogClientBuilder();
+        newBuilder.name = builder.name;
+        newBuilder.clientId = builder.clientId;
+        newBuilder.clientBuilder = builder.clientBuilder;
+        newBuilder.routingServiceBuilder = builder.routingServiceBuilder;
+        newBuilder.statsReceiver = builder.statsReceiver;
+        newBuilder.streamStatsReceiver = builder.streamStatsReceiver;
+        newBuilder.enableRegionStats = builder.enableRegionStats;
+        newBuilder.serverRoutingServiceFinagleName = builder.serverRoutingServiceFinagleName;
+        newBuilder.clientConfig = ClientConfig.newConfig(builder.clientConfig);
+        return newBuilder;
+    }
+
+    // private constructor
+    private DistributedLogClientBuilder() {}
+
+    /**
+     * Client Name.
+     *
+     * @param name
+     *          client name
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder name(String name) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.name = name;
+        return newBuilder;
+    }
+
+    /**
+     * Client ID.
+     *
+     * @param clientId
+     *          client id
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder clientId(ClientId clientId) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientId = clientId;
+        return newBuilder;
+    }
+
+    /**
+     * Serverset to access proxy services.
+     *
+     * @param serverSet
+     *          server set.
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder serverSet(ServerSet serverSet) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(serverSet);
+        newBuilder.enableRegionStats = false;
+        return newBuilder;
+    }
+
+    /**
+     * Server Sets to access proxy services.
+     *
+     * <p>The <i>local</i> server set will be tried first then <i>remotes</i>.
+     *
+     * @param local local server set.
+     * @param remotes remote server sets.
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder serverSets(ServerSet local, ServerSet...remotes) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        RoutingService.Builder[] builders = new RoutingService.Builder[remotes.length + 1];
+        builders[0] = RoutingUtils.buildRoutingService(local);
+        for (int i = 1; i < builders.length; i++) {
+            builders[i] = RoutingUtils.buildRoutingService(remotes[i - 1]);
+        }
+        newBuilder.routingServiceBuilder = RegionsRoutingService.newBuilder()
+                .resolver(regionResolver)
+                .routingServiceBuilders(builders);
+        newBuilder.enableRegionStats = remotes.length > 0;
+        return newBuilder;
+    }
+
+    /**
+     * Name to access proxy services.
+     *
+     * @param finagleNameStr
+     *          finagle name string.
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder finagleNameStr(String finagleNameStr) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(finagleNameStr);
+        newBuilder.enableRegionStats = false;
+        return newBuilder;
+    }
+
+    /**
+     * Finagle name strs to access proxy services.
+     *
+     * <p>The <i>local</i> finalge name str will be tried first, then <i>remotes</i>.
+     *
+     * @param local local server set.
+     * @param remotes remote server sets.
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder finagleNameStrs(String local, String...remotes) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        RoutingService.Builder[] builders = new RoutingService.Builder[remotes.length + 1];
+        builders[0] = RoutingUtils.buildRoutingService(local);
+        for (int i = 1; i < builders.length; i++) {
+            builders[i] = RoutingUtils.buildRoutingService(remotes[i - 1]);
+        }
+        newBuilder.routingServiceBuilder = RegionsRoutingService.newBuilder()
+                .routingServiceBuilders(builders)
+                .resolver(regionResolver);
+        newBuilder.enableRegionStats = remotes.length > 0;
+        return newBuilder;
+    }
+
+    /**
+     * URI to access proxy services.
+     *
+     * <p>Assuming the write proxies are announced under `.write_proxy` of the provided namespace uri.
+     * The builder will convert the dl uri (e.g. distributedlog://{zkserver}/path/to/namespace) to
+     * zookeeper serverset based finagle name str (`zk!{zkserver}!/path/to/namespace/.write_proxy`)
+     *
+     * @param uri namespace uri to access the serverset of write proxies
+     * @return distributedlog builder
+     */
+    public DistributedLogClientBuilder uri(URI uri) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        String zkServers = uri.getAuthority().replace(";", ",");
+        String[] zkServerList = StringUtils.split(zkServers, ',');
+        String finagleNameStr = String.format(
+                "zk!%s!%s/.write_proxy",
+                zkServerList[random.nextInt(zkServerList.length)], // zk server
+                uri.getPath());
+        newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(finagleNameStr);
+        newBuilder.enableRegionStats = false;
+        return newBuilder;
+    }
+
+    /**
+     * Address of write proxy to connect.
+     *
+     * @param address
+     *          write proxy address.
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder host(SocketAddress address) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(address);
+        newBuilder.enableRegionStats = false;
+        return newBuilder;
+    }
+
+    private DistributedLogClientBuilder routingServiceBuilder(RoutingService.Builder builder) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.routingServiceBuilder = builder;
+        newBuilder.enableRegionStats = false;
+        return newBuilder;
+    }
+
+    /**
+     * Routing Service to access proxy services.
+     *
+     * @param routingService
+     *          routing service
+     * @return client builder.
+     */
+    @VisibleForTesting
+    public DistributedLogClientBuilder routingService(RoutingService routingService) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.routingServiceBuilder = RoutingUtils.buildRoutingService(routingService);
+        newBuilder.enableRegionStats = false;
+        return newBuilder;
+    }
+
+    /**
+     * Stats receiver to expose client stats.
+     *
+     * @param statsReceiver
+     *          stats receiver.
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder statsReceiver(StatsReceiver statsReceiver) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.statsReceiver = statsReceiver;
+        return newBuilder;
+    }
+
+    /**
+     * Stream Stats Receiver to expose per stream stats.
+     *
+     * @param streamStatsReceiver
+     *          stream stats receiver
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder streamStatsReceiver(StatsReceiver streamStatsReceiver) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.streamStatsReceiver = streamStatsReceiver;
+        return newBuilder;
+    }
+
+    /**
+     * Set underlying finagle client builder.
+     *
+     * @param builder
+     *          finagle client builder.
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder clientBuilder(ClientBuilder builder) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientBuilder = builder;
+        return newBuilder;
+    }
+
+    /**
+     * Backoff time when redirecting to an already retried host.
+     *
+     * @param ms
+     *          backoff time.
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder redirectBackoffStartMs(int ms) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setRedirectBackoffStartMs(ms);
+        return newBuilder;
+    }
+
+    /**
+     * Max backoff time when redirecting to an already retried host.
+     *
+     * @param ms
+     *          backoff time.
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder redirectBackoffMaxMs(int ms) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setRedirectBackoffMaxMs(ms);
+        return newBuilder;
+    }
+
+    /**
+     * Max redirects that is allowed per request.
+     *
+     * <p>If <i>redirects</i> are exhausted, fail the request immediately.
+     *
+     * @param redirects
+     *          max redirects allowed before failing a request.
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder maxRedirects(int redirects) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setMaxRedirects(redirects);
+        return newBuilder;
+    }
+
+    /**
+     * Timeout per request in millis.
+     *
+     * @param timeoutMs
+     *          timeout per request in millis.
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder requestTimeoutMs(int timeoutMs) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setRequestTimeoutMs(timeoutMs);
+        return newBuilder;
+    }
+
+    /**
+     * Set thriftmux enabled.
+     *
+     * @param enabled
+     *          is thriftmux enabled
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder thriftmux(boolean enabled) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setThriftMux(enabled);
+        return newBuilder;
+    }
+
+    /**
+     * Set failfast stream exception handling enabled.
+     *
+     * @param enabled
+     *          is failfast exception handling enabled
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder streamFailfast(boolean enabled) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setStreamFailfast(enabled);
+        return newBuilder;
+    }
+
+    /**
+     * Set the regex to match stream names that the client cares about.
+     *
+     * @param nameRegex
+     *          stream name regex
+     * @return client builder
+     */
+    public DistributedLogClientBuilder streamNameRegex(String nameRegex) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setStreamNameRegex(nameRegex);
+        return newBuilder;
+    }
+
+    /**
+     * Whether to use the new handshake endpoint to exchange ownership cache.
+     *
+     * <p>Enable this when the servers are updated to support handshaking with client info.
+     *
+     * @param enabled
+     *          new handshake endpoint is enabled.
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder handshakeWithClientInfo(boolean enabled) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setHandshakeWithClientInfo(enabled);
+        return newBuilder;
+    }
+
+    /**
+     * Set the periodic handshake interval in milliseconds.
+     *
+     * <p>Every <code>intervalMs</code>, the DL client will handshake with existing proxies again.
+     * If the interval is less than ownership sync interval, the handshake won't sync ownerships. Otherwise, it will.
+     *
+     * @see #periodicOwnershipSyncIntervalMs(long)
+     * @param intervalMs
+     *          handshake interval
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder periodicHandshakeIntervalMs(long intervalMs) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setPeriodicHandshakeIntervalMs(intervalMs);
+        return newBuilder;
+    }
+
+    /**
+     * Set the periodic ownership sync interval in milliseconds.
+     *
+     * <p>If periodic handshake is enabled, the handshake will sync ownership if the elapsed time is larger than
+     * sync interval.
+     *
+     * @see #periodicHandshakeIntervalMs(long)
+     * @param intervalMs
+     *          interval that handshake should sync ownerships.
+     * @return client builder
+     */
+    public DistributedLogClientBuilder periodicOwnershipSyncIntervalMs(long intervalMs) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setPeriodicOwnershipSyncIntervalMs(intervalMs);
+        return newBuilder;
+    }
+
+    /**
+     * Enable/Disable periodic dumping ownership cache.
+     *
+     * @param enabled
+     *          flag to enable/disable periodic dumping ownership cache
+     * @return client builder.
+     */
+    public DistributedLogClientBuilder periodicDumpOwnershipCache(boolean enabled) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setPeriodicDumpOwnershipCacheEnabled(enabled);
+        return newBuilder;
+    }
+
+    /**
+     * Set periodic dumping ownership cache interval.
+     *
+     * @param intervalMs
+     *          interval on dumping ownership cache, in millis.
+     * @return client builder
+     */
+    public DistributedLogClientBuilder periodicDumpOwnershipCacheIntervalMs(long intervalMs) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setPeriodicDumpOwnershipCacheIntervalMs(intervalMs);
+        return newBuilder;
+    }
+
+    /**
+     * Enable handshake tracing.
+     *
+     * @param enabled
+     *          flag to enable/disable handshake tracing
+     * @return client builder
+     */
+    public DistributedLogClientBuilder handshakeTracing(boolean enabled) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setHandshakeTracingEnabled(enabled);
+        return newBuilder;
+    }
+
+    /**
+     * Enable checksum on requests to the proxy.
+     *
+     * @param enabled
+     *          flag to enable/disable checksum
+     * @return client builder
+     */
+    public DistributedLogClientBuilder checksum(boolean enabled) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig.setChecksumEnabled(enabled);
+        return newBuilder;
+    }
+
+    /**
+     * Configure the finagle name string for the server-side routing service.
+     *
+     * @param nameStr name string of the server-side routing service
+     * @return client builder
+     */
+    public DistributedLogClientBuilder serverRoutingServiceFinagleNameStr(String nameStr) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.serverRoutingServiceFinagleName = nameStr;
+        return newBuilder;
+    }
+
+    DistributedLogClientBuilder clientConfig(ClientConfig clientConfig) {
+        DistributedLogClientBuilder newBuilder = newBuilder(this);
+        newBuilder.clientConfig = ClientConfig.newConfig(clientConfig);
+        return newBuilder;
+    }
+
+    /**
+     * Build distributedlog client.
+     *
+     * @return distributedlog client.
+     */
+    public DistributedLogClient build() {
+        return buildClient();
+    }
+
+    /**
+     * Build monitor service client.
+     *
+     * @return monitor service client.
+     */
+    public MonitorServiceClient buildMonitorClient() {
+
+        return buildClient();
+    }
+
+    @SuppressWarnings("unchecked")
+    ClusterClient buildServerRoutingServiceClient(String serverRoutingServiceFinagleName) {
+        ClientBuilder builder = this.clientBuilder;
+        if (null == builder) {
+            builder = ClientBuilder.get()
+                    .tcpConnectTimeout(Duration.fromMilliseconds(200))
+                    .connectTimeout(Duration.fromMilliseconds(200))
+                    .requestTimeout(Duration.fromSeconds(1))
+                    .retries(20);
+            if (!clientConfig.getThriftMux()) {
+                builder = builder.hostConnectionLimit(1);
+            }
+        }
+        if (clientConfig.getThriftMux()) {
+            builder = builder.stack(ThriftMux.client().withClientId(clientId));
+        } else {
+            builder = builder.codec(ThriftClientFramedCodec.apply(Option.apply(clientId)));
+        }
+
+        Name name;
+        try {
+            name = Resolver$.MODULE$.eval(serverRoutingServiceFinagleName);
+        } catch (Exception exc) {
+            logger.error("Exception in Resolver.eval for name {}", serverRoutingServiceFinagleName, exc);
+            throw new RuntimeException(exc);
+        }
+
+        // builder the client
+        Service<ThriftClientRequest, byte[]> client =
+                ClientBuilder.safeBuildFactory(
+                        builder.dest(name).reportTo(statsReceiver.scope("routing"))
+                ).toService();
+        DistributedLogService.ServiceIface service =
+                new DistributedLogService.ServiceToClient(client, new TBinaryProtocol.Factory());
+        return new ClusterClient(client, service);
+    }
+
+    DistributedLogClientImpl buildClient() {
+        checkNotNull(name, "No name provided.");
+        checkNotNull(clientId, "No client id provided.");
+        checkNotNull(routingServiceBuilder, "No routing service builder provided.");
+        checkNotNull(statsReceiver, "No stats receiver provided.");
+        if (null == streamStatsReceiver) {
+            streamStatsReceiver = new NullStatsReceiver();
+        }
+
+        Optional<ClusterClient> serverRoutingServiceClient = Optional.absent();
+        if (null != serverRoutingServiceFinagleName) {
+            serverRoutingServiceClient = Optional.of(
+                    buildServerRoutingServiceClient(serverRoutingServiceFinagleName));
+        }
+
+        RoutingService routingService = routingServiceBuilder
+                .statsReceiver(statsReceiver.scope("routing"))
+                .build();
+        DistributedLogClientImpl clientImpl =
+                new DistributedLogClientImpl(
+                        name,
+                        clientId,
+                        routingService,
+                        clientBuilder,
+                        clientConfig,
+                        serverRoutingServiceClient,
+                        statsReceiver,
+                        streamStatsReceiver,
+                        regionResolver,
+                        enableRegionStats);
+        routingService.startService();
+        clientImpl.handshake();
+        return clientImpl;
+    }
+
+}
diff --git a/distributedlog-client/src/main/java/org/apache/distributedlog/service/package-info.java b/distributedlog-client/src/main/java/org/apache/distributedlog/service/package-info.java
new file mode 100644
index 0000000..033882f
--- /dev/null
+++ b/distributedlog-client/src/main/java/org/apache/distributedlog/service/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * DistributedLog Service Client.
+ */
+package org.apache.distributedlog.service;
diff --git a/distributedlog-client/src/main/resources/findbugsExclude.xml b/distributedlog-client/src/main/resources/findbugsExclude.xml
index 29e1a16..05ee085 100644
--- a/distributedlog-client/src/main/resources/findbugsExclude.xml
+++ b/distributedlog-client/src/main/resources/findbugsExclude.xml
@@ -18,6 +18,6 @@
 <FindBugsFilter>
   <Match>
     <!-- generated code, we can't be held responsible for findbugs in it //-->
-    <Class name="~com\.twitter\.distributedlog\.thrift.*" />
+    <Class name="~org\.apache\.distributedlog\.thrift.*" />
   </Match>
 </FindBugsFilter>
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/TestDistributedLogMultiStreamWriter.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/TestDistributedLogMultiStreamWriter.java
deleted file mode 100644
index b302439..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/TestDistributedLogMultiStreamWriter.java
+++ /dev/null
@@ -1,383 +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 com.twitter.distributedlog.client;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Mockito.any;
-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.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.LogRecordSet;
-import com.twitter.distributedlog.LogRecordSetBuffer;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.finagle.IndividualRequestTimeoutException;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import java.nio.ByteBuffer;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test {@link DistributedLogMultiStreamWriter}.
- */
-public class TestDistributedLogMultiStreamWriter {
-
-    @Test(timeout = 20000, expected = IllegalArgumentException.class)
-    public void testBuildWithNullStreams() throws Exception {
-        DistributedLogMultiStreamWriter.newBuilder()
-                .build();
-    }
-
-    @Test(timeout = 20000, expected = IllegalArgumentException.class)
-    public void testBuildWithEmptyStreamList() throws Exception {
-        DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.<String>newArrayList())
-                .build();
-    }
-
-    @Test(timeout = 20000, expected = NullPointerException.class)
-    public void testBuildWithNullClient() throws Exception {
-        DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .build();
-    }
-
-    @Test(timeout = 20000, expected = NullPointerException.class)
-    public void testBuildWithNullCodec() throws Exception {
-        DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(mock(DistributedLogClient.class))
-                .compressionCodec(null)
-                .build();
-    }
-
-    @Test(timeout = 20000, expected = IllegalArgumentException.class)
-    public void testBuildWithInvalidSpeculativeSettings1()
-            throws Exception {
-        DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(mock(DistributedLogClient.class))
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(-1)
-                .build();
-    }
-
-    @Test(timeout = 20000, expected = IllegalArgumentException.class)
-    public void testBuildWithInvalidSpeculativeSettings2()
-            throws Exception {
-        DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(mock(DistributedLogClient.class))
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(10)
-                .maxSpeculativeTimeoutMs(5)
-                .build();
-    }
-
-    @Test(timeout = 20000, expected = IllegalArgumentException.class)
-    public void testBuildWithInvalidSpeculativeSettings3()
-            throws Exception {
-        DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(mock(DistributedLogClient.class))
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(10)
-                .maxSpeculativeTimeoutMs(20)
-                .speculativeBackoffMultiplier(-1)
-                .build();
-    }
-
-    @Test(timeout = 20000, expected = IllegalArgumentException.class)
-    public void testBuildWithInvalidSpeculativeSettings4()
-            throws Exception {
-        DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(mock(DistributedLogClient.class))
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(10)
-                .maxSpeculativeTimeoutMs(20)
-                .speculativeBackoffMultiplier(2)
-                .requestTimeoutMs(10)
-                .build();
-    }
-
-    @Test(timeout = 20000)
-    public void testBuildMultiStreamWriter()
-            throws Exception {
-        DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(mock(DistributedLogClient.class))
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(10)
-                .maxSpeculativeTimeoutMs(20)
-                .speculativeBackoffMultiplier(2)
-                .requestTimeoutMs(50)
-                .build();
-        assertTrue(true);
-    }
-
-    @Test(timeout = 20000)
-    public void testBuildWithPeriodicalFlushEnabled() throws Exception {
-        ScheduledExecutorService executorService = mock(ScheduledExecutorService.class);
-        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(mock(DistributedLogClient.class))
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(10)
-                .maxSpeculativeTimeoutMs(20)
-                .speculativeBackoffMultiplier(2)
-                .requestTimeoutMs(50)
-                .flushIntervalMs(1000)
-                .scheduler(executorService)
-                .build();
-        verify(executorService, times(1)).scheduleAtFixedRate(writer, 1000000, 1000000, TimeUnit.MICROSECONDS);
-    }
-
-    @Test(timeout = 20000)
-    public void testBuildWithPeriodicalFlushDisabled() throws Exception {
-        ScheduledExecutorService executorService = mock(ScheduledExecutorService.class);
-        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(mock(DistributedLogClient.class))
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(10)
-                .maxSpeculativeTimeoutMs(20)
-                .speculativeBackoffMultiplier(2)
-                .requestTimeoutMs(50)
-                .flushIntervalMs(0)
-                .scheduler(executorService)
-                .build();
-        verify(executorService, times(0)).scheduleAtFixedRate(writer, 1000, 1000, TimeUnit.MILLISECONDS);
-        writer.close();
-    }
-
-    @Test(timeout = 20000)
-    public void testFlushWhenBufferIsFull() throws Exception {
-        DistributedLogClient client = mock(DistributedLogClient.class);
-        when(client.writeRecordSet((String) any(), (LogRecordSetBuffer) any()))
-                .thenReturn(Future.value(new DLSN(1L, 1L, 999L)));
-
-        ScheduledExecutorService executorService =
-                Executors.newSingleThreadScheduledExecutor();
-        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(client)
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(100000)
-                .maxSpeculativeTimeoutMs(200000)
-                .speculativeBackoffMultiplier(2)
-                .requestTimeoutMs(500000)
-                .flushIntervalMs(0)
-                .bufferSize(0)
-                .scheduler(executorService)
-                .build();
-
-        ByteBuffer buffer = ByteBuffer.wrap("test".getBytes(UTF_8));
-        writer.write(buffer);
-
-        verify(client, times(1)).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
-
-        writer.close();
-    }
-
-    @Test(timeout = 20000)
-    public void testFlushWhenExceedMaxLogRecordSetSize()
-            throws Exception {
-        DistributedLogClient client = mock(DistributedLogClient.class);
-        when(client.writeRecordSet((String) any(), (LogRecordSetBuffer) any()))
-                .thenReturn(Future.value(new DLSN(1L, 1L, 999L)));
-        ScheduledExecutorService executorService =
-                Executors.newSingleThreadScheduledExecutor();
-        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(client)
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(100000)
-                .maxSpeculativeTimeoutMs(200000)
-                .speculativeBackoffMultiplier(2)
-                .requestTimeoutMs(500000)
-                .flushIntervalMs(0)
-                .bufferSize(Integer.MAX_VALUE)
-                .scheduler(executorService)
-                .build();
-
-        byte[] data = new byte[LogRecord.MAX_LOGRECORD_SIZE - 3 * 100];
-        ByteBuffer buffer1 = ByteBuffer.wrap(data);
-        writer.write(buffer1);
-        verify(client, times(0)).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
-        LogRecordSet.Writer recordSetWriter1 = writer.getLogRecordSetWriter();
-        assertEquals(1, recordSetWriter1.getNumRecords());
-        assertEquals(LogRecordSet.HEADER_LEN + 4 + data.length, recordSetWriter1.getNumBytes());
-
-        ByteBuffer buffer2 = ByteBuffer.wrap(data);
-        writer.write(buffer2);
-        verify(client, times(1)).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
-        LogRecordSet.Writer recordSetWriter2 = writer.getLogRecordSetWriter();
-        assertEquals(1, recordSetWriter2.getNumRecords());
-        assertEquals(LogRecordSet.HEADER_LEN + 4 + data.length, recordSetWriter2.getNumBytes());
-        assertTrue(recordSetWriter1 != recordSetWriter2);
-
-        writer.close();
-    }
-
-    @Test(timeout = 20000)
-    public void testWriteTooLargeRecord() throws Exception {
-        DistributedLogClient client = mock(DistributedLogClient.class);
-        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(client)
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(100000)
-                .maxSpeculativeTimeoutMs(200000)
-                .speculativeBackoffMultiplier(2)
-                .requestTimeoutMs(5000000)
-                .flushIntervalMs(0)
-                .bufferSize(0)
-                .build();
-
-        byte[] data = new byte[LogRecord.MAX_LOGRECORD_SIZE + 10];
-        ByteBuffer buffer = ByteBuffer.wrap(data);
-        Future<DLSN> writeFuture = writer.write(buffer);
-        assertTrue(writeFuture.isDefined());
-        try {
-            Await.result(writeFuture);
-            fail("Should fail on writing too long record");
-        } catch (LogRecordTooLongException lrtle) {
-            // expected
-        }
-        writer.close();
-    }
-
-    @Test(timeout = 20000)
-    public void testSpeculativeWrite() throws Exception {
-        DistributedLogClient client = mock(DistributedLogClient.class);
-        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(client)
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(10)
-                .maxSpeculativeTimeoutMs(20)
-                .speculativeBackoffMultiplier(2)
-                .requestTimeoutMs(5000000)
-                .flushIntervalMs(0)
-                .bufferSize(0)
-                .build();
-
-        final String secondStream = writer.getStream(1);
-
-        final DLSN dlsn = new DLSN(99L, 88L, 0L);
-
-        Mockito.doAnswer(new Answer() {
-            @Override
-            public Object answer(InvocationOnMock invocation) throws Throwable {
-                Object[] arguments = invocation.getArguments();
-                String stream = (String) arguments[0];
-                if (stream.equals(secondStream)) {
-                    return Future.value(dlsn);
-                } else {
-                    return new Promise<DLSN>();
-                }
-            }
-        }).when(client).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
-
-        byte[] data = "test-test".getBytes(UTF_8);
-        ByteBuffer buffer = ByteBuffer.wrap(data);
-        Future<DLSN> writeFuture = writer.write(buffer);
-        DLSN writeDLSN = Await.result(writeFuture);
-        assertEquals(dlsn, writeDLSN);
-        writer.close();
-    }
-
-    @Test(timeout = 20000)
-    public void testPeriodicalFlush() throws Exception {
-        DistributedLogClient client = mock(DistributedLogClient.class);
-        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(client)
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(10)
-                .maxSpeculativeTimeoutMs(20)
-                .speculativeBackoffMultiplier(2)
-                .requestTimeoutMs(5000000)
-                .flushIntervalMs(10)
-                .bufferSize(Integer.MAX_VALUE)
-                .build();
-
-        final DLSN dlsn = new DLSN(99L, 88L, 0L);
-
-        Mockito.doAnswer(new Answer() {
-            @Override
-            public Object answer(InvocationOnMock invocation) throws Throwable {
-                return Future.value(dlsn);
-            }
-        }).when(client).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
-
-        byte[] data = "test-test".getBytes(UTF_8);
-        ByteBuffer buffer = ByteBuffer.wrap(data);
-        Future<DLSN> writeFuture = writer.write(buffer);
-        DLSN writeDLSN = Await.result(writeFuture);
-        assertEquals(dlsn, writeDLSN);
-        writer.close();
-    }
-
-    @Test(timeout = 20000)
-    public void testFailRequestAfterRetriedAllStreams() throws Exception {
-        DistributedLogClient client = mock(DistributedLogClient.class);
-        when(client.writeRecordSet((String) any(), (LogRecordSetBuffer) any()))
-                .thenReturn(new Promise<DLSN>());
-        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList("stream1", "stream2"))
-                .client(client)
-                .compressionCodec(CompressionCodec.Type.LZ4)
-                .firstSpeculativeTimeoutMs(10)
-                .maxSpeculativeTimeoutMs(20)
-                .speculativeBackoffMultiplier(2)
-                .requestTimeoutMs(5000000)
-                .flushIntervalMs(10)
-                .bufferSize(Integer.MAX_VALUE)
-                .build();
-
-        byte[] data = "test-test".getBytes(UTF_8);
-        ByteBuffer buffer = ByteBuffer.wrap(data);
-        Future<DLSN> writeFuture = writer.write(buffer);
-        try {
-            Await.result(writeFuture);
-            fail("Should fail the request after retries all streams");
-        } catch (IndividualRequestTimeoutException e) {
-            long timeoutMs = e.timeout().inMilliseconds();
-            assertTrue(timeoutMs >= (10 + 20) && timeoutMs < 5000000);
-        }
-        writer.close();
-    }
-}
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/ownership/TestOwnershipCache.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/ownership/TestOwnershipCache.java
deleted file mode 100644
index c0e077b..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/ownership/TestOwnershipCache.java
+++ /dev/null
@@ -1,207 +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 com.twitter.distributedlog.client.ownership;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-
-import com.twitter.distributedlog.client.ClientConfig;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.Map;
-import java.util.Set;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-/**
- * Test Case for Ownership Cache.
- */
-public class TestOwnershipCache {
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    private static OwnershipCache createOwnershipCache() {
-        ClientConfig clientConfig = new ClientConfig();
-        return new OwnershipCache(clientConfig, null,
-                                  NullStatsReceiver.get(), NullStatsReceiver.get());
-    }
-
-    private static SocketAddress createSocketAddress(int port) {
-        return new InetSocketAddress("127.0.0.1", port);
-    }
-
-    @Test(timeout = 60000)
-    public void testUpdateOwner() {
-        OwnershipCache cache = createOwnershipCache();
-        SocketAddress addr = createSocketAddress(1000);
-        String stream = runtime.getMethodName();
-
-        assertTrue("Should successfully update owner if no owner exists before",
-                cache.updateOwner(stream, addr));
-        assertEquals("Owner should be " + addr + " for stream " + stream,
-                addr, cache.getOwner(stream));
-        assertTrue("Should successfully update owner if old owner is same",
-                cache.updateOwner(stream, addr));
-        assertEquals("Owner should be " + addr + " for stream " + stream,
-                addr, cache.getOwner(stream));
-    }
-
-    @Test(timeout = 60000)
-    public void testRemoveOwnerFromStream() {
-        OwnershipCache cache = createOwnershipCache();
-        int initialPort = 2000;
-        int numProxies = 2;
-        int numStreamsPerProxy = 2;
-        for (int i = 0; i < numProxies; i++) {
-            SocketAddress addr = createSocketAddress(initialPort + i);
-            for (int j = 0; j < numStreamsPerProxy; j++) {
-                String stream = runtime.getMethodName() + "_" + i + "_" + j;
-                cache.updateOwner(stream, addr);
-            }
-        }
-        Map<String, SocketAddress> ownershipMap = cache.getStreamOwnerMapping();
-        assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
-                numProxies * numStreamsPerProxy, ownershipMap.size());
-        Map<SocketAddress, Set<String>> ownershipDistribution = cache.getStreamOwnershipDistribution();
-        assertEquals("There should be " + numProxies + " proxies cached",
-                numProxies, ownershipDistribution.size());
-
-        String stream = runtime.getMethodName() + "_0_0";
-        SocketAddress owner = createSocketAddress(initialPort);
-
-        // remove non-existent mapping won't change anything
-        SocketAddress nonExistentAddr = createSocketAddress(initialPort + 999);
-        cache.removeOwnerFromStream(stream, nonExistentAddr, "remove-non-existent-addr");
-        assertEquals("Owner " + owner + " should not be removed",
-                owner, cache.getOwner(stream));
-        ownershipMap = cache.getStreamOwnerMapping();
-        assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
-                numProxies * numStreamsPerProxy, ownershipMap.size());
-
-        // remove existent mapping should remove ownership mapping
-        cache.removeOwnerFromStream(stream, owner, "remove-owner");
-        assertNull("Owner " + owner + " should be removed", cache.getOwner(stream));
-        ownershipMap = cache.getStreamOwnerMapping();
-        assertEquals("There should be " + (numProxies * numStreamsPerProxy - 1) + " entries left in cache",
-                numProxies * numStreamsPerProxy - 1, ownershipMap.size());
-        ownershipDistribution = cache.getStreamOwnershipDistribution();
-        assertEquals("There should still be " + numProxies + " proxies cached",
-                numProxies, ownershipDistribution.size());
-        Set<String> ownedStreams = ownershipDistribution.get(owner);
-        assertEquals("There should be only " + (numStreamsPerProxy - 1) + " streams owned for " + owner,
-                numStreamsPerProxy - 1, ownedStreams.size());
-        assertFalse("Stream " + stream + " should not be owned by " + owner,
-                ownedStreams.contains(stream));
-    }
-
-    @Test(timeout = 60000)
-    public void testRemoveAllStreamsFromOwner() {
-        OwnershipCache cache = createOwnershipCache();
-        int initialPort = 2000;
-        int numProxies = 2;
-        int numStreamsPerProxy = 2;
-        for (int i = 0; i < numProxies; i++) {
-            SocketAddress addr = createSocketAddress(initialPort + i);
-            for (int j = 0; j < numStreamsPerProxy; j++) {
-                String stream = runtime.getMethodName() + "_" + i + "_" + j;
-                cache.updateOwner(stream, addr);
-            }
-        }
-        Map<String, SocketAddress> ownershipMap = cache.getStreamOwnerMapping();
-        assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
-                numProxies * numStreamsPerProxy, ownershipMap.size());
-        Map<SocketAddress, Set<String>> ownershipDistribution = cache.getStreamOwnershipDistribution();
-        assertEquals("There should be " + numProxies + " proxies cached",
-                numProxies, ownershipDistribution.size());
-
-        SocketAddress owner = createSocketAddress(initialPort);
-
-        // remove non-existent host won't change anything
-        SocketAddress nonExistentAddr = createSocketAddress(initialPort + 999);
-        cache.removeAllStreamsFromOwner(nonExistentAddr);
-        ownershipMap = cache.getStreamOwnerMapping();
-        assertEquals("There should still be " + (numProxies * numStreamsPerProxy) + " entries in cache",
-                numProxies * numStreamsPerProxy, ownershipMap.size());
-        ownershipDistribution = cache.getStreamOwnershipDistribution();
-        assertEquals("There should still be " + numProxies + " proxies cached",
-                numProxies, ownershipDistribution.size());
-
-        // remove existent host should remove ownership mapping
-        cache.removeAllStreamsFromOwner(owner);
-        ownershipMap = cache.getStreamOwnerMapping();
-        assertEquals("There should be " + ((numProxies - 1) * numStreamsPerProxy) + " entries left in cache",
-                (numProxies - 1) * numStreamsPerProxy, ownershipMap.size());
-        ownershipDistribution = cache.getStreamOwnershipDistribution();
-        assertEquals("There should be " + (numProxies - 1) + " proxies cached",
-                numProxies - 1, ownershipDistribution.size());
-        assertFalse("Host " + owner + " should not be cached",
-                ownershipDistribution.containsKey(owner));
-    }
-
-    @Test(timeout = 60000)
-    public void testReplaceOwner() {
-        OwnershipCache cache = createOwnershipCache();
-        int initialPort = 2000;
-        int numProxies = 2;
-        int numStreamsPerProxy = 2;
-        for (int i = 0; i < numProxies; i++) {
-            SocketAddress addr = createSocketAddress(initialPort + i);
-            for (int j = 0; j < numStreamsPerProxy; j++) {
-                String stream = runtime.getMethodName() + "_" + i + "_" + j;
-                cache.updateOwner(stream, addr);
-            }
-        }
-        Map<String, SocketAddress> ownershipMap = cache.getStreamOwnerMapping();
-        assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
-                numProxies * numStreamsPerProxy, ownershipMap.size());
-        Map<SocketAddress, Set<String>> ownershipDistribution = cache.getStreamOwnershipDistribution();
-        assertEquals("There should be " + numProxies + " proxies cached",
-                numProxies, ownershipDistribution.size());
-
-        String stream = runtime.getMethodName() + "_0_0";
-        SocketAddress oldOwner = createSocketAddress(initialPort);
-        SocketAddress newOwner = createSocketAddress(initialPort + 999);
-
-        cache.updateOwner(stream, newOwner);
-        assertEquals("Owner of " + stream + " should be changed from " + oldOwner + " to " + newOwner,
-                newOwner, cache.getOwner(stream));
-        ownershipMap = cache.getStreamOwnerMapping();
-        assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
-                numProxies * numStreamsPerProxy, ownershipMap.size());
-        assertEquals("Owner of " + stream + " should be " + newOwner,
-                newOwner, ownershipMap.get(stream));
-        ownershipDistribution = cache.getStreamOwnershipDistribution();
-        assertEquals("There should be " + (numProxies + 1) + " proxies cached",
-                numProxies + 1, ownershipDistribution.size());
-        Set<String> oldOwnedStreams = ownershipDistribution.get(oldOwner);
-        assertEquals("There should be only " + (numStreamsPerProxy - 1) + " streams owned by " + oldOwner,
-                numStreamsPerProxy - 1, oldOwnedStreams.size());
-        assertFalse("Stream " + stream + " should not be owned by " + oldOwner,
-                oldOwnedStreams.contains(stream));
-        Set<String> newOwnedStreams = ownershipDistribution.get(newOwner);
-        assertEquals("There should be only " + (numStreamsPerProxy - 1) + " streams owned by " + newOwner,
-                1, newOwnedStreams.size());
-        assertTrue("Stream " + stream + " should be owned by " + newOwner,
-                newOwnedStreams.contains(stream));
-    }
-}
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockDistributedLogServices.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockDistributedLogServices.java
deleted file mode 100644
index f088c0d..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockDistributedLogServices.java
+++ /dev/null
@@ -1,144 +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 com.twitter.distributedlog.client.proxy;
-
-import com.twitter.distributedlog.thrift.service.BulkWriteResponse;
-import com.twitter.distributedlog.thrift.service.ClientInfo;
-import com.twitter.distributedlog.thrift.service.DistributedLogService;
-import com.twitter.distributedlog.thrift.service.HeartbeatOptions;
-import com.twitter.distributedlog.thrift.service.ServerInfo;
-import com.twitter.distributedlog.thrift.service.WriteContext;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.util.Future;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-/**
- * Mock DistributedLog Related Services.
- */
-public class MockDistributedLogServices {
-
-    /**
-     * Mock basic service.
-     */
-    static class MockBasicService implements DistributedLogService.ServiceIface {
-
-        @Override
-        public Future<ServerInfo> handshake() {
-            return Future.value(new ServerInfo());
-        }
-
-        @Override
-        public Future<ServerInfo> handshakeWithClientInfo(ClientInfo clientInfo) {
-            return Future.value(new ServerInfo());
-        }
-
-        @Override
-        public Future<WriteResponse> heartbeat(String stream, WriteContext ctx) {
-            return Future.value(new WriteResponse());
-        }
-
-        @Override
-        public Future<WriteResponse> heartbeatWithOptions(String stream,
-                                                          WriteContext ctx,
-                                                          HeartbeatOptions options) {
-            return Future.value(new WriteResponse());
-        }
-
-        @Override
-        public Future<WriteResponse> write(String stream,
-                                           ByteBuffer data) {
-            return Future.value(new WriteResponse());
-        }
-
-        @Override
-        public Future<WriteResponse> writeWithContext(String stream,
-                                                      ByteBuffer data,
-                                                      WriteContext ctx) {
-            return Future.value(new WriteResponse());
-        }
-
-        @Override
-        public Future<BulkWriteResponse> writeBulkWithContext(String stream,
-                                                              List<ByteBuffer> data,
-                                                              WriteContext ctx) {
-            return Future.value(new BulkWriteResponse());
-        }
-
-        @Override
-        public Future<WriteResponse> truncate(String stream,
-                                              String dlsn,
-                                              WriteContext ctx) {
-            return Future.value(new WriteResponse());
-        }
-
-        @Override
-        public Future<WriteResponse> release(String stream,
-                                             WriteContext ctx) {
-            return Future.value(new WriteResponse());
-        }
-
-        @Override
-        public Future<WriteResponse> create(String stream, WriteContext ctx) {
-            return Future.value(new WriteResponse());
-        }
-
-        @Override
-        public Future<WriteResponse> delete(String stream,
-                                            WriteContext ctx) {
-            return Future.value(new WriteResponse());
-        }
-
-        @Override
-        public Future<WriteResponse> getOwner(String stream, WriteContext ctx) {
-            return Future.value(new WriteResponse());
-        }
-
-        @Override
-        public Future<Void> setAcceptNewStream(boolean enabled) {
-            return Future.value(null);
-        }
-    }
-
-    /**
-     * Mock server info service.
-     */
-    public static class MockServerInfoService extends MockBasicService {
-
-        protected ServerInfo serverInfo;
-
-        public MockServerInfoService() {
-            serverInfo = new ServerInfo();
-        }
-
-        public void updateServerInfo(ServerInfo serverInfo) {
-            this.serverInfo = serverInfo;
-        }
-
-        @Override
-        public Future<ServerInfo> handshake() {
-            return Future.value(serverInfo);
-        }
-
-        @Override
-        public Future<ServerInfo> handshakeWithClientInfo(ClientInfo clientInfo) {
-            return Future.value(serverInfo);
-        }
-    }
-
-}
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockProxyClientBuilder.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockProxyClientBuilder.java
deleted file mode 100644
index ff0bd05..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockProxyClientBuilder.java
+++ /dev/null
@@ -1,49 +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 com.twitter.distributedlog.client.proxy;
-
-import com.twitter.distributedlog.thrift.service.DistributedLogService;
-import java.net.SocketAddress;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Mock Proxy Client Builder.
- */
-class MockProxyClientBuilder implements ProxyClient.Builder {
-
-    static class MockProxyClient extends ProxyClient {
-        MockProxyClient(SocketAddress address,
-                        DistributedLogService.ServiceIface service) {
-            super(address, new MockThriftClient(), service);
-        }
-    }
-
-    private final ConcurrentMap<SocketAddress, MockProxyClient> clients =
-            new ConcurrentHashMap<SocketAddress, MockProxyClient>();
-
-    public void provideProxyClient(SocketAddress address,
-                                   MockProxyClient proxyClient) {
-        clients.put(address, proxyClient);
-    }
-
-    @Override
-    public ProxyClient build(SocketAddress address) {
-        return clients.get(address);
-    }
-}
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockThriftClient.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockThriftClient.java
deleted file mode 100644
index 7877ed7..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockThriftClient.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.client.proxy;
-
-import com.twitter.finagle.Service;
-import com.twitter.finagle.thrift.ThriftClientRequest;
-import com.twitter.util.Future;
-
-/**
- * Mock Thrift Client.
- */
-class MockThriftClient extends Service<ThriftClientRequest, byte[]> {
-    @Override
-    public Future<byte[]> apply(ThriftClientRequest request) {
-        return Future.value(request.message);
-    }
-}
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/TestProxyClientManager.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/TestProxyClientManager.java
deleted file mode 100644
index 11e1e58..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/TestProxyClientManager.java
+++ /dev/null
@@ -1,368 +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 com.twitter.distributedlog.client.proxy;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Maps;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.client.ClientConfig;
-import com.twitter.distributedlog.client.proxy.MockDistributedLogServices.MockBasicService;
-import com.twitter.distributedlog.client.proxy.MockDistributedLogServices.MockServerInfoService;
-import com.twitter.distributedlog.client.proxy.MockProxyClientBuilder.MockProxyClient;
-import com.twitter.distributedlog.client.resolver.DefaultRegionResolver;
-import com.twitter.distributedlog.client.stats.ClientStats;
-import com.twitter.distributedlog.thrift.service.ServerInfo;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-import org.apache.commons.lang3.tuple.Pair;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-/**
- * Test Proxy Client Manager.
- */
-public class TestProxyClientManager {
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    static class TestHostProvider implements HostProvider {
-
-        Set<SocketAddress> hosts = new HashSet<SocketAddress>();
-
-        synchronized void addHost(SocketAddress host) {
-            hosts.add(host);
-        }
-
-        @Override
-        public synchronized Set<SocketAddress> getHosts() {
-            return ImmutableSet.copyOf(hosts);
-        }
-
-    }
-
-    private static ProxyClientManager createProxyClientManager(ProxyClient.Builder builder,
-                                                               long periodicHandshakeIntervalMs) {
-        HostProvider provider = new TestHostProvider();
-        return createProxyClientManager(builder, provider, periodicHandshakeIntervalMs);
-    }
-
-    private static ProxyClientManager createProxyClientManager(ProxyClient.Builder builder,
-                                                               HostProvider hostProvider,
-                                                               long periodicHandshakeIntervalMs) {
-        ClientConfig clientConfig = new ClientConfig();
-        clientConfig.setPeriodicHandshakeIntervalMs(periodicHandshakeIntervalMs);
-        clientConfig.setPeriodicOwnershipSyncIntervalMs(-1);
-        HashedWheelTimer dlTimer = new HashedWheelTimer(
-                new ThreadFactoryBuilder().setNameFormat("TestProxyClientManager-timer-%d").build(),
-                clientConfig.getRedirectBackoffStartMs(),
-                TimeUnit.MILLISECONDS);
-        return new ProxyClientManager(clientConfig, builder, dlTimer, hostProvider,
-                new ClientStats(NullStatsReceiver.get(), false, new DefaultRegionResolver()));
-    }
-
-    private static SocketAddress createSocketAddress(int port) {
-        return new InetSocketAddress("127.0.0.1", port);
-    }
-
-    private static MockProxyClient createMockProxyClient(SocketAddress address) {
-        return new MockProxyClient(address, new MockBasicService());
-    }
-
-    private static Pair<MockProxyClient, MockServerInfoService> createMockProxyClient(
-            SocketAddress address, ServerInfo serverInfo) {
-        MockServerInfoService service = new MockServerInfoService();
-        MockProxyClient proxyClient = new MockProxyClient(address, service);
-        service.updateServerInfo(serverInfo);
-        return Pair.of(proxyClient, service);
-    }
-
-    @Test(timeout = 60000)
-    public void testBasicCreateRemove() throws Exception {
-        SocketAddress address = createSocketAddress(1000);
-        MockProxyClientBuilder builder = new MockProxyClientBuilder();
-        MockProxyClient mockProxyClient = createMockProxyClient(address);
-        builder.provideProxyClient(address, mockProxyClient);
-
-        ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
-        assertEquals("There should be no clients in the manager",
-                0, clientManager.getNumProxies());
-        ProxyClient proxyClient =  clientManager.createClient(address);
-        assertEquals("Create client should build the proxy client",
-                1, clientManager.getNumProxies());
-        assertTrue("The client returned should be the same client that builder built",
-                mockProxyClient == proxyClient);
-    }
-
-    @Test(timeout = 60000)
-    public void testGetShouldCreateClient() throws Exception {
-        SocketAddress address = createSocketAddress(2000);
-        MockProxyClientBuilder builder = new MockProxyClientBuilder();
-        MockProxyClient mockProxyClient = createMockProxyClient(address);
-        builder.provideProxyClient(address, mockProxyClient);
-
-        ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
-        assertEquals("There should be no clients in the manager",
-                0, clientManager.getNumProxies());
-        ProxyClient proxyClient =  clientManager.getClient(address);
-        assertEquals("Get client should build the proxy client",
-                1, clientManager.getNumProxies());
-        assertTrue("The client returned should be the same client that builder built",
-                mockProxyClient == proxyClient);
-    }
-
-    @Test(timeout = 60000)
-    public void testConditionalRemoveClient() throws Exception {
-        SocketAddress address = createSocketAddress(3000);
-        MockProxyClientBuilder builder = new MockProxyClientBuilder();
-        MockProxyClient mockProxyClient = createMockProxyClient(address);
-        MockProxyClient anotherMockProxyClient = createMockProxyClient(address);
-        builder.provideProxyClient(address, mockProxyClient);
-
-        ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
-        assertEquals("There should be no clients in the manager",
-                0, clientManager.getNumProxies());
-        clientManager.createClient(address);
-        assertEquals("Create client should build the proxy client",
-                1, clientManager.getNumProxies());
-        clientManager.removeClient(address, anotherMockProxyClient);
-        assertEquals("Conditional remove should not remove proxy client",
-                1, clientManager.getNumProxies());
-        clientManager.removeClient(address, mockProxyClient);
-        assertEquals("Conditional remove should remove proxy client",
-                0, clientManager.getNumProxies());
-    }
-
-    @Test(timeout = 60000)
-    public void testRemoveClient() throws Exception {
-        SocketAddress address = createSocketAddress(3000);
-        MockProxyClientBuilder builder = new MockProxyClientBuilder();
-        MockProxyClient mockProxyClient = createMockProxyClient(address);
-        builder.provideProxyClient(address, mockProxyClient);
-
-        ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
-        assertEquals("There should be no clients in the manager",
-                0, clientManager.getNumProxies());
-        clientManager.createClient(address);
-        assertEquals("Create client should build the proxy client",
-                1, clientManager.getNumProxies());
-        clientManager.removeClient(address);
-        assertEquals("Remove should remove proxy client",
-                0, clientManager.getNumProxies());
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateClientShouldHandshake() throws Exception {
-        SocketAddress address = createSocketAddress(3000);
-        MockProxyClientBuilder builder = new MockProxyClientBuilder();
-        ServerInfo serverInfo = new ServerInfo();
-        serverInfo.putToOwnerships(runtime.getMethodName() + "_stream",
-                runtime.getMethodName() + "_owner");
-        Pair<MockProxyClient, MockServerInfoService> mockProxyClient =
-                createMockProxyClient(address, serverInfo);
-        builder.provideProxyClient(address, mockProxyClient.getLeft());
-
-        final AtomicReference<ServerInfo> resultHolder = new AtomicReference<ServerInfo>(null);
-        final CountDownLatch doneLatch = new CountDownLatch(1);
-        ProxyListener listener = new ProxyListener() {
-            @Override
-            public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
-                resultHolder.set(serverInfo);
-                doneLatch.countDown();
-            }
-            @Override
-            public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
-            }
-        };
-
-        ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
-        clientManager.registerProxyListener(listener);
-        assertEquals("There should be no clients in the manager",
-                0, clientManager.getNumProxies());
-        clientManager.createClient(address);
-        assertEquals("Create client should build the proxy client",
-                1, clientManager.getNumProxies());
-
-        // When a client is created, it would handshake with that proxy
-        doneLatch.await();
-        assertEquals("Handshake should return server info",
-                serverInfo, resultHolder.get());
-    }
-
-    @Test(timeout = 60000)
-    public void testHandshake() throws Exception {
-        final int numHosts = 3;
-        final int numStreamsPerHost = 3;
-        final int initialPort = 4000;
-
-        MockProxyClientBuilder builder = new MockProxyClientBuilder();
-        Map<SocketAddress, ServerInfo> serverInfoMap =
-                new HashMap<SocketAddress, ServerInfo>();
-        for (int i = 0; i < numHosts; i++) {
-            SocketAddress address = createSocketAddress(initialPort + i);
-            ServerInfo serverInfo = new ServerInfo();
-            for (int j = 0; j < numStreamsPerHost; j++) {
-                serverInfo.putToOwnerships(runtime.getMethodName() + "_stream_" + j,
-                        address.toString());
-            }
-            Pair<MockProxyClient, MockServerInfoService> mockProxyClient =
-                    createMockProxyClient(address, serverInfo);
-            builder.provideProxyClient(address, mockProxyClient.getLeft());
-            serverInfoMap.put(address, serverInfo);
-        }
-
-        final Map<SocketAddress, ServerInfo> results = new HashMap<SocketAddress, ServerInfo>();
-        final CountDownLatch doneLatch = new CountDownLatch(2 * numHosts);
-        ProxyListener listener = new ProxyListener() {
-            @Override
-            public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
-                synchronized (results) {
-                    results.put(address, serverInfo);
-                }
-                doneLatch.countDown();
-            }
-
-            @Override
-            public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
-            }
-        };
-
-        TestHostProvider rs = new TestHostProvider();
-        ProxyClientManager clientManager = createProxyClientManager(builder, rs, 0L);
-        clientManager.registerProxyListener(listener);
-        assertEquals("There should be no clients in the manager",
-                0, clientManager.getNumProxies());
-        for (int i = 0; i < numHosts; i++) {
-            rs.addHost(createSocketAddress(initialPort + i));
-        }
-        // handshake would handshake with 3 hosts again
-        clientManager.handshake();
-        doneLatch.await();
-        assertEquals("Handshake should return server info",
-                numHosts, results.size());
-        assertTrue("Handshake should get all server infos",
-                Maps.difference(serverInfoMap, results).areEqual());
-    }
-
-    @Test(timeout = 60000)
-    public void testPeriodicHandshake() throws Exception {
-        final int numHosts = 3;
-        final int numStreamsPerHost = 3;
-        final int initialPort = 5000;
-
-        MockProxyClientBuilder builder = new MockProxyClientBuilder();
-        Map<SocketAddress, ServerInfo> serverInfoMap =
-                new HashMap<SocketAddress, ServerInfo>();
-        Map<SocketAddress, MockServerInfoService> mockServiceMap =
-                new HashMap<SocketAddress, MockServerInfoService>();
-        final Map<SocketAddress, CountDownLatch> hostDoneLatches =
-                new HashMap<SocketAddress, CountDownLatch>();
-        for (int i = 0; i < numHosts; i++) {
-            SocketAddress address = createSocketAddress(initialPort + i);
-            ServerInfo serverInfo = new ServerInfo();
-            for (int j = 0; j < numStreamsPerHost; j++) {
-                serverInfo.putToOwnerships(runtime.getMethodName() + "_stream_" + j,
-                        address.toString());
-            }
-            Pair<MockProxyClient, MockServerInfoService> mockProxyClient =
-                    createMockProxyClient(address, serverInfo);
-            builder.provideProxyClient(address, mockProxyClient.getLeft());
-            serverInfoMap.put(address, serverInfo);
-            mockServiceMap.put(address, mockProxyClient.getRight());
-            hostDoneLatches.put(address, new CountDownLatch(2));
-        }
-
-        final Map<SocketAddress, ServerInfo> results = new HashMap<SocketAddress, ServerInfo>();
-        final CountDownLatch doneLatch = new CountDownLatch(numHosts);
-        ProxyListener listener = new ProxyListener() {
-            @Override
-            public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
-                synchronized (results) {
-                    results.put(address, serverInfo);
-                    CountDownLatch latch = hostDoneLatches.get(address);
-                    if (null != latch) {
-                        latch.countDown();
-                    }
-                }
-                doneLatch.countDown();
-            }
-
-            @Override
-            public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
-            }
-        };
-
-        TestHostProvider rs = new TestHostProvider();
-        ProxyClientManager clientManager = createProxyClientManager(builder, rs, 50L);
-        clientManager.setPeriodicHandshakeEnabled(false);
-        clientManager.registerProxyListener(listener);
-
-        assertEquals("There should be no clients in the manager",
-                0, clientManager.getNumProxies());
-        for (int i = 0; i < numHosts; i++) {
-            SocketAddress address = createSocketAddress(initialPort + i);
-            rs.addHost(address);
-            clientManager.createClient(address);
-        }
-
-        // make sure the first 3 handshakes going through
-        doneLatch.await();
-
-        assertEquals("Handshake should return server info",
-                numHosts, results.size());
-        assertTrue("Handshake should get all server infos",
-                Maps.difference(serverInfoMap, results).areEqual());
-
-        // update server info
-        for (int i = 0; i < numHosts; i++) {
-            SocketAddress address = createSocketAddress(initialPort + i);
-            ServerInfo serverInfo = new ServerInfo();
-            for (int j = 0; j < numStreamsPerHost; j++) {
-                serverInfo.putToOwnerships(runtime.getMethodName() + "_new_stream_" + j,
-                        address.toString());
-            }
-            MockServerInfoService service = mockServiceMap.get(address);
-            serverInfoMap.put(address, serverInfo);
-            service.updateServerInfo(serverInfo);
-        }
-
-        clientManager.setPeriodicHandshakeEnabled(true);
-        for (int i = 0; i < numHosts; i++) {
-            SocketAddress address = createSocketAddress(initialPort + i);
-            CountDownLatch latch = hostDoneLatches.get(address);
-            latch.await();
-        }
-
-        assertTrue("Periodic handshake should update all server infos",
-                Maps.difference(serverInfoMap, results).areEqual());
-    }
-
-}
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestConsistentHashRoutingService.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestConsistentHashRoutingService.java
deleted file mode 100644
index 0f4804c..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestConsistentHashRoutingService.java
+++ /dev/null
@@ -1,417 +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 com.twitter.distributedlog.client.routing;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import com.google.common.collect.ImmutableSet;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.client.resolver.DefaultRegionResolver;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.finagle.Address;
-import com.twitter.finagle.Addresses;
-import com.twitter.finagle.ChannelWriteException;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArrayList;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import org.junit.Test;
-
-/**
- * Test Case for {@link ConsistentHashRoutingService}.
- */
-public class TestConsistentHashRoutingService {
-
-    @Test(timeout = 60000)
-    public void testBlackoutHost() throws Exception {
-        TestName name = new TestName();
-        RoutingService routingService = ConsistentHashRoutingService.newBuilder()
-                .serverSet(new NameServerSet(name))
-                .resolveFromName(true)
-                .numReplicas(997)
-                .blackoutSeconds(2)
-                .build();
-
-        InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", 3181);
-        Address address = Addresses.newInetAddress(inetAddress);
-        List<Address> addresses = new ArrayList<Address>(1);
-        addresses.add(address);
-        name.changeAddrs(addresses);
-
-        routingService.startService();
-
-        RoutingService.RoutingContext routingContext =
-                RoutingService.RoutingContext.of(new DefaultRegionResolver());
-
-        String streamName = "test-blackout-host";
-        assertEquals(inetAddress, routingService.getHost(streamName, routingContext));
-        routingService.removeHost(inetAddress, new ChannelWriteException(new IOException("test exception")));
-        try {
-            routingService.getHost(streamName, routingContext);
-            fail("Should fail to get host since no brokers are available");
-        } catch (NoBrokersAvailableException nbae) {
-            // expected
-        }
-
-        TimeUnit.SECONDS.sleep(3);
-        assertEquals(inetAddress, routingService.getHost(streamName, routingContext));
-
-        routingService.stopService();
-    }
-
-    @Test(timeout = 60000)
-    public void testPerformServerSetChangeOnName() throws Exception {
-        TestName name = new TestName();
-        ConsistentHashRoutingService routingService = (ConsistentHashRoutingService)
-                ConsistentHashRoutingService.newBuilder()
-                        .serverSet(new NameServerSet(name))
-                        .resolveFromName(true)
-                        .numReplicas(997)
-                        .build();
-
-        int basePort = 3180;
-        int numHosts = 4;
-        List<Address> addresses1 = Lists.newArrayListWithExpectedSize(4);
-        List<Address> addresses2 = Lists.newArrayListWithExpectedSize(4);
-        List<Address> addresses3 = Lists.newArrayListWithExpectedSize(4);
-
-        // fill up the addresses1
-        for (int i = 0; i < numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
-            Address address = Addresses.newInetAddress(inetAddress);
-            addresses1.add(address);
-        }
-        // fill up the addresses2 - overlap with addresses1
-        for (int i = 0; i < numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 2 + i);
-            Address address = Addresses.newInetAddress(inetAddress);
-            addresses2.add(address);
-        }
-        // fill up the addresses3 - not overlap with addresses2
-        for (int i = 0; i < numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
-            Address address = Addresses.newInetAddress(inetAddress);
-            addresses3.add(address);
-        }
-
-        final List<SocketAddress> leftAddresses = Lists.newArrayList();
-        final List<SocketAddress> joinAddresses = Lists.newArrayList();
-
-        RoutingService.RoutingListener routingListener = new RoutingService.RoutingListener() {
-            @Override
-            public void onServerLeft(SocketAddress address) {
-                synchronized (leftAddresses) {
-                    leftAddresses.add(address);
-                    leftAddresses.notifyAll();
-                }
-            }
-
-            @Override
-            public void onServerJoin(SocketAddress address) {
-                synchronized (joinAddresses) {
-                    joinAddresses.add(address);
-                    joinAddresses.notifyAll();
-                }
-            }
-        };
-
-        routingService.registerListener(routingListener);
-        name.changeAddrs(addresses1);
-
-        routingService.startService();
-
-        synchronized (joinAddresses) {
-            while (joinAddresses.size() < numHosts) {
-                joinAddresses.wait();
-            }
-        }
-
-        // validate 4 nodes joined
-        synchronized (joinAddresses) {
-            assertEquals(numHosts, joinAddresses.size());
-        }
-        synchronized (leftAddresses) {
-            assertEquals(0, leftAddresses.size());
-        }
-        assertEquals(numHosts, routingService.shardId2Address.size());
-        assertEquals(numHosts, routingService.address2ShardId.size());
-        for (int i = 0; i < numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
-            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
-            int shardId = routingService.address2ShardId.get(inetAddress);
-            SocketAddress sa = routingService.shardId2Address.get(shardId);
-            assertNotNull(sa);
-            assertEquals(inetAddress, sa);
-        }
-
-        // update addresses2 - 2 new hosts joined, 2 old hosts left
-        name.changeAddrs(addresses2);
-        synchronized (joinAddresses) {
-            while (joinAddresses.size() < numHosts + 2) {
-                joinAddresses.wait();
-            }
-        }
-        synchronized (leftAddresses) {
-            while (leftAddresses.size() < numHosts - 2) {
-                leftAddresses.wait();
-            }
-        }
-        assertEquals(numHosts, routingService.shardId2Address.size());
-        assertEquals(numHosts, routingService.address2ShardId.size());
-
-        // first 2 shards should leave
-        for (int i = 0; i < 2; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
-            assertFalse(routingService.address2ShardId.containsKey(inetAddress));
-        }
-
-        for (int i = 0; i < numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 2 + i);
-            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
-            int shardId = routingService.address2ShardId.get(inetAddress);
-            SocketAddress sa = routingService.shardId2Address.get(shardId);
-            assertNotNull(sa);
-            assertEquals(inetAddress, sa);
-        }
-
-        // update addresses3 - 2 new hosts joined, 2 old hosts left
-        name.changeAddrs(addresses3);
-        synchronized (joinAddresses) {
-            while (joinAddresses.size() < numHosts + 2 + numHosts) {
-                joinAddresses.wait();
-            }
-        }
-        synchronized (leftAddresses) {
-            while (leftAddresses.size() < numHosts - 2 + numHosts) {
-                leftAddresses.wait();
-            }
-        }
-        assertEquals(numHosts, routingService.shardId2Address.size());
-        assertEquals(numHosts, routingService.address2ShardId.size());
-
-        // first 6 shards should leave
-        for (int i = 0; i < 2 + numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
-            assertFalse(routingService.address2ShardId.containsKey(inetAddress));
-        }
-        // new 4 shards should exist
-        for (int i = 0; i < numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
-            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
-            int shardId = routingService.address2ShardId.get(inetAddress);
-            SocketAddress sa = routingService.shardId2Address.get(shardId);
-            assertNotNull(sa);
-            assertEquals(inetAddress, sa);
-        }
-
-    }
-
-    private static class TestServerSetWatcher implements ServerSetWatcher {
-
-        final LinkedBlockingQueue<ImmutableSet<DLSocketAddress>> changeQueue =
-                new LinkedBlockingQueue<ImmutableSet<DLSocketAddress>>();
-        final CopyOnWriteArrayList<ServerSetMonitor> monitors =
-                new CopyOnWriteArrayList<ServerSetMonitor>();
-
-        @Override
-        public void watch(ServerSetMonitor monitor) throws MonitorException {
-            monitors.add(monitor);
-            ImmutableSet<DLSocketAddress> change;
-            while ((change = changeQueue.poll()) != null) {
-                notifyChanges(change);
-            }
-        }
-
-        void notifyChanges(ImmutableSet<DLSocketAddress> addresses) {
-            if (monitors.isEmpty()) {
-                changeQueue.add(addresses);
-            } else {
-                for (ServerSetMonitor monitor : monitors) {
-                    monitor.onChange(addresses);
-                }
-            }
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testPerformServerSetChangeOnServerSet() throws Exception {
-        TestServerSetWatcher serverSetWatcher = new TestServerSetWatcher();
-        ConsistentHashRoutingService routingService = new ConsistentHashRoutingService(
-                serverSetWatcher, 997, Integer.MAX_VALUE, NullStatsReceiver.get());
-
-        int basePort = 3180;
-        int numHosts = 4;
-        Set<DLSocketAddress> addresses1 = Sets.newConcurrentHashSet();
-        Set<DLSocketAddress> addresses2 = Sets.newConcurrentHashSet();
-        Set<DLSocketAddress> addresses3 = Sets.newConcurrentHashSet();
-
-        // fill up the addresses1
-        for (int i = 0; i < numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
-            DLSocketAddress dsa = new DLSocketAddress(i, inetAddress);
-            addresses1.add(dsa);
-        }
-        // fill up the addresses2 - overlap with addresses1
-        for (int i = 0; i < numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + numHosts + i);
-            DLSocketAddress dsa = new DLSocketAddress(i + 2, inetAddress);
-            addresses2.add(dsa);
-        }
-        // fill up the addresses3 - not overlap with addresses2
-        for (int i = 0; i < numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
-            DLSocketAddress dsa = new DLSocketAddress(i, inetAddress);
-            addresses3.add(dsa);
-        }
-
-        final List<SocketAddress> leftAddresses = Lists.newArrayList();
-        final List<SocketAddress> joinAddresses = Lists.newArrayList();
-
-        RoutingService.RoutingListener routingListener = new RoutingService.RoutingListener() {
-            @Override
-            public void onServerLeft(SocketAddress address) {
-                synchronized (leftAddresses) {
-                    leftAddresses.add(address);
-                    leftAddresses.notifyAll();
-                }
-            }
-
-            @Override
-            public void onServerJoin(SocketAddress address) {
-                synchronized (joinAddresses) {
-                    joinAddresses.add(address);
-                    joinAddresses.notifyAll();
-                }
-            }
-        };
-
-        routingService.registerListener(routingListener);
-        serverSetWatcher.notifyChanges(ImmutableSet.copyOf(addresses1));
-
-        routingService.startService();
-
-        synchronized (joinAddresses) {
-            while (joinAddresses.size() < numHosts) {
-                joinAddresses.wait();
-            }
-        }
-
-        // validate 4 nodes joined
-        synchronized (joinAddresses) {
-            assertEquals(numHosts, joinAddresses.size());
-        }
-        synchronized (leftAddresses) {
-            assertEquals(0, leftAddresses.size());
-        }
-        assertEquals(numHosts, routingService.shardId2Address.size());
-        assertEquals(numHosts, routingService.address2ShardId.size());
-        for (int i = 0; i < numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
-            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
-            int shardId = routingService.address2ShardId.get(inetAddress);
-            assertEquals(i, shardId);
-            SocketAddress sa = routingService.shardId2Address.get(shardId);
-            assertNotNull(sa);
-            assertEquals(inetAddress, sa);
-        }
-
-        // update addresses2 - 2 new hosts joined, 2 old hosts left
-        serverSetWatcher.notifyChanges(ImmutableSet.copyOf(addresses2));
-        synchronized (joinAddresses) {
-            while (joinAddresses.size() < numHosts + 2) {
-                joinAddresses.wait();
-            }
-        }
-        synchronized (leftAddresses) {
-            while (leftAddresses.size() < 2) {
-                leftAddresses.wait();
-            }
-        }
-
-        assertEquals(numHosts + 2, routingService.shardId2Address.size());
-        assertEquals(numHosts + 2, routingService.address2ShardId.size());
-        // first 2 shards should not leave
-        for (int i = 0; i < 2; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
-            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
-            int shardId = routingService.address2ShardId.get(inetAddress);
-            assertEquals(i, shardId);
-            SocketAddress sa = routingService.shardId2Address.get(shardId);
-            assertNotNull(sa);
-            assertEquals(inetAddress, sa);
-        }
-        for (int i = 0; i < numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + numHosts + i);
-            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
-            int shardId = routingService.address2ShardId.get(inetAddress);
-            assertEquals(i + 2, shardId);
-            SocketAddress sa = routingService.shardId2Address.get(shardId);
-            assertNotNull(sa);
-            assertEquals(inetAddress, sa);
-        }
-
-        // update addresses3
-        serverSetWatcher.notifyChanges(ImmutableSet.copyOf(addresses3));
-        synchronized (joinAddresses) {
-            while (joinAddresses.size() < numHosts + 2 + numHosts) {
-                joinAddresses.wait();
-            }
-        }
-        synchronized (leftAddresses) {
-            while (leftAddresses.size() < 2 + numHosts) {
-                leftAddresses.wait();
-            }
-        }
-        assertEquals(numHosts + 2, routingService.shardId2Address.size());
-        assertEquals(numHosts + 2, routingService.address2ShardId.size());
-
-        // first 4 shards should leave
-        for (int i = 0; i < numHosts; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
-            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
-            int shardId = routingService.address2ShardId.get(inetAddress);
-            assertEquals(i, shardId);
-            SocketAddress sa = routingService.shardId2Address.get(shardId);
-            assertNotNull(sa);
-            assertEquals(inetAddress, sa);
-        }
-        // the other 2 shards should be still there
-        for (int i = 0; i < 2; i++) {
-            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + numHosts + 2 + i);
-            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
-            int shardId = routingService.address2ShardId.get(inetAddress);
-            assertEquals(numHosts + i, shardId);
-            SocketAddress sa = routingService.shardId2Address.get(shardId);
-            assertNotNull(sa);
-            assertEquals(inetAddress, sa);
-        }
-
-    }
-}
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestInetNameResolution.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestInetNameResolution.java
deleted file mode 100644
index 2552f9e..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestInetNameResolution.java
+++ /dev/null
@@ -1,73 +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 com.twitter.distributedlog.client.routing;
-
-import com.google.common.collect.ImmutableSet;
-import com.twitter.common.net.pool.DynamicHostSet;
-import com.twitter.thrift.Endpoint;
-import com.twitter.thrift.ServiceInstance;
-import java.net.InetSocketAddress;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test Case for `inet` name resolution.
- */
-public class TestInetNameResolution {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestRoutingService.class);
-
-    @Test(timeout = 10000)
-    public void testInetNameResolution() throws Exception {
-        String nameStr = "inet!127.0.0.1:3181";
-        final CountDownLatch resolved = new CountDownLatch(1);
-        final AtomicBoolean validationFailed = new AtomicBoolean(false);
-
-        NameServerSet serverSet = new NameServerSet(nameStr);
-        serverSet.watch(new DynamicHostSet.HostChangeMonitor<ServiceInstance>() {
-            @Override
-            public void onChange(ImmutableSet<ServiceInstance> hostSet) {
-                if (hostSet.size() > 1) {
-                    logger.error("HostSet has more elements than expected {}", hostSet);
-                    validationFailed.set(true);
-                    resolved.countDown();
-                } else if (hostSet.size() == 1) {
-                    ServiceInstance serviceInstance = hostSet.iterator().next();
-                    Endpoint endpoint = serviceInstance.getAdditionalEndpoints().get("thrift");
-                    InetSocketAddress address = new InetSocketAddress(endpoint.getHost(), endpoint.getPort());
-                    if (endpoint.getPort() != 3181) {
-                        logger.error("Port does not match the expected port {}", endpoint.getPort());
-                        validationFailed.set(true);
-                    } else if (!address.getAddress().getHostAddress().equals("127.0.0.1")) {
-                        logger.error("Host address does not match the expected address {}",
-                            address.getAddress().getHostAddress());
-                        validationFailed.set(true);
-                    }
-                    resolved.countDown();
-                }
-            }
-        });
-
-        resolved.await();
-        Assert.assertEquals(false, validationFailed.get());
-    }
-}
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRegionsRoutingService.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRegionsRoutingService.java
deleted file mode 100644
index 49a375c..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRegionsRoutingService.java
+++ /dev/null
@@ -1,133 +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 com.twitter.distributedlog.client.routing;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.client.resolver.DefaultRegionResolver;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.finagle.NoBrokersAvailableException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.junit.Test;
-
-/**
- * Test Case for {@link RegionsRoutingService}.
- */
-public class TestRegionsRoutingService {
-
-    @Test(timeout = 60000)
-    public void testRoutingListener() throws Exception {
-        int numRoutingServices = 5;
-        RoutingService.Builder[] routingServiceBuilders = new RoutingService.Builder[numRoutingServices];
-        Set<SocketAddress> hosts = new HashSet<SocketAddress>();
-        Map<SocketAddress, String> regionMap = new HashMap<SocketAddress, String>();
-        for (int i = 0; i < numRoutingServices; i++) {
-            String finagleNameStr = "inet!127.0.0.1:" + (3181 + i);
-            routingServiceBuilders[i] = RoutingUtils.buildRoutingService(finagleNameStr);
-            SocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i);
-            hosts.add(address);
-            regionMap.put(address, "region-" + i);
-        }
-
-        final CountDownLatch doneLatch = new CountDownLatch(numRoutingServices);
-        final AtomicInteger numHostsLeft = new AtomicInteger(0);
-        final Set<SocketAddress> jointHosts = new HashSet<SocketAddress>();
-        RegionsRoutingService regionsRoutingService =
-                RegionsRoutingService.newBuilder()
-                    .routingServiceBuilders(routingServiceBuilders)
-                    .resolver(new DefaultRegionResolver(regionMap))
-                    .build();
-        regionsRoutingService.registerListener(new RoutingService.RoutingListener() {
-            @Override
-            public void onServerLeft(SocketAddress address) {
-                numHostsLeft.incrementAndGet();
-            }
-
-            @Override
-            public void onServerJoin(SocketAddress address) {
-                jointHosts.add(address);
-                doneLatch.countDown();
-            }
-        });
-
-        regionsRoutingService.startService();
-
-        doneLatch.await();
-
-        assertEquals(numRoutingServices, jointHosts.size());
-        assertEquals(0, numHostsLeft.get());
-        assertTrue(Sets.difference(hosts, jointHosts).immutableCopy().isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetHost() throws Exception {
-        int numRoutingServices = 3;
-        RoutingService.Builder[] routingServiceBuilders = new RoutingService.Builder[numRoutingServices];
-        Map<SocketAddress, String> regionMap = new HashMap<SocketAddress, String>();
-        for (int i = 0; i < numRoutingServices; i++) {
-            String finagleNameStr = "inet!127.0.0.1:" + (3181 + i);
-            routingServiceBuilders[i] = RoutingUtils.buildRoutingService(finagleNameStr);
-            SocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i);
-            regionMap.put(address, "region-" + i);
-        }
-
-        RegionsRoutingService regionsRoutingService =
-                RegionsRoutingService.newBuilder()
-                    .resolver(new DefaultRegionResolver(regionMap))
-                    .routingServiceBuilders(routingServiceBuilders)
-                    .build();
-        regionsRoutingService.startService();
-
-        RoutingService.RoutingContext routingContext =
-                RoutingService.RoutingContext.of(new DefaultRegionResolver())
-                        .addTriedHost(new InetSocketAddress("127.0.0.1", 3183), StatusCode.WRITE_EXCEPTION);
-        assertEquals(new InetSocketAddress("127.0.0.1", 3181),
-                regionsRoutingService.getHost("any", routingContext));
-
-        routingContext =
-                RoutingService.RoutingContext.of(new DefaultRegionResolver())
-                        .addTriedHost(new InetSocketAddress("127.0.0.1", 3181), StatusCode.WRITE_EXCEPTION);
-        assertEquals(new InetSocketAddress("127.0.0.1", 3182),
-                regionsRoutingService.getHost("any", routingContext));
-
-        // add 3182 to routing context as tried host
-        routingContext.addTriedHost(new InetSocketAddress("127.0.0.1", 3182), StatusCode.WRITE_EXCEPTION);
-        assertEquals(new InetSocketAddress("127.0.0.1", 3183),
-                regionsRoutingService.getHost("any", routingContext));
-
-        // add 3183 to routing context as tried host
-        routingContext.addTriedHost(new InetSocketAddress("127.0.0.1", 3183), StatusCode.WRITE_EXCEPTION);
-        try {
-            regionsRoutingService.getHost("any", routingContext);
-            fail("Should fail to get host since all regions are tried.");
-        } catch (NoBrokersAvailableException nbae) {
-            // expected
-        }
-    }
-
-}
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRoutingService.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRoutingService.java
deleted file mode 100644
index b79557e..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRoutingService.java
+++ /dev/null
@@ -1,146 +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 com.twitter.distributedlog.client.routing;
-
-import static org.junit.Assert.assertEquals;
-
-import com.twitter.distributedlog.client.resolver.DefaultRegionResolver;
-import com.twitter.finagle.Address;
-import com.twitter.finagle.Addresses;
-import com.twitter.finagle.addr.WeightedAddress;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test Case for {@link RoutingService}.
- */
-@RunWith(Parameterized.class)
-public class TestRoutingService {
-
-    static final Logger LOG = LoggerFactory.getLogger(TestRoutingService.class);
-
-    @Parameterized.Parameters
-    public static Collection<Object[]> configs() {
-        ArrayList<Object[]> list = new ArrayList<Object[]>();
-        for (int i = 0; i <= 1; i++) {
-            for (int j = 0; j <= 1; j++) {
-                for (int k = 0; k <= 1; k++) {
-                    list.add(new Boolean[] {i == 1, j == 1, k == 1});
-                }
-            }
-        }
-        return list;
-    }
-
-    private final boolean consistentHash;
-    private final boolean weightedAddresses;
-    private final boolean asyncResolution;
-
-    public TestRoutingService(boolean consistentHash, boolean weightedAddresses, boolean asyncResolution) {
-        this.consistentHash = consistentHash;
-        this.weightedAddresses = weightedAddresses;
-        this.asyncResolution = asyncResolution;
-    }
-
-    private List<Address> getAddresses(boolean weightedAddresses) {
-        ArrayList<Address> addresses = new ArrayList<Address>();
-        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.1", 3181)));
-        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.2", 3181)));
-        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.3", 3181)));
-        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.4", 3181)));
-        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.5", 3181)));
-        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.6", 3181)));
-        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.7", 3181)));
-
-        if (weightedAddresses) {
-            ArrayList<Address> wAddresses = new ArrayList<Address>();
-            for (Address address: addresses) {
-                wAddresses.add(WeightedAddress.apply(address, 1.0));
-            }
-            return wAddresses;
-        } else {
-            return addresses;
-        }
-    }
-
-    private void testRoutingServiceHelper(boolean consistentHash,
-                                          boolean weightedAddresses,
-                                          boolean asyncResolution)
-        throws Exception {
-        ExecutorService executorService = null;
-        final List<Address> addresses = getAddresses(weightedAddresses);
-        final TestName name = new TestName();
-        RoutingService routingService;
-        if (consistentHash) {
-            routingService = ConsistentHashRoutingService.newBuilder()
-                    .serverSet(new NameServerSet(name))
-                    .resolveFromName(true)
-                    .numReplicas(997)
-                    .build();
-        } else {
-            routingService = ServerSetRoutingService.newServerSetRoutingServiceBuilder()
-                    .serverSetWatcher(new TwitterServerSetWatcher(new NameServerSet(name), true)).build();
-        }
-
-        if (asyncResolution) {
-            executorService = Executors.newSingleThreadExecutor();
-            executorService.submit(new Runnable() {
-                @Override
-                public void run() {
-                    name.changeAddrs(addresses);
-                }
-            });
-        } else {
-            name.changeAddrs(addresses);
-        }
-        routingService.startService();
-
-        HashSet<SocketAddress> mapping = new HashSet<SocketAddress>();
-
-        for (int i = 0; i < 1000; i++) {
-            for (int j = 0; j < 5; j++) {
-                String stream = "TestStream-" + i + "-" + j;
-                mapping.add(routingService.getHost(stream,
-                        RoutingService.RoutingContext.of(new DefaultRegionResolver())));
-            }
-        }
-
-        assertEquals(mapping.size(), addresses.size());
-
-        if (null != executorService) {
-            executorService.shutdown();
-        }
-
-    }
-
-    @Test(timeout = 5000)
-    public void testRoutingService() throws Exception {
-        testRoutingServiceHelper(this.consistentHash, this.weightedAddresses, this.asyncResolution);
-    }
-}
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
deleted file mode 100644
index 71d0b01..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
+++ /dev/null
@@ -1,105 +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 com.twitter.distributedlog.client.speculative;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.mock;
-
-import com.twitter.util.CountDownLatch;
-import com.twitter.util.Future;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.junit.Test;
-import org.mockito.Mockito;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test {@link TestDefaultSpeculativeRequestExecutionPolicy}.
- */
-public class TestDefaultSpeculativeRequestExecutionPolicy {
-
-    @Test(timeout = 20000, expected = IllegalArgumentException.class)
-    public void testInvalidBackoffMultiplier() throws Exception {
-        new DefaultSpeculativeRequestExecutionPolicy(100, 200, -1);
-    }
-
-    @Test(timeout = 20000, expected = IllegalArgumentException.class)
-    public void testInvalidMaxSpeculativeTimeout() throws Exception {
-        new DefaultSpeculativeRequestExecutionPolicy(100, Integer.MAX_VALUE, 2);
-    }
-
-    @Test(timeout = 20000)
-    public void testSpeculativeRequests() throws Exception {
-        DefaultSpeculativeRequestExecutionPolicy policy =
-                new DefaultSpeculativeRequestExecutionPolicy(10, 10000, 2);
-        SpeculativeRequestExecutor executor = mock(SpeculativeRequestExecutor.class);
-
-        final AtomicInteger callCount = new AtomicInteger(0);
-        final CountDownLatch latch = new CountDownLatch(3);
-
-        Mockito.doAnswer(new Answer() {
-            @Override
-            public Object answer(InvocationOnMock invocation) throws Throwable {
-                try {
-                    return Future.value(callCount.incrementAndGet() < 3);
-                } finally {
-                    latch.countDown();
-                }
-            }
-        }).when(executor).issueSpeculativeRequest();
-
-        ScheduledExecutorService executorService =
-                Executors.newSingleThreadScheduledExecutor();
-        policy.initiateSpeculativeRequest(executorService, executor);
-
-        latch.await();
-
-        assertEquals(40, policy.getNextSpeculativeRequestTimeout());
-    }
-
-    @Test(timeout = 20000)
-    public void testSpeculativeRequestsWithMaxTimeout() throws Exception {
-        DefaultSpeculativeRequestExecutionPolicy policy =
-                new DefaultSpeculativeRequestExecutionPolicy(10, 15, 2);
-        SpeculativeRequestExecutor executor = mock(SpeculativeRequestExecutor.class);
-
-        final AtomicInteger callCount = new AtomicInteger(0);
-        final CountDownLatch latch = new CountDownLatch(3);
-
-        Mockito.doAnswer(new Answer() {
-            @Override
-            public Object answer(InvocationOnMock invocation) throws Throwable {
-                try {
-                    return Future.value(callCount.incrementAndGet() < 3);
-                } finally {
-                    latch.countDown();
-                }
-            }
-        }).when(executor).issueSpeculativeRequest();
-
-        ScheduledExecutorService executorService =
-                Executors.newSingleThreadScheduledExecutor();
-        policy.initiateSpeculativeRequest(executorService, executor);
-
-        latch.await();
-
-        assertEquals(15, policy.getNextSpeculativeRequestTimeout());
-    }
-}
diff --git a/distributedlog-client/src/test/java/com/twitter/distributedlog/service/TestDistributedLogClientBuilder.java b/distributedlog-client/src/test/java/com/twitter/distributedlog/service/TestDistributedLogClientBuilder.java
deleted file mode 100644
index 986cdd3..0000000
--- a/distributedlog-client/src/test/java/com/twitter/distributedlog/service/TestDistributedLogClientBuilder.java
+++ /dev/null
@@ -1,49 +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 com.twitter.distributedlog.service;
-
-import static org.junit.Assert.assertFalse;
-
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Duration;
-import org.junit.Test;
-
-/**
- * Test Case of {@link com.twitter.distributedlog.service.DistributedLogClientBuilder}.
- */
-public class TestDistributedLogClientBuilder {
-
-    @Test(timeout = 60000)
-    public void testBuildClientsFromSameBuilder() throws Exception {
-        DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
-                .name("build-clients-from-same-builder")
-                .clientId(ClientId$.MODULE$.apply("test-builder"))
-                .finagleNameStr("inet!127.0.0.1:7001")
-                .streamNameRegex(".*")
-                .handshakeWithClientInfo(true)
-                .clientBuilder(ClientBuilder.get()
-                    .hostConnectionLimit(1)
-                    .connectTimeout(Duration.fromSeconds(1))
-                    .tcpConnectTimeout(Duration.fromSeconds(1))
-                    .requestTimeout(Duration.fromSeconds(10)));
-        DistributedLogClient client1 = builder.build();
-        DistributedLogClient client2 = builder.build();
-        assertFalse(client1 == client2);
-    }
-}
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/TestDistributedLogMultiStreamWriter.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/TestDistributedLogMultiStreamWriter.java
new file mode 100644
index 0000000..d7494de
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/TestDistributedLogMultiStreamWriter.java
@@ -0,0 +1,383 @@
+/**
+ * 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.client;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Mockito.any;
+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 org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.LogRecordSet;
+import org.apache.distributedlog.LogRecordSetBuffer;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.service.DistributedLogClient;
+import com.twitter.finagle.IndividualRequestTimeoutException;
+import com.twitter.util.Await;
+import com.twitter.util.Future;
+import com.twitter.util.Promise;
+import java.nio.ByteBuffer;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Test {@link DistributedLogMultiStreamWriter}.
+ */
+public class TestDistributedLogMultiStreamWriter {
+
+    @Test(timeout = 20000, expected = IllegalArgumentException.class)
+    public void testBuildWithNullStreams() throws Exception {
+        DistributedLogMultiStreamWriter.newBuilder()
+                .build();
+    }
+
+    @Test(timeout = 20000, expected = IllegalArgumentException.class)
+    public void testBuildWithEmptyStreamList() throws Exception {
+        DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.<String>newArrayList())
+                .build();
+    }
+
+    @Test(timeout = 20000, expected = NullPointerException.class)
+    public void testBuildWithNullClient() throws Exception {
+        DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .build();
+    }
+
+    @Test(timeout = 20000, expected = NullPointerException.class)
+    public void testBuildWithNullCodec() throws Exception {
+        DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(mock(DistributedLogClient.class))
+                .compressionCodec(null)
+                .build();
+    }
+
+    @Test(timeout = 20000, expected = IllegalArgumentException.class)
+    public void testBuildWithInvalidSpeculativeSettings1()
+            throws Exception {
+        DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(mock(DistributedLogClient.class))
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(-1)
+                .build();
+    }
+
+    @Test(timeout = 20000, expected = IllegalArgumentException.class)
+    public void testBuildWithInvalidSpeculativeSettings2()
+            throws Exception {
+        DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(mock(DistributedLogClient.class))
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(10)
+                .maxSpeculativeTimeoutMs(5)
+                .build();
+    }
+
+    @Test(timeout = 20000, expected = IllegalArgumentException.class)
+    public void testBuildWithInvalidSpeculativeSettings3()
+            throws Exception {
+        DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(mock(DistributedLogClient.class))
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(10)
+                .maxSpeculativeTimeoutMs(20)
+                .speculativeBackoffMultiplier(-1)
+                .build();
+    }
+
+    @Test(timeout = 20000, expected = IllegalArgumentException.class)
+    public void testBuildWithInvalidSpeculativeSettings4()
+            throws Exception {
+        DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(mock(DistributedLogClient.class))
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(10)
+                .maxSpeculativeTimeoutMs(20)
+                .speculativeBackoffMultiplier(2)
+                .requestTimeoutMs(10)
+                .build();
+    }
+
+    @Test(timeout = 20000)
+    public void testBuildMultiStreamWriter()
+            throws Exception {
+        DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(mock(DistributedLogClient.class))
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(10)
+                .maxSpeculativeTimeoutMs(20)
+                .speculativeBackoffMultiplier(2)
+                .requestTimeoutMs(50)
+                .build();
+        assertTrue(true);
+    }
+
+    @Test(timeout = 20000)
+    public void testBuildWithPeriodicalFlushEnabled() throws Exception {
+        ScheduledExecutorService executorService = mock(ScheduledExecutorService.class);
+        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(mock(DistributedLogClient.class))
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(10)
+                .maxSpeculativeTimeoutMs(20)
+                .speculativeBackoffMultiplier(2)
+                .requestTimeoutMs(50)
+                .flushIntervalMs(1000)
+                .scheduler(executorService)
+                .build();
+        verify(executorService, times(1)).scheduleAtFixedRate(writer, 1000000, 1000000, TimeUnit.MICROSECONDS);
+    }
+
+    @Test(timeout = 20000)
+    public void testBuildWithPeriodicalFlushDisabled() throws Exception {
+        ScheduledExecutorService executorService = mock(ScheduledExecutorService.class);
+        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(mock(DistributedLogClient.class))
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(10)
+                .maxSpeculativeTimeoutMs(20)
+                .speculativeBackoffMultiplier(2)
+                .requestTimeoutMs(50)
+                .flushIntervalMs(0)
+                .scheduler(executorService)
+                .build();
+        verify(executorService, times(0)).scheduleAtFixedRate(writer, 1000, 1000, TimeUnit.MILLISECONDS);
+        writer.close();
+    }
+
+    @Test(timeout = 20000)
+    public void testFlushWhenBufferIsFull() throws Exception {
+        DistributedLogClient client = mock(DistributedLogClient.class);
+        when(client.writeRecordSet((String) any(), (LogRecordSetBuffer) any()))
+                .thenReturn(Future.value(new DLSN(1L, 1L, 999L)));
+
+        ScheduledExecutorService executorService =
+                Executors.newSingleThreadScheduledExecutor();
+        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(client)
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(100000)
+                .maxSpeculativeTimeoutMs(200000)
+                .speculativeBackoffMultiplier(2)
+                .requestTimeoutMs(500000)
+                .flushIntervalMs(0)
+                .bufferSize(0)
+                .scheduler(executorService)
+                .build();
+
+        ByteBuffer buffer = ByteBuffer.wrap("test".getBytes(UTF_8));
+        writer.write(buffer);
+
+        verify(client, times(1)).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
+
+        writer.close();
+    }
+
+    @Test(timeout = 20000)
+    public void testFlushWhenExceedMaxLogRecordSetSize()
+            throws Exception {
+        DistributedLogClient client = mock(DistributedLogClient.class);
+        when(client.writeRecordSet((String) any(), (LogRecordSetBuffer) any()))
+                .thenReturn(Future.value(new DLSN(1L, 1L, 999L)));
+        ScheduledExecutorService executorService =
+                Executors.newSingleThreadScheduledExecutor();
+        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(client)
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(100000)
+                .maxSpeculativeTimeoutMs(200000)
+                .speculativeBackoffMultiplier(2)
+                .requestTimeoutMs(500000)
+                .flushIntervalMs(0)
+                .bufferSize(Integer.MAX_VALUE)
+                .scheduler(executorService)
+                .build();
+
+        byte[] data = new byte[LogRecord.MAX_LOGRECORD_SIZE - 3 * 100];
+        ByteBuffer buffer1 = ByteBuffer.wrap(data);
+        writer.write(buffer1);
+        verify(client, times(0)).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
+        LogRecordSet.Writer recordSetWriter1 = writer.getLogRecordSetWriter();
+        assertEquals(1, recordSetWriter1.getNumRecords());
+        assertEquals(LogRecordSet.HEADER_LEN + 4 + data.length, recordSetWriter1.getNumBytes());
+
+        ByteBuffer buffer2 = ByteBuffer.wrap(data);
+        writer.write(buffer2);
+        verify(client, times(1)).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
+        LogRecordSet.Writer recordSetWriter2 = writer.getLogRecordSetWriter();
+        assertEquals(1, recordSetWriter2.getNumRecords());
+        assertEquals(LogRecordSet.HEADER_LEN + 4 + data.length, recordSetWriter2.getNumBytes());
+        assertTrue(recordSetWriter1 != recordSetWriter2);
+
+        writer.close();
+    }
+
+    @Test(timeout = 20000)
+    public void testWriteTooLargeRecord() throws Exception {
+        DistributedLogClient client = mock(DistributedLogClient.class);
+        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(client)
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(100000)
+                .maxSpeculativeTimeoutMs(200000)
+                .speculativeBackoffMultiplier(2)
+                .requestTimeoutMs(5000000)
+                .flushIntervalMs(0)
+                .bufferSize(0)
+                .build();
+
+        byte[] data = new byte[LogRecord.MAX_LOGRECORD_SIZE + 10];
+        ByteBuffer buffer = ByteBuffer.wrap(data);
+        Future<DLSN> writeFuture = writer.write(buffer);
+        assertTrue(writeFuture.isDefined());
+        try {
+            Await.result(writeFuture);
+            fail("Should fail on writing too long record");
+        } catch (LogRecordTooLongException lrtle) {
+            // expected
+        }
+        writer.close();
+    }
+
+    @Test(timeout = 20000)
+    public void testSpeculativeWrite() throws Exception {
+        DistributedLogClient client = mock(DistributedLogClient.class);
+        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(client)
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(10)
+                .maxSpeculativeTimeoutMs(20)
+                .speculativeBackoffMultiplier(2)
+                .requestTimeoutMs(5000000)
+                .flushIntervalMs(0)
+                .bufferSize(0)
+                .build();
+
+        final String secondStream = writer.getStream(1);
+
+        final DLSN dlsn = new DLSN(99L, 88L, 0L);
+
+        Mockito.doAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                Object[] arguments = invocation.getArguments();
+                String stream = (String) arguments[0];
+                if (stream.equals(secondStream)) {
+                    return Future.value(dlsn);
+                } else {
+                    return new Promise<DLSN>();
+                }
+            }
+        }).when(client).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
+
+        byte[] data = "test-test".getBytes(UTF_8);
+        ByteBuffer buffer = ByteBuffer.wrap(data);
+        Future<DLSN> writeFuture = writer.write(buffer);
+        DLSN writeDLSN = Await.result(writeFuture);
+        assertEquals(dlsn, writeDLSN);
+        writer.close();
+    }
+
+    @Test(timeout = 20000)
+    public void testPeriodicalFlush() throws Exception {
+        DistributedLogClient client = mock(DistributedLogClient.class);
+        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(client)
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(10)
+                .maxSpeculativeTimeoutMs(20)
+                .speculativeBackoffMultiplier(2)
+                .requestTimeoutMs(5000000)
+                .flushIntervalMs(10)
+                .bufferSize(Integer.MAX_VALUE)
+                .build();
+
+        final DLSN dlsn = new DLSN(99L, 88L, 0L);
+
+        Mockito.doAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                return Future.value(dlsn);
+            }
+        }).when(client).writeRecordSet((String) any(), (LogRecordSetBuffer) any());
+
+        byte[] data = "test-test".getBytes(UTF_8);
+        ByteBuffer buffer = ByteBuffer.wrap(data);
+        Future<DLSN> writeFuture = writer.write(buffer);
+        DLSN writeDLSN = Await.result(writeFuture);
+        assertEquals(dlsn, writeDLSN);
+        writer.close();
+    }
+
+    @Test(timeout = 20000)
+    public void testFailRequestAfterRetriedAllStreams() throws Exception {
+        DistributedLogClient client = mock(DistributedLogClient.class);
+        when(client.writeRecordSet((String) any(), (LogRecordSetBuffer) any()))
+                .thenReturn(new Promise<DLSN>());
+        DistributedLogMultiStreamWriter writer = DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList("stream1", "stream2"))
+                .client(client)
+                .compressionCodec(CompressionCodec.Type.LZ4)
+                .firstSpeculativeTimeoutMs(10)
+                .maxSpeculativeTimeoutMs(20)
+                .speculativeBackoffMultiplier(2)
+                .requestTimeoutMs(5000000)
+                .flushIntervalMs(10)
+                .bufferSize(Integer.MAX_VALUE)
+                .build();
+
+        byte[] data = "test-test".getBytes(UTF_8);
+        ByteBuffer buffer = ByteBuffer.wrap(data);
+        Future<DLSN> writeFuture = writer.write(buffer);
+        try {
+            Await.result(writeFuture);
+            fail("Should fail the request after retries all streams");
+        } catch (IndividualRequestTimeoutException e) {
+            long timeoutMs = e.timeout().inMilliseconds();
+            assertTrue(timeoutMs >= (10 + 20) && timeoutMs < 5000000);
+        }
+        writer.close();
+    }
+}
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/ownership/TestOwnershipCache.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/ownership/TestOwnershipCache.java
new file mode 100644
index 0000000..86d1c11
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/ownership/TestOwnershipCache.java
@@ -0,0 +1,207 @@
+/**
+ * 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.client.ownership;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.distributedlog.client.ClientConfig;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.Map;
+import java.util.Set;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Test Case for Ownership Cache.
+ */
+public class TestOwnershipCache {
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    private static OwnershipCache createOwnershipCache() {
+        ClientConfig clientConfig = new ClientConfig();
+        return new OwnershipCache(clientConfig, null,
+                                  NullStatsReceiver.get(), NullStatsReceiver.get());
+    }
+
+    private static SocketAddress createSocketAddress(int port) {
+        return new InetSocketAddress("127.0.0.1", port);
+    }
+
+    @Test(timeout = 60000)
+    public void testUpdateOwner() {
+        OwnershipCache cache = createOwnershipCache();
+        SocketAddress addr = createSocketAddress(1000);
+        String stream = runtime.getMethodName();
+
+        assertTrue("Should successfully update owner if no owner exists before",
+                cache.updateOwner(stream, addr));
+        assertEquals("Owner should be " + addr + " for stream " + stream,
+                addr, cache.getOwner(stream));
+        assertTrue("Should successfully update owner if old owner is same",
+                cache.updateOwner(stream, addr));
+        assertEquals("Owner should be " + addr + " for stream " + stream,
+                addr, cache.getOwner(stream));
+    }
+
+    @Test(timeout = 60000)
+    public void testRemoveOwnerFromStream() {
+        OwnershipCache cache = createOwnershipCache();
+        int initialPort = 2000;
+        int numProxies = 2;
+        int numStreamsPerProxy = 2;
+        for (int i = 0; i < numProxies; i++) {
+            SocketAddress addr = createSocketAddress(initialPort + i);
+            for (int j = 0; j < numStreamsPerProxy; j++) {
+                String stream = runtime.getMethodName() + "_" + i + "_" + j;
+                cache.updateOwner(stream, addr);
+            }
+        }
+        Map<String, SocketAddress> ownershipMap = cache.getStreamOwnerMapping();
+        assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
+                numProxies * numStreamsPerProxy, ownershipMap.size());
+        Map<SocketAddress, Set<String>> ownershipDistribution = cache.getStreamOwnershipDistribution();
+        assertEquals("There should be " + numProxies + " proxies cached",
+                numProxies, ownershipDistribution.size());
+
+        String stream = runtime.getMethodName() + "_0_0";
+        SocketAddress owner = createSocketAddress(initialPort);
+
+        // remove non-existent mapping won't change anything
+        SocketAddress nonExistentAddr = createSocketAddress(initialPort + 999);
+        cache.removeOwnerFromStream(stream, nonExistentAddr, "remove-non-existent-addr");
+        assertEquals("Owner " + owner + " should not be removed",
+                owner, cache.getOwner(stream));
+        ownershipMap = cache.getStreamOwnerMapping();
+        assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
+                numProxies * numStreamsPerProxy, ownershipMap.size());
+
+        // remove existent mapping should remove ownership mapping
+        cache.removeOwnerFromStream(stream, owner, "remove-owner");
+        assertNull("Owner " + owner + " should be removed", cache.getOwner(stream));
+        ownershipMap = cache.getStreamOwnerMapping();
+        assertEquals("There should be " + (numProxies * numStreamsPerProxy - 1) + " entries left in cache",
+                numProxies * numStreamsPerProxy - 1, ownershipMap.size());
+        ownershipDistribution = cache.getStreamOwnershipDistribution();
+        assertEquals("There should still be " + numProxies + " proxies cached",
+                numProxies, ownershipDistribution.size());
+        Set<String> ownedStreams = ownershipDistribution.get(owner);
+        assertEquals("There should be only " + (numStreamsPerProxy - 1) + " streams owned for " + owner,
+                numStreamsPerProxy - 1, ownedStreams.size());
+        assertFalse("Stream " + stream + " should not be owned by " + owner,
+                ownedStreams.contains(stream));
+    }
+
+    @Test(timeout = 60000)
+    public void testRemoveAllStreamsFromOwner() {
+        OwnershipCache cache = createOwnershipCache();
+        int initialPort = 2000;
+        int numProxies = 2;
+        int numStreamsPerProxy = 2;
+        for (int i = 0; i < numProxies; i++) {
+            SocketAddress addr = createSocketAddress(initialPort + i);
+            for (int j = 0; j < numStreamsPerProxy; j++) {
+                String stream = runtime.getMethodName() + "_" + i + "_" + j;
+                cache.updateOwner(stream, addr);
+            }
+        }
+        Map<String, SocketAddress> ownershipMap = cache.getStreamOwnerMapping();
+        assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
+                numProxies * numStreamsPerProxy, ownershipMap.size());
+        Map<SocketAddress, Set<String>> ownershipDistribution = cache.getStreamOwnershipDistribution();
+        assertEquals("There should be " + numProxies + " proxies cached",
+                numProxies, ownershipDistribution.size());
+
+        SocketAddress owner = createSocketAddress(initialPort);
+
+        // remove non-existent host won't change anything
+        SocketAddress nonExistentAddr = createSocketAddress(initialPort + 999);
+        cache.removeAllStreamsFromOwner(nonExistentAddr);
+        ownershipMap = cache.getStreamOwnerMapping();
+        assertEquals("There should still be " + (numProxies * numStreamsPerProxy) + " entries in cache",
+                numProxies * numStreamsPerProxy, ownershipMap.size());
+        ownershipDistribution = cache.getStreamOwnershipDistribution();
+        assertEquals("There should still be " + numProxies + " proxies cached",
+                numProxies, ownershipDistribution.size());
+
+        // remove existent host should remove ownership mapping
+        cache.removeAllStreamsFromOwner(owner);
+        ownershipMap = cache.getStreamOwnerMapping();
+        assertEquals("There should be " + ((numProxies - 1) * numStreamsPerProxy) + " entries left in cache",
+                (numProxies - 1) * numStreamsPerProxy, ownershipMap.size());
+        ownershipDistribution = cache.getStreamOwnershipDistribution();
+        assertEquals("There should be " + (numProxies - 1) + " proxies cached",
+                numProxies - 1, ownershipDistribution.size());
+        assertFalse("Host " + owner + " should not be cached",
+                ownershipDistribution.containsKey(owner));
+    }
+
+    @Test(timeout = 60000)
+    public void testReplaceOwner() {
+        OwnershipCache cache = createOwnershipCache();
+        int initialPort = 2000;
+        int numProxies = 2;
+        int numStreamsPerProxy = 2;
+        for (int i = 0; i < numProxies; i++) {
+            SocketAddress addr = createSocketAddress(initialPort + i);
+            for (int j = 0; j < numStreamsPerProxy; j++) {
+                String stream = runtime.getMethodName() + "_" + i + "_" + j;
+                cache.updateOwner(stream, addr);
+            }
+        }
+        Map<String, SocketAddress> ownershipMap = cache.getStreamOwnerMapping();
+        assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
+                numProxies * numStreamsPerProxy, ownershipMap.size());
+        Map<SocketAddress, Set<String>> ownershipDistribution = cache.getStreamOwnershipDistribution();
+        assertEquals("There should be " + numProxies + " proxies cached",
+                numProxies, ownershipDistribution.size());
+
+        String stream = runtime.getMethodName() + "_0_0";
+        SocketAddress oldOwner = createSocketAddress(initialPort);
+        SocketAddress newOwner = createSocketAddress(initialPort + 999);
+
+        cache.updateOwner(stream, newOwner);
+        assertEquals("Owner of " + stream + " should be changed from " + oldOwner + " to " + newOwner,
+                newOwner, cache.getOwner(stream));
+        ownershipMap = cache.getStreamOwnerMapping();
+        assertEquals("There should be " + (numProxies * numStreamsPerProxy) + " entries in cache",
+                numProxies * numStreamsPerProxy, ownershipMap.size());
+        assertEquals("Owner of " + stream + " should be " + newOwner,
+                newOwner, ownershipMap.get(stream));
+        ownershipDistribution = cache.getStreamOwnershipDistribution();
+        assertEquals("There should be " + (numProxies + 1) + " proxies cached",
+                numProxies + 1, ownershipDistribution.size());
+        Set<String> oldOwnedStreams = ownershipDistribution.get(oldOwner);
+        assertEquals("There should be only " + (numStreamsPerProxy - 1) + " streams owned by " + oldOwner,
+                numStreamsPerProxy - 1, oldOwnedStreams.size());
+        assertFalse("Stream " + stream + " should not be owned by " + oldOwner,
+                oldOwnedStreams.contains(stream));
+        Set<String> newOwnedStreams = ownershipDistribution.get(newOwner);
+        assertEquals("There should be only " + (numStreamsPerProxy - 1) + " streams owned by " + newOwner,
+                1, newOwnedStreams.size());
+        assertTrue("Stream " + stream + " should be owned by " + newOwner,
+                newOwnedStreams.contains(stream));
+    }
+}
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockDistributedLogServices.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockDistributedLogServices.java
new file mode 100644
index 0000000..8ef33bd
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockDistributedLogServices.java
@@ -0,0 +1,144 @@
+/**
+ * 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.client.proxy;
+
+import org.apache.distributedlog.thrift.service.BulkWriteResponse;
+import org.apache.distributedlog.thrift.service.ClientInfo;
+import org.apache.distributedlog.thrift.service.DistributedLogService;
+import org.apache.distributedlog.thrift.service.HeartbeatOptions;
+import org.apache.distributedlog.thrift.service.ServerInfo;
+import org.apache.distributedlog.thrift.service.WriteContext;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import com.twitter.util.Future;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+/**
+ * Mock DistributedLog Related Services.
+ */
+public class MockDistributedLogServices {
+
+    /**
+     * Mock basic service.
+     */
+    static class MockBasicService implements DistributedLogService.ServiceIface {
+
+        @Override
+        public Future<ServerInfo> handshake() {
+            return Future.value(new ServerInfo());
+        }
+
+        @Override
+        public Future<ServerInfo> handshakeWithClientInfo(ClientInfo clientInfo) {
+            return Future.value(new ServerInfo());
+        }
+
+        @Override
+        public Future<WriteResponse> heartbeat(String stream, WriteContext ctx) {
+            return Future.value(new WriteResponse());
+        }
+
+        @Override
+        public Future<WriteResponse> heartbeatWithOptions(String stream,
+                                                          WriteContext ctx,
+                                                          HeartbeatOptions options) {
+            return Future.value(new WriteResponse());
+        }
+
+        @Override
+        public Future<WriteResponse> write(String stream,
+                                           ByteBuffer data) {
+            return Future.value(new WriteResponse());
+        }
+
+        @Override
+        public Future<WriteResponse> writeWithContext(String stream,
+                                                      ByteBuffer data,
+                                                      WriteContext ctx) {
+            return Future.value(new WriteResponse());
+        }
+
+        @Override
+        public Future<BulkWriteResponse> writeBulkWithContext(String stream,
+                                                              List<ByteBuffer> data,
+                                                              WriteContext ctx) {
+            return Future.value(new BulkWriteResponse());
+        }
+
+        @Override
+        public Future<WriteResponse> truncate(String stream,
+                                              String dlsn,
+                                              WriteContext ctx) {
+            return Future.value(new WriteResponse());
+        }
+
+        @Override
+        public Future<WriteResponse> release(String stream,
+                                             WriteContext ctx) {
+            return Future.value(new WriteResponse());
+        }
+
+        @Override
+        public Future<WriteResponse> create(String stream, WriteContext ctx) {
+            return Future.value(new WriteResponse());
+        }
+
+        @Override
+        public Future<WriteResponse> delete(String stream,
+                                            WriteContext ctx) {
+            return Future.value(new WriteResponse());
+        }
+
+        @Override
+        public Future<WriteResponse> getOwner(String stream, WriteContext ctx) {
+            return Future.value(new WriteResponse());
+        }
+
+        @Override
+        public Future<Void> setAcceptNewStream(boolean enabled) {
+            return Future.value(null);
+        }
+    }
+
+    /**
+     * Mock server info service.
+     */
+    public static class MockServerInfoService extends MockBasicService {
+
+        protected ServerInfo serverInfo;
+
+        public MockServerInfoService() {
+            serverInfo = new ServerInfo();
+        }
+
+        public void updateServerInfo(ServerInfo serverInfo) {
+            this.serverInfo = serverInfo;
+        }
+
+        @Override
+        public Future<ServerInfo> handshake() {
+            return Future.value(serverInfo);
+        }
+
+        @Override
+        public Future<ServerInfo> handshakeWithClientInfo(ClientInfo clientInfo) {
+            return Future.value(serverInfo);
+        }
+    }
+
+}
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockProxyClientBuilder.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockProxyClientBuilder.java
new file mode 100644
index 0000000..e38c2ed
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockProxyClientBuilder.java
@@ -0,0 +1,49 @@
+/**
+ * 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.client.proxy;
+
+import org.apache.distributedlog.thrift.service.DistributedLogService;
+import java.net.SocketAddress;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Mock Proxy Client Builder.
+ */
+class MockProxyClientBuilder implements ProxyClient.Builder {
+
+    static class MockProxyClient extends ProxyClient {
+        MockProxyClient(SocketAddress address,
+                        DistributedLogService.ServiceIface service) {
+            super(address, new MockThriftClient(), service);
+        }
+    }
+
+    private final ConcurrentMap<SocketAddress, MockProxyClient> clients =
+            new ConcurrentHashMap<SocketAddress, MockProxyClient>();
+
+    public void provideProxyClient(SocketAddress address,
+                                   MockProxyClient proxyClient) {
+        clients.put(address, proxyClient);
+    }
+
+    @Override
+    public ProxyClient build(SocketAddress address) {
+        return clients.get(address);
+    }
+}
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockThriftClient.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockThriftClient.java
new file mode 100644
index 0000000..ad1c878
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockThriftClient.java
@@ -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.
+ */
+package org.apache.distributedlog.client.proxy;
+
+import com.twitter.finagle.Service;
+import com.twitter.finagle.thrift.ThriftClientRequest;
+import com.twitter.util.Future;
+
+/**
+ * Mock Thrift Client.
+ */
+class MockThriftClient extends Service<ThriftClientRequest, byte[]> {
+    @Override
+    public Future<byte[]> apply(ThriftClientRequest request) {
+        return Future.value(request.message);
+    }
+}
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/TestProxyClientManager.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/TestProxyClientManager.java
new file mode 100644
index 0000000..6d9a471
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/TestProxyClientManager.java
@@ -0,0 +1,368 @@
+/**
+ * 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.client.proxy;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Maps;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.distributedlog.client.ClientConfig;
+import org.apache.distributedlog.client.proxy.MockDistributedLogServices.MockBasicService;
+import org.apache.distributedlog.client.proxy.MockDistributedLogServices.MockServerInfoService;
+import org.apache.distributedlog.client.proxy.MockProxyClientBuilder.MockProxyClient;
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import org.apache.distributedlog.client.stats.ClientStats;
+import org.apache.distributedlog.thrift.service.ServerInfo;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+import org.apache.commons.lang3.tuple.Pair;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Test Proxy Client Manager.
+ */
+public class TestProxyClientManager {
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    static class TestHostProvider implements HostProvider {
+
+        Set<SocketAddress> hosts = new HashSet<SocketAddress>();
+
+        synchronized void addHost(SocketAddress host) {
+            hosts.add(host);
+        }
+
+        @Override
+        public synchronized Set<SocketAddress> getHosts() {
+            return ImmutableSet.copyOf(hosts);
+        }
+
+    }
+
+    private static ProxyClientManager createProxyClientManager(ProxyClient.Builder builder,
+                                                               long periodicHandshakeIntervalMs) {
+        HostProvider provider = new TestHostProvider();
+        return createProxyClientManager(builder, provider, periodicHandshakeIntervalMs);
+    }
+
+    private static ProxyClientManager createProxyClientManager(ProxyClient.Builder builder,
+                                                               HostProvider hostProvider,
+                                                               long periodicHandshakeIntervalMs) {
+        ClientConfig clientConfig = new ClientConfig();
+        clientConfig.setPeriodicHandshakeIntervalMs(periodicHandshakeIntervalMs);
+        clientConfig.setPeriodicOwnershipSyncIntervalMs(-1);
+        HashedWheelTimer dlTimer = new HashedWheelTimer(
+                new ThreadFactoryBuilder().setNameFormat("TestProxyClientManager-timer-%d").build(),
+                clientConfig.getRedirectBackoffStartMs(),
+                TimeUnit.MILLISECONDS);
+        return new ProxyClientManager(clientConfig, builder, dlTimer, hostProvider,
+                new ClientStats(NullStatsReceiver.get(), false, new DefaultRegionResolver()));
+    }
+
+    private static SocketAddress createSocketAddress(int port) {
+        return new InetSocketAddress("127.0.0.1", port);
+    }
+
+    private static MockProxyClient createMockProxyClient(SocketAddress address) {
+        return new MockProxyClient(address, new MockBasicService());
+    }
+
+    private static Pair<MockProxyClient, MockServerInfoService> createMockProxyClient(
+            SocketAddress address, ServerInfo serverInfo) {
+        MockServerInfoService service = new MockServerInfoService();
+        MockProxyClient proxyClient = new MockProxyClient(address, service);
+        service.updateServerInfo(serverInfo);
+        return Pair.of(proxyClient, service);
+    }
+
+    @Test(timeout = 60000)
+    public void testBasicCreateRemove() throws Exception {
+        SocketAddress address = createSocketAddress(1000);
+        MockProxyClientBuilder builder = new MockProxyClientBuilder();
+        MockProxyClient mockProxyClient = createMockProxyClient(address);
+        builder.provideProxyClient(address, mockProxyClient);
+
+        ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
+        assertEquals("There should be no clients in the manager",
+                0, clientManager.getNumProxies());
+        ProxyClient proxyClient =  clientManager.createClient(address);
+        assertEquals("Create client should build the proxy client",
+                1, clientManager.getNumProxies());
+        assertTrue("The client returned should be the same client that builder built",
+                mockProxyClient == proxyClient);
+    }
+
+    @Test(timeout = 60000)
+    public void testGetShouldCreateClient() throws Exception {
+        SocketAddress address = createSocketAddress(2000);
+        MockProxyClientBuilder builder = new MockProxyClientBuilder();
+        MockProxyClient mockProxyClient = createMockProxyClient(address);
+        builder.provideProxyClient(address, mockProxyClient);
+
+        ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
+        assertEquals("There should be no clients in the manager",
+                0, clientManager.getNumProxies());
+        ProxyClient proxyClient =  clientManager.getClient(address);
+        assertEquals("Get client should build the proxy client",
+                1, clientManager.getNumProxies());
+        assertTrue("The client returned should be the same client that builder built",
+                mockProxyClient == proxyClient);
+    }
+
+    @Test(timeout = 60000)
+    public void testConditionalRemoveClient() throws Exception {
+        SocketAddress address = createSocketAddress(3000);
+        MockProxyClientBuilder builder = new MockProxyClientBuilder();
+        MockProxyClient mockProxyClient = createMockProxyClient(address);
+        MockProxyClient anotherMockProxyClient = createMockProxyClient(address);
+        builder.provideProxyClient(address, mockProxyClient);
+
+        ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
+        assertEquals("There should be no clients in the manager",
+                0, clientManager.getNumProxies());
+        clientManager.createClient(address);
+        assertEquals("Create client should build the proxy client",
+                1, clientManager.getNumProxies());
+        clientManager.removeClient(address, anotherMockProxyClient);
+        assertEquals("Conditional remove should not remove proxy client",
+                1, clientManager.getNumProxies());
+        clientManager.removeClient(address, mockProxyClient);
+        assertEquals("Conditional remove should remove proxy client",
+                0, clientManager.getNumProxies());
+    }
+
+    @Test(timeout = 60000)
+    public void testRemoveClient() throws Exception {
+        SocketAddress address = createSocketAddress(3000);
+        MockProxyClientBuilder builder = new MockProxyClientBuilder();
+        MockProxyClient mockProxyClient = createMockProxyClient(address);
+        builder.provideProxyClient(address, mockProxyClient);
+
+        ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
+        assertEquals("There should be no clients in the manager",
+                0, clientManager.getNumProxies());
+        clientManager.createClient(address);
+        assertEquals("Create client should build the proxy client",
+                1, clientManager.getNumProxies());
+        clientManager.removeClient(address);
+        assertEquals("Remove should remove proxy client",
+                0, clientManager.getNumProxies());
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateClientShouldHandshake() throws Exception {
+        SocketAddress address = createSocketAddress(3000);
+        MockProxyClientBuilder builder = new MockProxyClientBuilder();
+        ServerInfo serverInfo = new ServerInfo();
+        serverInfo.putToOwnerships(runtime.getMethodName() + "_stream",
+                runtime.getMethodName() + "_owner");
+        Pair<MockProxyClient, MockServerInfoService> mockProxyClient =
+                createMockProxyClient(address, serverInfo);
+        builder.provideProxyClient(address, mockProxyClient.getLeft());
+
+        final AtomicReference<ServerInfo> resultHolder = new AtomicReference<ServerInfo>(null);
+        final CountDownLatch doneLatch = new CountDownLatch(1);
+        ProxyListener listener = new ProxyListener() {
+            @Override
+            public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
+                resultHolder.set(serverInfo);
+                doneLatch.countDown();
+            }
+            @Override
+            public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
+            }
+        };
+
+        ProxyClientManager clientManager = createProxyClientManager(builder, 0L);
+        clientManager.registerProxyListener(listener);
+        assertEquals("There should be no clients in the manager",
+                0, clientManager.getNumProxies());
+        clientManager.createClient(address);
+        assertEquals("Create client should build the proxy client",
+                1, clientManager.getNumProxies());
+
+        // When a client is created, it would handshake with that proxy
+        doneLatch.await();
+        assertEquals("Handshake should return server info",
+                serverInfo, resultHolder.get());
+    }
+
+    @Test(timeout = 60000)
+    public void testHandshake() throws Exception {
+        final int numHosts = 3;
+        final int numStreamsPerHost = 3;
+        final int initialPort = 4000;
+
+        MockProxyClientBuilder builder = new MockProxyClientBuilder();
+        Map<SocketAddress, ServerInfo> serverInfoMap =
+                new HashMap<SocketAddress, ServerInfo>();
+        for (int i = 0; i < numHosts; i++) {
+            SocketAddress address = createSocketAddress(initialPort + i);
+            ServerInfo serverInfo = new ServerInfo();
+            for (int j = 0; j < numStreamsPerHost; j++) {
+                serverInfo.putToOwnerships(runtime.getMethodName() + "_stream_" + j,
+                        address.toString());
+            }
+            Pair<MockProxyClient, MockServerInfoService> mockProxyClient =
+                    createMockProxyClient(address, serverInfo);
+            builder.provideProxyClient(address, mockProxyClient.getLeft());
+            serverInfoMap.put(address, serverInfo);
+        }
+
+        final Map<SocketAddress, ServerInfo> results = new HashMap<SocketAddress, ServerInfo>();
+        final CountDownLatch doneLatch = new CountDownLatch(2 * numHosts);
+        ProxyListener listener = new ProxyListener() {
+            @Override
+            public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
+                synchronized (results) {
+                    results.put(address, serverInfo);
+                }
+                doneLatch.countDown();
+            }
+
+            @Override
+            public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
+            }
+        };
+
+        TestHostProvider rs = new TestHostProvider();
+        ProxyClientManager clientManager = createProxyClientManager(builder, rs, 0L);
+        clientManager.registerProxyListener(listener);
+        assertEquals("There should be no clients in the manager",
+                0, clientManager.getNumProxies());
+        for (int i = 0; i < numHosts; i++) {
+            rs.addHost(createSocketAddress(initialPort + i));
+        }
+        // handshake would handshake with 3 hosts again
+        clientManager.handshake();
+        doneLatch.await();
+        assertEquals("Handshake should return server info",
+                numHosts, results.size());
+        assertTrue("Handshake should get all server infos",
+                Maps.difference(serverInfoMap, results).areEqual());
+    }
+
+    @Test(timeout = 60000)
+    public void testPeriodicHandshake() throws Exception {
+        final int numHosts = 3;
+        final int numStreamsPerHost = 3;
+        final int initialPort = 5000;
+
+        MockProxyClientBuilder builder = new MockProxyClientBuilder();
+        Map<SocketAddress, ServerInfo> serverInfoMap =
+                new HashMap<SocketAddress, ServerInfo>();
+        Map<SocketAddress, MockServerInfoService> mockServiceMap =
+                new HashMap<SocketAddress, MockServerInfoService>();
+        final Map<SocketAddress, CountDownLatch> hostDoneLatches =
+                new HashMap<SocketAddress, CountDownLatch>();
+        for (int i = 0; i < numHosts; i++) {
+            SocketAddress address = createSocketAddress(initialPort + i);
+            ServerInfo serverInfo = new ServerInfo();
+            for (int j = 0; j < numStreamsPerHost; j++) {
+                serverInfo.putToOwnerships(runtime.getMethodName() + "_stream_" + j,
+                        address.toString());
+            }
+            Pair<MockProxyClient, MockServerInfoService> mockProxyClient =
+                    createMockProxyClient(address, serverInfo);
+            builder.provideProxyClient(address, mockProxyClient.getLeft());
+            serverInfoMap.put(address, serverInfo);
+            mockServiceMap.put(address, mockProxyClient.getRight());
+            hostDoneLatches.put(address, new CountDownLatch(2));
+        }
+
+        final Map<SocketAddress, ServerInfo> results = new HashMap<SocketAddress, ServerInfo>();
+        final CountDownLatch doneLatch = new CountDownLatch(numHosts);
+        ProxyListener listener = new ProxyListener() {
+            @Override
+            public void onHandshakeSuccess(SocketAddress address, ProxyClient client, ServerInfo serverInfo) {
+                synchronized (results) {
+                    results.put(address, serverInfo);
+                    CountDownLatch latch = hostDoneLatches.get(address);
+                    if (null != latch) {
+                        latch.countDown();
+                    }
+                }
+                doneLatch.countDown();
+            }
+
+            @Override
+            public void onHandshakeFailure(SocketAddress address, ProxyClient client, Throwable cause) {
+            }
+        };
+
+        TestHostProvider rs = new TestHostProvider();
+        ProxyClientManager clientManager = createProxyClientManager(builder, rs, 50L);
+        clientManager.setPeriodicHandshakeEnabled(false);
+        clientManager.registerProxyListener(listener);
+
+        assertEquals("There should be no clients in the manager",
+                0, clientManager.getNumProxies());
+        for (int i = 0; i < numHosts; i++) {
+            SocketAddress address = createSocketAddress(initialPort + i);
+            rs.addHost(address);
+            clientManager.createClient(address);
+        }
+
+        // make sure the first 3 handshakes going through
+        doneLatch.await();
+
+        assertEquals("Handshake should return server info",
+                numHosts, results.size());
+        assertTrue("Handshake should get all server infos",
+                Maps.difference(serverInfoMap, results).areEqual());
+
+        // update server info
+        for (int i = 0; i < numHosts; i++) {
+            SocketAddress address = createSocketAddress(initialPort + i);
+            ServerInfo serverInfo = new ServerInfo();
+            for (int j = 0; j < numStreamsPerHost; j++) {
+                serverInfo.putToOwnerships(runtime.getMethodName() + "_new_stream_" + j,
+                        address.toString());
+            }
+            MockServerInfoService service = mockServiceMap.get(address);
+            serverInfoMap.put(address, serverInfo);
+            service.updateServerInfo(serverInfo);
+        }
+
+        clientManager.setPeriodicHandshakeEnabled(true);
+        for (int i = 0; i < numHosts; i++) {
+            SocketAddress address = createSocketAddress(initialPort + i);
+            CountDownLatch latch = hostDoneLatches.get(address);
+            latch.await();
+        }
+
+        assertTrue("Periodic handshake should update all server infos",
+                Maps.difference(serverInfoMap, results).areEqual());
+    }
+
+}
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestConsistentHashRoutingService.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestConsistentHashRoutingService.java
new file mode 100644
index 0000000..f44cddd
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestConsistentHashRoutingService.java
@@ -0,0 +1,417 @@
+/**
+ * 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.client.routing;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.ImmutableSet;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import org.apache.distributedlog.service.DLSocketAddress;
+import com.twitter.finagle.Address;
+import com.twitter.finagle.Addresses;
+import com.twitter.finagle.ChannelWriteException;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link ConsistentHashRoutingService}.
+ */
+public class TestConsistentHashRoutingService {
+
+    @Test(timeout = 60000)
+    public void testBlackoutHost() throws Exception {
+        TestName name = new TestName();
+        RoutingService routingService = ConsistentHashRoutingService.newBuilder()
+                .serverSet(new NameServerSet(name))
+                .resolveFromName(true)
+                .numReplicas(997)
+                .blackoutSeconds(2)
+                .build();
+
+        InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", 3181);
+        Address address = Addresses.newInetAddress(inetAddress);
+        List<Address> addresses = new ArrayList<Address>(1);
+        addresses.add(address);
+        name.changeAddrs(addresses);
+
+        routingService.startService();
+
+        RoutingService.RoutingContext routingContext =
+                RoutingService.RoutingContext.of(new DefaultRegionResolver());
+
+        String streamName = "test-blackout-host";
+        assertEquals(inetAddress, routingService.getHost(streamName, routingContext));
+        routingService.removeHost(inetAddress, new ChannelWriteException(new IOException("test exception")));
+        try {
+            routingService.getHost(streamName, routingContext);
+            fail("Should fail to get host since no brokers are available");
+        } catch (NoBrokersAvailableException nbae) {
+            // expected
+        }
+
+        TimeUnit.SECONDS.sleep(3);
+        assertEquals(inetAddress, routingService.getHost(streamName, routingContext));
+
+        routingService.stopService();
+    }
+
+    @Test(timeout = 60000)
+    public void testPerformServerSetChangeOnName() throws Exception {
+        TestName name = new TestName();
+        ConsistentHashRoutingService routingService = (ConsistentHashRoutingService)
+                ConsistentHashRoutingService.newBuilder()
+                        .serverSet(new NameServerSet(name))
+                        .resolveFromName(true)
+                        .numReplicas(997)
+                        .build();
+
+        int basePort = 3180;
+        int numHosts = 4;
+        List<Address> addresses1 = Lists.newArrayListWithExpectedSize(4);
+        List<Address> addresses2 = Lists.newArrayListWithExpectedSize(4);
+        List<Address> addresses3 = Lists.newArrayListWithExpectedSize(4);
+
+        // fill up the addresses1
+        for (int i = 0; i < numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+            Address address = Addresses.newInetAddress(inetAddress);
+            addresses1.add(address);
+        }
+        // fill up the addresses2 - overlap with addresses1
+        for (int i = 0; i < numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 2 + i);
+            Address address = Addresses.newInetAddress(inetAddress);
+            addresses2.add(address);
+        }
+        // fill up the addresses3 - not overlap with addresses2
+        for (int i = 0; i < numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
+            Address address = Addresses.newInetAddress(inetAddress);
+            addresses3.add(address);
+        }
+
+        final List<SocketAddress> leftAddresses = Lists.newArrayList();
+        final List<SocketAddress> joinAddresses = Lists.newArrayList();
+
+        RoutingService.RoutingListener routingListener = new RoutingService.RoutingListener() {
+            @Override
+            public void onServerLeft(SocketAddress address) {
+                synchronized (leftAddresses) {
+                    leftAddresses.add(address);
+                    leftAddresses.notifyAll();
+                }
+            }
+
+            @Override
+            public void onServerJoin(SocketAddress address) {
+                synchronized (joinAddresses) {
+                    joinAddresses.add(address);
+                    joinAddresses.notifyAll();
+                }
+            }
+        };
+
+        routingService.registerListener(routingListener);
+        name.changeAddrs(addresses1);
+
+        routingService.startService();
+
+        synchronized (joinAddresses) {
+            while (joinAddresses.size() < numHosts) {
+                joinAddresses.wait();
+            }
+        }
+
+        // validate 4 nodes joined
+        synchronized (joinAddresses) {
+            assertEquals(numHosts, joinAddresses.size());
+        }
+        synchronized (leftAddresses) {
+            assertEquals(0, leftAddresses.size());
+        }
+        assertEquals(numHosts, routingService.shardId2Address.size());
+        assertEquals(numHosts, routingService.address2ShardId.size());
+        for (int i = 0; i < numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+            int shardId = routingService.address2ShardId.get(inetAddress);
+            SocketAddress sa = routingService.shardId2Address.get(shardId);
+            assertNotNull(sa);
+            assertEquals(inetAddress, sa);
+        }
+
+        // update addresses2 - 2 new hosts joined, 2 old hosts left
+        name.changeAddrs(addresses2);
+        synchronized (joinAddresses) {
+            while (joinAddresses.size() < numHosts + 2) {
+                joinAddresses.wait();
+            }
+        }
+        synchronized (leftAddresses) {
+            while (leftAddresses.size() < numHosts - 2) {
+                leftAddresses.wait();
+            }
+        }
+        assertEquals(numHosts, routingService.shardId2Address.size());
+        assertEquals(numHosts, routingService.address2ShardId.size());
+
+        // first 2 shards should leave
+        for (int i = 0; i < 2; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+            assertFalse(routingService.address2ShardId.containsKey(inetAddress));
+        }
+
+        for (int i = 0; i < numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 2 + i);
+            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+            int shardId = routingService.address2ShardId.get(inetAddress);
+            SocketAddress sa = routingService.shardId2Address.get(shardId);
+            assertNotNull(sa);
+            assertEquals(inetAddress, sa);
+        }
+
+        // update addresses3 - 2 new hosts joined, 2 old hosts left
+        name.changeAddrs(addresses3);
+        synchronized (joinAddresses) {
+            while (joinAddresses.size() < numHosts + 2 + numHosts) {
+                joinAddresses.wait();
+            }
+        }
+        synchronized (leftAddresses) {
+            while (leftAddresses.size() < numHosts - 2 + numHosts) {
+                leftAddresses.wait();
+            }
+        }
+        assertEquals(numHosts, routingService.shardId2Address.size());
+        assertEquals(numHosts, routingService.address2ShardId.size());
+
+        // first 6 shards should leave
+        for (int i = 0; i < 2 + numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+            assertFalse(routingService.address2ShardId.containsKey(inetAddress));
+        }
+        // new 4 shards should exist
+        for (int i = 0; i < numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
+            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+            int shardId = routingService.address2ShardId.get(inetAddress);
+            SocketAddress sa = routingService.shardId2Address.get(shardId);
+            assertNotNull(sa);
+            assertEquals(inetAddress, sa);
+        }
+
+    }
+
+    private static class TestServerSetWatcher implements ServerSetWatcher {
+
+        final LinkedBlockingQueue<ImmutableSet<DLSocketAddress>> changeQueue =
+                new LinkedBlockingQueue<ImmutableSet<DLSocketAddress>>();
+        final CopyOnWriteArrayList<ServerSetMonitor> monitors =
+                new CopyOnWriteArrayList<ServerSetMonitor>();
+
+        @Override
+        public void watch(ServerSetMonitor monitor) throws MonitorException {
+            monitors.add(monitor);
+            ImmutableSet<DLSocketAddress> change;
+            while ((change = changeQueue.poll()) != null) {
+                notifyChanges(change);
+            }
+        }
+
+        void notifyChanges(ImmutableSet<DLSocketAddress> addresses) {
+            if (monitors.isEmpty()) {
+                changeQueue.add(addresses);
+            } else {
+                for (ServerSetMonitor monitor : monitors) {
+                    monitor.onChange(addresses);
+                }
+            }
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testPerformServerSetChangeOnServerSet() throws Exception {
+        TestServerSetWatcher serverSetWatcher = new TestServerSetWatcher();
+        ConsistentHashRoutingService routingService = new ConsistentHashRoutingService(
+                serverSetWatcher, 997, Integer.MAX_VALUE, NullStatsReceiver.get());
+
+        int basePort = 3180;
+        int numHosts = 4;
+        Set<DLSocketAddress> addresses1 = Sets.newConcurrentHashSet();
+        Set<DLSocketAddress> addresses2 = Sets.newConcurrentHashSet();
+        Set<DLSocketAddress> addresses3 = Sets.newConcurrentHashSet();
+
+        // fill up the addresses1
+        for (int i = 0; i < numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+            DLSocketAddress dsa = new DLSocketAddress(i, inetAddress);
+            addresses1.add(dsa);
+        }
+        // fill up the addresses2 - overlap with addresses1
+        for (int i = 0; i < numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + numHosts + i);
+            DLSocketAddress dsa = new DLSocketAddress(i + 2, inetAddress);
+            addresses2.add(dsa);
+        }
+        // fill up the addresses3 - not overlap with addresses2
+        for (int i = 0; i < numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
+            DLSocketAddress dsa = new DLSocketAddress(i, inetAddress);
+            addresses3.add(dsa);
+        }
+
+        final List<SocketAddress> leftAddresses = Lists.newArrayList();
+        final List<SocketAddress> joinAddresses = Lists.newArrayList();
+
+        RoutingService.RoutingListener routingListener = new RoutingService.RoutingListener() {
+            @Override
+            public void onServerLeft(SocketAddress address) {
+                synchronized (leftAddresses) {
+                    leftAddresses.add(address);
+                    leftAddresses.notifyAll();
+                }
+            }
+
+            @Override
+            public void onServerJoin(SocketAddress address) {
+                synchronized (joinAddresses) {
+                    joinAddresses.add(address);
+                    joinAddresses.notifyAll();
+                }
+            }
+        };
+
+        routingService.registerListener(routingListener);
+        serverSetWatcher.notifyChanges(ImmutableSet.copyOf(addresses1));
+
+        routingService.startService();
+
+        synchronized (joinAddresses) {
+            while (joinAddresses.size() < numHosts) {
+                joinAddresses.wait();
+            }
+        }
+
+        // validate 4 nodes joined
+        synchronized (joinAddresses) {
+            assertEquals(numHosts, joinAddresses.size());
+        }
+        synchronized (leftAddresses) {
+            assertEquals(0, leftAddresses.size());
+        }
+        assertEquals(numHosts, routingService.shardId2Address.size());
+        assertEquals(numHosts, routingService.address2ShardId.size());
+        for (int i = 0; i < numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+            int shardId = routingService.address2ShardId.get(inetAddress);
+            assertEquals(i, shardId);
+            SocketAddress sa = routingService.shardId2Address.get(shardId);
+            assertNotNull(sa);
+            assertEquals(inetAddress, sa);
+        }
+
+        // update addresses2 - 2 new hosts joined, 2 old hosts left
+        serverSetWatcher.notifyChanges(ImmutableSet.copyOf(addresses2));
+        synchronized (joinAddresses) {
+            while (joinAddresses.size() < numHosts + 2) {
+                joinAddresses.wait();
+            }
+        }
+        synchronized (leftAddresses) {
+            while (leftAddresses.size() < 2) {
+                leftAddresses.wait();
+            }
+        }
+
+        assertEquals(numHosts + 2, routingService.shardId2Address.size());
+        assertEquals(numHosts + 2, routingService.address2ShardId.size());
+        // first 2 shards should not leave
+        for (int i = 0; i < 2; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + i);
+            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+            int shardId = routingService.address2ShardId.get(inetAddress);
+            assertEquals(i, shardId);
+            SocketAddress sa = routingService.shardId2Address.get(shardId);
+            assertNotNull(sa);
+            assertEquals(inetAddress, sa);
+        }
+        for (int i = 0; i < numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + numHosts + i);
+            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+            int shardId = routingService.address2ShardId.get(inetAddress);
+            assertEquals(i + 2, shardId);
+            SocketAddress sa = routingService.shardId2Address.get(shardId);
+            assertNotNull(sa);
+            assertEquals(inetAddress, sa);
+        }
+
+        // update addresses3
+        serverSetWatcher.notifyChanges(ImmutableSet.copyOf(addresses3));
+        synchronized (joinAddresses) {
+            while (joinAddresses.size() < numHosts + 2 + numHosts) {
+                joinAddresses.wait();
+            }
+        }
+        synchronized (leftAddresses) {
+            while (leftAddresses.size() < 2 + numHosts) {
+                leftAddresses.wait();
+            }
+        }
+        assertEquals(numHosts + 2, routingService.shardId2Address.size());
+        assertEquals(numHosts + 2, routingService.address2ShardId.size());
+
+        // first 4 shards should leave
+        for (int i = 0; i < numHosts; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + 10 + i);
+            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+            int shardId = routingService.address2ShardId.get(inetAddress);
+            assertEquals(i, shardId);
+            SocketAddress sa = routingService.shardId2Address.get(shardId);
+            assertNotNull(sa);
+            assertEquals(inetAddress, sa);
+        }
+        // the other 2 shards should be still there
+        for (int i = 0; i < 2; i++) {
+            InetSocketAddress inetAddress = new InetSocketAddress("127.0.0.1", basePort + numHosts + 2 + i);
+            assertTrue(routingService.address2ShardId.containsKey(inetAddress));
+            int shardId = routingService.address2ShardId.get(inetAddress);
+            assertEquals(numHosts + i, shardId);
+            SocketAddress sa = routingService.shardId2Address.get(shardId);
+            assertNotNull(sa);
+            assertEquals(inetAddress, sa);
+        }
+
+    }
+}
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestInetNameResolution.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestInetNameResolution.java
new file mode 100644
index 0000000..59665b9
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestInetNameResolution.java
@@ -0,0 +1,73 @@
+/**
+ * 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.client.routing;
+
+import com.google.common.collect.ImmutableSet;
+import com.twitter.common.net.pool.DynamicHostSet;
+import com.twitter.thrift.Endpoint;
+import com.twitter.thrift.ServiceInstance;
+import java.net.InetSocketAddress;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Case for `inet` name resolution.
+ */
+public class TestInetNameResolution {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestRoutingService.class);
+
+    @Test(timeout = 10000)
+    public void testInetNameResolution() throws Exception {
+        String nameStr = "inet!127.0.0.1:3181";
+        final CountDownLatch resolved = new CountDownLatch(1);
+        final AtomicBoolean validationFailed = new AtomicBoolean(false);
+
+        NameServerSet serverSet = new NameServerSet(nameStr);
+        serverSet.watch(new DynamicHostSet.HostChangeMonitor<ServiceInstance>() {
+            @Override
+            public void onChange(ImmutableSet<ServiceInstance> hostSet) {
+                if (hostSet.size() > 1) {
+                    logger.error("HostSet has more elements than expected {}", hostSet);
+                    validationFailed.set(true);
+                    resolved.countDown();
+                } else if (hostSet.size() == 1) {
+                    ServiceInstance serviceInstance = hostSet.iterator().next();
+                    Endpoint endpoint = serviceInstance.getAdditionalEndpoints().get("thrift");
+                    InetSocketAddress address = new InetSocketAddress(endpoint.getHost(), endpoint.getPort());
+                    if (endpoint.getPort() != 3181) {
+                        logger.error("Port does not match the expected port {}", endpoint.getPort());
+                        validationFailed.set(true);
+                    } else if (!address.getAddress().getHostAddress().equals("127.0.0.1")) {
+                        logger.error("Host address does not match the expected address {}",
+                            address.getAddress().getHostAddress());
+                        validationFailed.set(true);
+                    }
+                    resolved.countDown();
+                }
+            }
+        });
+
+        resolved.await();
+        Assert.assertEquals(false, validationFailed.get());
+    }
+}
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRegionsRoutingService.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRegionsRoutingService.java
new file mode 100644
index 0000000..151663e
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRegionsRoutingService.java
@@ -0,0 +1,133 @@
+/**
+ * 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.client.routing;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import com.twitter.finagle.NoBrokersAvailableException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link RegionsRoutingService}.
+ */
+public class TestRegionsRoutingService {
+
+    @Test(timeout = 60000)
+    public void testRoutingListener() throws Exception {
+        int numRoutingServices = 5;
+        RoutingService.Builder[] routingServiceBuilders = new RoutingService.Builder[numRoutingServices];
+        Set<SocketAddress> hosts = new HashSet<SocketAddress>();
+        Map<SocketAddress, String> regionMap = new HashMap<SocketAddress, String>();
+        for (int i = 0; i < numRoutingServices; i++) {
+            String finagleNameStr = "inet!127.0.0.1:" + (3181 + i);
+            routingServiceBuilders[i] = RoutingUtils.buildRoutingService(finagleNameStr);
+            SocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i);
+            hosts.add(address);
+            regionMap.put(address, "region-" + i);
+        }
+
+        final CountDownLatch doneLatch = new CountDownLatch(numRoutingServices);
+        final AtomicInteger numHostsLeft = new AtomicInteger(0);
+        final Set<SocketAddress> jointHosts = new HashSet<SocketAddress>();
+        RegionsRoutingService regionsRoutingService =
+                RegionsRoutingService.newBuilder()
+                    .routingServiceBuilders(routingServiceBuilders)
+                    .resolver(new DefaultRegionResolver(regionMap))
+                    .build();
+        regionsRoutingService.registerListener(new RoutingService.RoutingListener() {
+            @Override
+            public void onServerLeft(SocketAddress address) {
+                numHostsLeft.incrementAndGet();
+            }
+
+            @Override
+            public void onServerJoin(SocketAddress address) {
+                jointHosts.add(address);
+                doneLatch.countDown();
+            }
+        });
+
+        regionsRoutingService.startService();
+
+        doneLatch.await();
+
+        assertEquals(numRoutingServices, jointHosts.size());
+        assertEquals(0, numHostsLeft.get());
+        assertTrue(Sets.difference(hosts, jointHosts).immutableCopy().isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetHost() throws Exception {
+        int numRoutingServices = 3;
+        RoutingService.Builder[] routingServiceBuilders = new RoutingService.Builder[numRoutingServices];
+        Map<SocketAddress, String> regionMap = new HashMap<SocketAddress, String>();
+        for (int i = 0; i < numRoutingServices; i++) {
+            String finagleNameStr = "inet!127.0.0.1:" + (3181 + i);
+            routingServiceBuilders[i] = RoutingUtils.buildRoutingService(finagleNameStr);
+            SocketAddress address = new InetSocketAddress("127.0.0.1", 3181 + i);
+            regionMap.put(address, "region-" + i);
+        }
+
+        RegionsRoutingService regionsRoutingService =
+                RegionsRoutingService.newBuilder()
+                    .resolver(new DefaultRegionResolver(regionMap))
+                    .routingServiceBuilders(routingServiceBuilders)
+                    .build();
+        regionsRoutingService.startService();
+
+        RoutingService.RoutingContext routingContext =
+                RoutingService.RoutingContext.of(new DefaultRegionResolver())
+                        .addTriedHost(new InetSocketAddress("127.0.0.1", 3183), StatusCode.WRITE_EXCEPTION);
+        assertEquals(new InetSocketAddress("127.0.0.1", 3181),
+                regionsRoutingService.getHost("any", routingContext));
+
+        routingContext =
+                RoutingService.RoutingContext.of(new DefaultRegionResolver())
+                        .addTriedHost(new InetSocketAddress("127.0.0.1", 3181), StatusCode.WRITE_EXCEPTION);
+        assertEquals(new InetSocketAddress("127.0.0.1", 3182),
+                regionsRoutingService.getHost("any", routingContext));
+
+        // add 3182 to routing context as tried host
+        routingContext.addTriedHost(new InetSocketAddress("127.0.0.1", 3182), StatusCode.WRITE_EXCEPTION);
+        assertEquals(new InetSocketAddress("127.0.0.1", 3183),
+                regionsRoutingService.getHost("any", routingContext));
+
+        // add 3183 to routing context as tried host
+        routingContext.addTriedHost(new InetSocketAddress("127.0.0.1", 3183), StatusCode.WRITE_EXCEPTION);
+        try {
+            regionsRoutingService.getHost("any", routingContext);
+            fail("Should fail to get host since all regions are tried.");
+        } catch (NoBrokersAvailableException nbae) {
+            // expected
+        }
+    }
+
+}
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRoutingService.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRoutingService.java
new file mode 100644
index 0000000..d2d61a9
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRoutingService.java
@@ -0,0 +1,146 @@
+/**
+ * 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.client.routing;
+
+import static org.junit.Assert.assertEquals;
+
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import com.twitter.finagle.Address;
+import com.twitter.finagle.Addresses;
+import com.twitter.finagle.addr.WeightedAddress;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Case for {@link RoutingService}.
+ */
+@RunWith(Parameterized.class)
+public class TestRoutingService {
+
+    static final Logger LOG = LoggerFactory.getLogger(TestRoutingService.class);
+
+    @Parameterized.Parameters
+    public static Collection<Object[]> configs() {
+        ArrayList<Object[]> list = new ArrayList<Object[]>();
+        for (int i = 0; i <= 1; i++) {
+            for (int j = 0; j <= 1; j++) {
+                for (int k = 0; k <= 1; k++) {
+                    list.add(new Boolean[] {i == 1, j == 1, k == 1});
+                }
+            }
+        }
+        return list;
+    }
+
+    private final boolean consistentHash;
+    private final boolean weightedAddresses;
+    private final boolean asyncResolution;
+
+    public TestRoutingService(boolean consistentHash, boolean weightedAddresses, boolean asyncResolution) {
+        this.consistentHash = consistentHash;
+        this.weightedAddresses = weightedAddresses;
+        this.asyncResolution = asyncResolution;
+    }
+
+    private List<Address> getAddresses(boolean weightedAddresses) {
+        ArrayList<Address> addresses = new ArrayList<Address>();
+        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.1", 3181)));
+        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.2", 3181)));
+        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.3", 3181)));
+        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.4", 3181)));
+        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.5", 3181)));
+        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.6", 3181)));
+        addresses.add(Addresses.newInetAddress(new InetSocketAddress("127.0.0.7", 3181)));
+
+        if (weightedAddresses) {
+            ArrayList<Address> wAddresses = new ArrayList<Address>();
+            for (Address address: addresses) {
+                wAddresses.add(WeightedAddress.apply(address, 1.0));
+            }
+            return wAddresses;
+        } else {
+            return addresses;
+        }
+    }
+
+    private void testRoutingServiceHelper(boolean consistentHash,
+                                          boolean weightedAddresses,
+                                          boolean asyncResolution)
+        throws Exception {
+        ExecutorService executorService = null;
+        final List<Address> addresses = getAddresses(weightedAddresses);
+        final TestName name = new TestName();
+        RoutingService routingService;
+        if (consistentHash) {
+            routingService = ConsistentHashRoutingService.newBuilder()
+                    .serverSet(new NameServerSet(name))
+                    .resolveFromName(true)
+                    .numReplicas(997)
+                    .build();
+        } else {
+            routingService = ServerSetRoutingService.newServerSetRoutingServiceBuilder()
+                    .serverSetWatcher(new TwitterServerSetWatcher(new NameServerSet(name), true)).build();
+        }
+
+        if (asyncResolution) {
+            executorService = Executors.newSingleThreadExecutor();
+            executorService.submit(new Runnable() {
+                @Override
+                public void run() {
+                    name.changeAddrs(addresses);
+                }
+            });
+        } else {
+            name.changeAddrs(addresses);
+        }
+        routingService.startService();
+
+        HashSet<SocketAddress> mapping = new HashSet<SocketAddress>();
+
+        for (int i = 0; i < 1000; i++) {
+            for (int j = 0; j < 5; j++) {
+                String stream = "TestStream-" + i + "-" + j;
+                mapping.add(routingService.getHost(stream,
+                        RoutingService.RoutingContext.of(new DefaultRegionResolver())));
+            }
+        }
+
+        assertEquals(mapping.size(), addresses.size());
+
+        if (null != executorService) {
+            executorService.shutdown();
+        }
+
+    }
+
+    @Test(timeout = 5000)
+    public void testRoutingService() throws Exception {
+        testRoutingServiceHelper(this.consistentHash, this.weightedAddresses, this.asyncResolution);
+    }
+}
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java b/distributedlog-client/src/test/java/org/apache/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
new file mode 100644
index 0000000..ab0cb58
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
@@ -0,0 +1,105 @@
+/**
+ * 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.client.speculative;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.mock;
+
+import com.twitter.util.CountDownLatch;
+import com.twitter.util.Future;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.junit.Test;
+import org.mockito.Mockito;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Test {@link TestDefaultSpeculativeRequestExecutionPolicy}.
+ */
+public class TestDefaultSpeculativeRequestExecutionPolicy {
+
+    @Test(timeout = 20000, expected = IllegalArgumentException.class)
+    public void testInvalidBackoffMultiplier() throws Exception {
+        new DefaultSpeculativeRequestExecutionPolicy(100, 200, -1);
+    }
+
+    @Test(timeout = 20000, expected = IllegalArgumentException.class)
+    public void testInvalidMaxSpeculativeTimeout() throws Exception {
+        new DefaultSpeculativeRequestExecutionPolicy(100, Integer.MAX_VALUE, 2);
+    }
+
+    @Test(timeout = 20000)
+    public void testSpeculativeRequests() throws Exception {
+        DefaultSpeculativeRequestExecutionPolicy policy =
+                new DefaultSpeculativeRequestExecutionPolicy(10, 10000, 2);
+        SpeculativeRequestExecutor executor = mock(SpeculativeRequestExecutor.class);
+
+        final AtomicInteger callCount = new AtomicInteger(0);
+        final CountDownLatch latch = new CountDownLatch(3);
+
+        Mockito.doAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                try {
+                    return Future.value(callCount.incrementAndGet() < 3);
+                } finally {
+                    latch.countDown();
+                }
+            }
+        }).when(executor).issueSpeculativeRequest();
+
+        ScheduledExecutorService executorService =
+                Executors.newSingleThreadScheduledExecutor();
+        policy.initiateSpeculativeRequest(executorService, executor);
+
+        latch.await();
+
+        assertEquals(40, policy.getNextSpeculativeRequestTimeout());
+    }
+
+    @Test(timeout = 20000)
+    public void testSpeculativeRequestsWithMaxTimeout() throws Exception {
+        DefaultSpeculativeRequestExecutionPolicy policy =
+                new DefaultSpeculativeRequestExecutionPolicy(10, 15, 2);
+        SpeculativeRequestExecutor executor = mock(SpeculativeRequestExecutor.class);
+
+        final AtomicInteger callCount = new AtomicInteger(0);
+        final CountDownLatch latch = new CountDownLatch(3);
+
+        Mockito.doAnswer(new Answer() {
+            @Override
+            public Object answer(InvocationOnMock invocation) throws Throwable {
+                try {
+                    return Future.value(callCount.incrementAndGet() < 3);
+                } finally {
+                    latch.countDown();
+                }
+            }
+        }).when(executor).issueSpeculativeRequest();
+
+        ScheduledExecutorService executorService =
+                Executors.newSingleThreadScheduledExecutor();
+        policy.initiateSpeculativeRequest(executorService, executor);
+
+        latch.await();
+
+        assertEquals(15, policy.getNextSpeculativeRequestTimeout());
+    }
+}
diff --git a/distributedlog-client/src/test/java/org/apache/distributedlog/service/TestDistributedLogClientBuilder.java b/distributedlog-client/src/test/java/org/apache/distributedlog/service/TestDistributedLogClientBuilder.java
new file mode 100644
index 0000000..d2df9a5
--- /dev/null
+++ b/distributedlog-client/src/test/java/org/apache/distributedlog/service/TestDistributedLogClientBuilder.java
@@ -0,0 +1,49 @@
+/**
+ * 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.service;
+
+import static org.junit.Assert.assertFalse;
+
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Duration;
+import org.junit.Test;
+
+/**
+ * Test Case of {@link org.apache.distributedlog.service.DistributedLogClientBuilder}.
+ */
+public class TestDistributedLogClientBuilder {
+
+    @Test(timeout = 60000)
+    public void testBuildClientsFromSameBuilder() throws Exception {
+        DistributedLogClientBuilder builder = DistributedLogClientBuilder.newBuilder()
+                .name("build-clients-from-same-builder")
+                .clientId(ClientId$.MODULE$.apply("test-builder"))
+                .finagleNameStr("inet!127.0.0.1:7001")
+                .streamNameRegex(".*")
+                .handshakeWithClientInfo(true)
+                .clientBuilder(ClientBuilder.get()
+                    .hostConnectionLimit(1)
+                    .connectTimeout(Duration.fromSeconds(1))
+                    .tcpConnectTimeout(Duration.fromSeconds(1))
+                    .requestTimeout(Duration.fromSeconds(10)));
+        DistributedLogClient client1 = builder.build();
+        DistributedLogClient client2 = builder.build();
+        assertFalse(client1 == client2);
+    }
+}
diff --git a/distributedlog-core/conf/log4j.properties b/distributedlog-core/conf/log4j.properties
index cafc888..38ab34d 100644
--- a/distributedlog-core/conf/log4j.properties
+++ b/distributedlog-core/conf/log4j.properties
@@ -32,11 +32,11 @@
 log4j.logger.org.apache.bookkeeper=INFO
 
 # redirect executor output to executors.log since slow op warnings can be quite verbose
-log4j.logger.com.twitter.distributedlog.util.MonitoredFuturePool=INFO, Executors
-log4j.logger.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=INFO, Executors
+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.com.twitter.distributedlog.util.MonitoredFuturePool=false
-log4j.additivity.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=false
+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/conf/zookeeper.conf.dynamic.template b/distributedlog-core/conf/zookeeper.conf.dynamic.template
index 4bda9f1..f4e35f5 100644
--- a/distributedlog-core/conf/zookeeper.conf.dynamic.template
+++ b/distributedlog-core/conf/zookeeper.conf.dynamic.template
Binary files differ
diff --git a/distributedlog-core/pom.xml b/distributedlog-core/pom.xml
index c5329aa..c4bfa8f 100644
--- a/distributedlog-core/pom.xml
+++ b/distributedlog-core/pom.xml
@@ -206,7 +206,7 @@
           <properties>
             <property>
               <name>listener</name>
-              <value>com.twitter.distributedlog.TimedOutTestsListener</value>
+              <value>org.apache.distributedlog.TimedOutTestsListener</value>
             </property>
           </properties>
         </configuration>
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamReader.java
deleted file mode 100644
index 0f93bfe..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamReader.java
+++ /dev/null
@@ -1,198 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Preconditions;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class AppendOnlyStreamReader extends InputStream {
-    static final Logger LOG = LoggerFactory.getLogger(AppendOnlyStreamReader.class);
-
-    private LogRecordWithInputStream currentLogRecord = null;
-    private final DistributedLogManager dlm;
-    private LogReader reader;
-    private long currentPosition;
-    private static final int SKIP_BUFFER_SIZE = 512;
-
-    // Cache the input stream for a log record.
-    private static class LogRecordWithInputStream {
-        private final InputStream payloadStream;
-        private final LogRecordWithDLSN logRecord;
-
-        LogRecordWithInputStream(LogRecordWithDLSN logRecord) {
-            Preconditions.checkNotNull(logRecord);
-
-            LOG.debug("Got record dlsn = {}, txid = {}, len = {}",
-                new Object[] {logRecord.getDlsn(), logRecord.getTransactionId(), logRecord.getPayload().length});
-
-            this.logRecord = logRecord;
-            this.payloadStream = logRecord.getPayLoadInputStream();
-        }
-
-        InputStream getPayLoadInputStream() {
-            return payloadStream;
-        }
-
-        LogRecordWithDLSN getLogRecord() {
-            return logRecord;
-        }
-
-        // The last txid of the log record is the position of the next byte in the stream.
-        // Subtract length to get starting offset.
-        long getOffset() {
-            return logRecord.getTransactionId() - logRecord.getPayload().length;
-        }
-    }
-
-    /**
-     * Construct ledger input stream
-     *
-     * @param dlm the Distributed Log Manager to access the stream
-     */
-    AppendOnlyStreamReader(DistributedLogManager dlm)
-        throws IOException {
-        this.dlm = dlm;
-        reader = dlm.getInputStream(0);
-        currentPosition = 0;
-    }
-
-    /**
-     * Get input stream representing next entry in the
-     * ledger.
-     *
-     * @return input stream, or null if no more entries
-     */
-    private LogRecordWithInputStream nextLogRecord() throws IOException {
-        return nextLogRecord(reader);
-    }
-
-    private static LogRecordWithInputStream nextLogRecord(LogReader reader) throws IOException {
-        LogRecordWithDLSN record = reader.readNext(false);
-
-        if (null != record) {
-            return new LogRecordWithInputStream(record);
-        } else {
-            record = reader.readNext(false);
-            if (null != record) {
-                return new LogRecordWithInputStream(record);
-            } else {
-                LOG.debug("No record");
-                return null;
-            }
-        }
-    }
-
-    @Override
-    public int read() throws IOException {
-        byte[] b = new byte[1];
-        if (read(b, 0, 1) != 1) {
-            return -1;
-        } else {
-            return b[0];
-        }
-    }
-
-    @Override
-    public int read(byte[] b, int off, int len) throws IOException {
-        int read = 0;
-        if (currentLogRecord == null) {
-            currentLogRecord = nextLogRecord();
-            if (currentLogRecord == null) {
-                return read;
-            }
-        }
-
-        while (read < len) {
-            int thisread = currentLogRecord.getPayLoadInputStream().read(b, off + read, (len - read));
-            if (thisread == -1) {
-                currentLogRecord = nextLogRecord();
-                if (currentLogRecord == null) {
-                    return read;
-                }
-            } else {
-                LOG.debug("Offset saved = {}, persisted = {}",
-                    currentPosition, currentLogRecord.getLogRecord().getTransactionId());
-                currentPosition += thisread;
-                read += thisread;
-            }
-        }
-        return read;
-    }
-
-    /**
-     * Position the reader at the given offset. If we fail to skip to the desired position
-     * and don't hit end of stream, return false.
-     *
-     * @throws com.twitter.distributedlog.exceptions.EndOfStreamException if we attempt to
-     *         skip past the end of the stream.
-     */
-    public boolean skipTo(long position) throws IOException {
-
-        // No need to skip anywhere.
-        if (position == position()) {
-            return true;
-        }
-
-        LogReader skipReader = dlm.getInputStream(position);
-        LogRecordWithInputStream logRecord = null;
-        try {
-            logRecord = nextLogRecord(skipReader);
-        } catch (IOException ex) {
-            skipReader.close();
-            throw ex;
-        }
-
-        if (null == logRecord) {
-            return false;
-        }
-
-        // We may end up with a reader positioned *before* the requested position if
-        // we're near the tail and the writer is still active, or if the desired position
-        // is not at a log record payload boundary.
-        // Transaction ID gives us the starting position of the log record. Read ahead
-        // if necessary.
-        currentPosition = logRecord.getOffset();
-        currentLogRecord = logRecord;
-        LogReader oldReader = reader;
-        reader = skipReader;
-
-        // Close the oldreader after swapping AppendOnlyStreamReader state. Close may fail
-        // and we need to make sure it leaves AppendOnlyStreamReader in a consistent state.
-        oldReader.close();
-
-        byte[] skipBuffer = new byte[SKIP_BUFFER_SIZE];
-        while (currentPosition < position) {
-            long bytesToRead = Math.min(position - currentPosition, SKIP_BUFFER_SIZE);
-            long bytesRead = read(skipBuffer, 0, (int)bytesToRead);
-            if (bytesRead < bytesToRead) {
-                return false;
-            }
-        }
-
-        return true;
-    }
-
-    public long position() {
-        return currentPosition;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamWriter.java
deleted file mode 100644
index aa0aef9..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamWriter.java
+++ /dev/null
@@ -1,107 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.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 org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class AppendOnlyStreamWriter implements Closeable {
-    static final Logger LOG = LoggerFactory.getLogger(AppendOnlyStreamWriter.class);
-
-    // Use a 1-length array to satisfy Java's inner class reference rules. Use primitive
-    // type because synchronized block is needed anyway.
-    final long[] syncPos = new long[1];
-    BKAsyncLogWriter logWriter;
-    long requestPos = 0;
-
-    public AppendOnlyStreamWriter(BKAsyncLogWriter logWriter, long pos) {
-        LOG.debug("initialize at position {}", pos);
-        this.logWriter = logWriter;
-        this.syncPos[0] = pos;
-        this.requestPos = pos;
-    }
-
-    public Future<DLSN> write(byte[] data) {
-        requestPos += data.length;
-        Future<DLSN> writeResult = logWriter.write(new LogRecord(requestPos, data));
-        return writeResult.addEventListener(new WriteCompleteListener(requestPos));
-    }
-
-    public void force(boolean metadata) throws IOException {
-        long pos = 0;
-        try {
-            pos = Await.result(logWriter.flushAndCommit());
-        } catch (IOException ioe) {
-            throw ioe;
-        } catch (Exception ex) {
-            LOG.error("unexpected exception in AppendOnlyStreamWriter.force ", ex);
-            throw new UnexpectedException("unexpected exception in AppendOnlyStreamWriter.force", ex);
-        }
-        synchronized (syncPos) {
-            syncPos[0] = pos;
-        }
-    }
-
-    public long position() {
-        synchronized (syncPos) {
-            return syncPos[0];
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        logWriter.closeAndComplete();
-    }
-
-    public void markEndOfStream() throws IOException {
-        try {
-            Await.result(logWriter.markEndOfStream());
-        } catch (IOException ioe) {
-            throw ioe;
-        } catch (Exception ex) {
-            throw new UnexpectedException("Mark end of stream hit unexpected exception", ex);
-        }
-    }
-
-    class WriteCompleteListener implements FutureEventListener<DLSN> {
-        private final long position;
-        public WriteCompleteListener(long position) {
-            this.position = position;
-        }
-        @Override
-        public void onSuccess(DLSN response) {
-            synchronized (syncPos) {
-                if (position > syncPos[0]) {
-                    syncPos[0] = position;
-                }
-            }
-        }
-        @Override
-        public void onFailure(Throwable cause) {
-            // Handled at the layer above
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogReader.java
deleted file mode 100644
index 8e07797..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogReader.java
+++ /dev/null
@@ -1,69 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-public interface AsyncLogReader extends AsyncCloseable {
-
-    /**
-     * Get stream name that the reader reads from.
-     *
-     * @return stream name.
-     */
-    public String getStreamName();
-
-    /**
-     * 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();
-
-    /**
-     * Read next <i>numEntries</i> entries. The future is only satisfied with non-empty list
-     * of entries. It doesn't block until returning exact <i>numEntries</i>. It is a best effort
-     * call.
-     *
-     * @param numEntries
-     *          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);
-
-    /**
-     * 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>.
-     * The only exception is if there isn't any new entries written within <i>waitTime</i>, it would
-     * wait until new entries are available.
-     *
-     * @param numEntries
-     *          max entries to return
-     * @param waitTime
-     *          maximum wait time if there are entries already for read
-     * @param timeUnit
-     *          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);
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogWriter.java
deleted file mode 100644
index e83e343..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogWriter.java
+++ /dev/null
@@ -1,70 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.io.AsyncAbortable;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.io.Closeable;
-import java.util.List;
-
-public interface AsyncLogWriter extends AsyncCloseable, AsyncAbortable {
-
-    /**
-     * Get the last committed transaction id.
-     *
-     * @return last committed transaction id.
-     */
-    public long getLastTxId();
-
-    /**
-     * Write a log record to the stream.
-     *
-     * @param record single log record
-     * @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);
-
-    /**
-     * Write log records to the stream in bulk. Each future in the list represents the result of
-     * one write operation. The size of the result list is equal to the size of the input list.
-     * Buffers are written in order, and the list of result futures has the same order.
-     *
-     * @param record set of log records
-     * @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);
-
-    /**
-     * Truncate the log until <i>dlsn</i>.
-     *
-     * @param dlsn
-     *          dlsn to truncate until.
-     * @return A Future indicates whether the operation succeeds or not, or an exception
-     * if the truncation fails.
-     */
-    public Future<Boolean> truncate(DLSN dlsn);
-
-    /**
-     * Get the name of the stream this writer writes data to
-     */
-    public String getStreamName();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncNotification.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncNotification.java
deleted file mode 100644
index bd71147..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncNotification.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog;
-
-public interface AsyncNotification {
-    /**
-     * Triggered when the background activity encounters an exception
-     *
-     * @param reason the exception that encountered.
-     */
-    void notifyOnError(Throwable reason);
-
-    /**
-     *  Triggered when the background activity completes an operation
-     */
-    void notifyOnOperationComplete();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAbstractLogWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAbstractLogWriter.java
deleted file mode 100644
index d1c28d7..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAbstractLogWriter.java
+++ /dev/null
@@ -1,555 +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 com.twitter.distributedlog;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.AlreadyClosedException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.io.Abortable;
-import com.twitter.distributedlog.io.Abortables;
-import com.twitter.distributedlog.io.AsyncAbortable;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.PermitManager;
-import com.twitter.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;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-abstract class BKAbstractLogWriter implements Closeable, AsyncCloseable, Abortable, AsyncAbortable {
-    static final Logger LOG = LoggerFactory.getLogger(BKAbstractLogWriter.class);
-
-    protected final DistributedLogConfiguration conf;
-    private final DynamicDistributedLogConfiguration dynConf;
-    protected final BKDistributedLogManager bkDistributedLogManager;
-
-    // States
-    private Promise<Void> closePromise = null;
-    private volatile boolean forceRolling = false;
-    private boolean forceRecovery = false;
-
-    // Truncation Related
-    private Future<List<LogSegmentMetadata>> lastTruncationAttempt = null;
-    @VisibleForTesting
-    private Long minTimestampToKeepOverride = null;
-
-    // Log Segment Writers
-    protected BKLogSegmentWriter segmentWriter = null;
-    protected Future<BKLogSegmentWriter> segmentWriterFuture = null;
-    protected BKLogSegmentWriter allocatedSegmentWriter = null;
-    protected BKLogWriteHandler writeHandler = null;
-
-    BKAbstractLogWriter(DistributedLogConfiguration conf,
-                        DynamicDistributedLogConfiguration dynConf,
-                        BKDistributedLogManager bkdlm) {
-        this.conf = conf;
-        this.dynConf = dynConf;
-        this.bkDistributedLogManager = bkdlm;
-        LOG.debug("Initial retention period for {} : {}", bkdlm.getStreamName(),
-                TimeUnit.MILLISECONDS.convert(dynConf.getRetentionPeriodHours(), TimeUnit.HOURS));
-    }
-
-    // manage write handler
-
-    synchronized protected BKLogWriteHandler getCachedWriteHandler() {
-        return writeHandler;
-    }
-
-    protected BKLogWriteHandler getWriteHandler() throws IOException {
-        BKLogWriteHandler writeHandler = createAndCacheWriteHandler();
-        writeHandler.checkMetadataException();
-        return writeHandler;
-    }
-
-    protected BKLogWriteHandler createAndCacheWriteHandler()
-            throws IOException {
-        synchronized (this) {
-            if (writeHandler != null) {
-                return writeHandler;
-            }
-        }
-        // 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));
-        boolean success = false;
-        try {
-            synchronized (this) {
-                if (writeHandler == null) {
-                    writeHandler = newHandler;
-                    success = true;
-                }
-                return writeHandler;
-            }
-        } finally {
-            if (!success) {
-                newHandler.asyncAbort();
-            }
-        }
-    }
-
-    // manage log segment writers
-
-    protected synchronized BKLogSegmentWriter getCachedLogWriter() {
-        return segmentWriter;
-    }
-
-    protected synchronized Future<BKLogSegmentWriter> getCachedLogWriterFuture() {
-        return segmentWriterFuture;
-    }
-
-    protected synchronized void cacheLogWriter(BKLogSegmentWriter logWriter) {
-        this.segmentWriter = logWriter;
-        this.segmentWriterFuture = Future.value(logWriter);
-    }
-
-    protected synchronized BKLogSegmentWriter removeCachedLogWriter() {
-        try {
-            return segmentWriter;
-        } finally {
-            segmentWriter = null;
-            segmentWriterFuture = null;
-        }
-    }
-
-    protected synchronized BKLogSegmentWriter getAllocatedLogWriter() {
-        return allocatedSegmentWriter;
-    }
-
-    protected synchronized void cacheAllocatedLogWriter(BKLogSegmentWriter logWriter) {
-        this.allocatedSegmentWriter = logWriter;
-    }
-
-    protected synchronized BKLogSegmentWriter removeAllocatedLogWriter() {
-        try {
-            return allocatedSegmentWriter;
-        } finally {
-            allocatedSegmentWriter = null;
-        }
-    }
-
-    private Future<Void> asyncCloseAndComplete(boolean shouldThrow) {
-        BKLogSegmentWriter segmentWriter = getCachedLogWriter();
-        BKLogWriteHandler writeHandler = getCachedWriteHandler();
-        if (null != segmentWriter && null != writeHandler) {
-            cancelTruncation();
-            Promise<Void> completePromise = new Promise<Void>();
-            asyncCloseAndComplete(segmentWriter, writeHandler, completePromise, shouldThrow);
-            return completePromise;
-        } else {
-            return closeNoThrow();
-        }
-    }
-
-    private void asyncCloseAndComplete(final BKLogSegmentWriter segmentWriter,
-                                       final BKLogWriteHandler writeHandler,
-                                       final Promise<Void> completePromise,
-                                       final boolean shouldThrow) {
-        writeHandler.completeAndCloseLogSegment(segmentWriter)
-                .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
-                    @Override
-                    public void onSuccess(LogSegmentMetadata segment) {
-                        removeCachedLogWriter();
-                        complete(null);
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        LOG.error("Completing Log segments encountered exception", cause);
-                        complete(cause);
-                    }
-
-                    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;
-                            }
-                        });
-                    }
-                });
-    }
-
-    @VisibleForTesting
-    void closeAndComplete() throws IOException {
-        FutureUtils.result(asyncCloseAndComplete(true));
-    }
-
-    protected Future<Void> asyncCloseAndComplete() {
-        return asyncCloseAndComplete(true);
-    }
-
-    @Override
-    public void close() throws IOException {
-        FutureUtils.result(asyncClose());
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        return asyncCloseAndComplete(false);
-    }
-
-    /**
-     * Close the writer and release all the underlying resources
-     */
-    protected Future<Void> closeNoThrow() {
-        Promise<Void> closeFuture;
-        synchronized (this) {
-            if (null != closePromise) {
-                return closePromise;
-            }
-            closeFuture = closePromise = new Promise<Void>();
-        }
-        cancelTruncation();
-        Utils.closeSequence(bkDistributedLogManager.getScheduler(),
-                true, /** ignore close errors **/
-                getCachedLogWriter(),
-                getAllocatedLogWriter(),
-                getCachedWriteHandler()
-        ).proxyTo(closeFuture);
-        return closeFuture;
-    }
-
-    @Override
-    public void abort() throws IOException {
-        FutureUtils.result(asyncAbort());
-    }
-
-    @Override
-    public Future<Void> asyncAbort() {
-        Promise<Void> closeFuture;
-        synchronized (this) {
-            if (null != closePromise) {
-                return closePromise;
-            }
-            closeFuture = closePromise = new Promise<Void>();
-        }
-        cancelTruncation();
-        Abortables.abortSequence(bkDistributedLogManager.getScheduler(),
-                getCachedLogWriter(),
-                getAllocatedLogWriter(),
-                getCachedWriteHandler()).proxyTo(closeFuture);
-        return closeFuture;
-    }
-
-    // used by sync writer
-    protected BKLogSegmentWriter getLedgerWriter(final long startTxId,
-                                                 final boolean allowMaxTxID)
-            throws IOException {
-        Future<BKLogSegmentWriter> logSegmentWriterFuture = asyncGetLedgerWriter(true);
-        BKLogSegmentWriter logSegmentWriter = null;
-        if (null != logSegmentWriterFuture) {
-            logSegmentWriter = FutureUtils.result(logSegmentWriterFuture);
-        }
-        if (null == logSegmentWriter || (shouldStartNewSegment(logSegmentWriter) || forceRolling)) {
-            logSegmentWriter = FutureUtils.result(rollLogSegmentIfNecessary(
-                    logSegmentWriter, startTxId, true /* bestEffort */, allowMaxTxID));
-        }
-        return logSegmentWriter;
-    }
-
-    // used by async writer
-    synchronized protected Future<BKLogSegmentWriter> asyncGetLedgerWriter(boolean resetOnError) {
-        final BKLogSegmentWriter ledgerWriter = getCachedLogWriter();
-        Future<BKLogSegmentWriter> ledgerWriterFuture = getCachedLogWriterFuture();
-        if (null == ledgerWriterFuture || null == ledgerWriter) {
-            return null;
-        }
-
-        // 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;
-            if (ledgerWriter.isLogSegmentInError()) {
-                closeFuture = ledgerWriter.asyncAbort();
-            } else {
-                closeFuture = ledgerWriter.asyncClose();
-            }
-            return closeFuture.flatMap(
-                    new AbstractFunction1<Void, Future<BKLogSegmentWriter>>() {
-                @Override
-                public Future<BKLogSegmentWriter> apply(Void result) {
-                    removeCachedLogWriter();
-
-                    if (ledgerWriter.isLogSegmentInError()) {
-                        return Future.value(null);
-                    }
-
-                    BKLogWriteHandler writeHandler;
-                    try {
-                        writeHandler = getWriteHandler();
-                    } catch (IOException e) {
-                        return Future.exception(e);
-                    }
-                    if (null != writeHandler && forceRecovery) {
-                        return writeHandler.completeAndCloseLogSegment(ledgerWriter)
-                                .map(new AbstractFunction1<LogSegmentMetadata, BKLogSegmentWriter>() {
-                            @Override
-                            public BKLogSegmentWriter apply(LogSegmentMetadata completedLogSegment) {
-                                return null;
-                            }
-                        });
-                    } else {
-                        return Future.value(null);
-                    }
-                }
-            });
-        } else {
-            return ledgerWriterFuture;
-        }
-    }
-
-    boolean shouldStartNewSegment(BKLogSegmentWriter ledgerWriter) throws IOException {
-        BKLogWriteHandler writeHandler = getWriteHandler();
-        return null == ledgerWriter || writeHandler.shouldStartNewSegment(ledgerWriter) || forceRolling;
-    }
-
-    private void truncateLogSegmentsIfNecessary(BKLogWriteHandler writeHandler) {
-        boolean truncationEnabled = false;
-
-        long minTimestampToKeep = 0;
-
-        long retentionPeriodInMillis = TimeUnit.MILLISECONDS.convert(dynConf.getRetentionPeriodHours(), TimeUnit.HOURS);
-        if (retentionPeriodInMillis > 0) {
-            minTimestampToKeep = Utils.nowInMillis() - retentionPeriodInMillis;
-            truncationEnabled = true;
-        }
-
-        if (null != minTimestampToKeepOverride) {
-            minTimestampToKeep = minTimestampToKeepOverride;
-            truncationEnabled = true;
-        }
-
-        // skip scheduling if there is task that's already running
-        //
-        synchronized (this) {
-            if (truncationEnabled && ((lastTruncationAttempt == null) || lastTruncationAttempt.isDefined())) {
-                lastTruncationAttempt = writeHandler.purgeLogSegmentsOlderThanTimestamp(minTimestampToKeep);
-            }
-        }
-    }
-
-    private Future<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) {
-                        cacheLogWriter(newSegmentWriter);
-                        return BoxedUnit.UNIT;
-                    }
-                });
-            }
-        });
-    }
-
-    private Future<BKLogSegmentWriter> closeOldLogSegmentAndStartNewOneWithPermit(
-            final BKLogSegmentWriter oldSegmentWriter,
-            final BKLogWriteHandler writeHandler,
-            final long startTxId,
-            final boolean bestEffort,
-            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());
-                            bkDistributedLogManager.getLogSegmentRollingPermitManager().disallowObtainPermits(switchPermit);
-                            return Future.value(oldSegmentWriter);
-                        }
-                    }
-                    return Future.exception(cause);
-                }
-            }).ensure(new AbstractFunction0<BoxedUnit>() {
-                @Override
-                public BoxedUnit apply() {
-                    bkDistributedLogManager.getLogSegmentRollingPermitManager()
-                            .releasePermit(switchPermit);
-                    return BoxedUnit.UNIT;
-                }
-            });
-        } else {
-            bkDistributedLogManager.getLogSegmentRollingPermitManager().releasePermit(switchPermit);
-            return Future.value(oldSegmentWriter);
-        }
-    }
-
-    private Future<BKLogSegmentWriter> closeOldLogSegmentAndStartNewOne(
-            final BKLogSegmentWriter oldSegmentWriter,
-            final BKLogWriteHandler writeHandler,
-            final long startTxId,
-            final boolean bestEffort,
-            final boolean allowMaxTxID) {
-        // we switch only when we could allocate a new log segment.
-        BKLogSegmentWriter newSegmentWriter = getAllocatedLogWriter();
-        if (null == newSegmentWriter) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Allocating a new log segment from {} for {}.", startTxId,
-                        writeHandler.getFullyQualifiedName());
-            }
-            return writeHandler.asyncStartLogSegment(startTxId, bestEffort, allowMaxTxID)
-                    .flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<BKLogSegmentWriter>>() {
-                        @Override
-                        public Future<BKLogSegmentWriter> apply(BKLogSegmentWriter newSegmentWriter) {
-                            if (null == newSegmentWriter) {
-                                if (bestEffort) {
-                                    return Future.value(oldSegmentWriter);
-                                } else {
-                                    return Future.exception(
-                                            new UnexpectedException("StartLogSegment returns null for bestEffort rolling"));
-                                }
-                            }
-                            cacheAllocatedLogWriter(newSegmentWriter);
-                            if (LOG.isDebugEnabled()) {
-                                LOG.debug("Allocated a new log segment from {} for {}.", startTxId,
-                                        writeHandler.getFullyQualifiedName());
-                            }
-                            return completeOldSegmentAndCacheNewLogSegmentWriter(oldSegmentWriter, newSegmentWriter);
-                        }
-                    });
-        } else {
-            return completeOldSegmentAndCacheNewLogSegmentWriter(oldSegmentWriter, newSegmentWriter);
-        }
-    }
-
-    private Future<BKLogSegmentWriter> completeOldSegmentAndCacheNewLogSegmentWriter(
-            BKLogSegmentWriter oldSegmentWriter,
-            final BKLogSegmentWriter newSegmentWriter) {
-        final Promise<BKLogSegmentWriter> completePromise = new Promise<BKLogSegmentWriter>();
-        // complete the old log segment
-        writeHandler.completeAndCloseLogSegment(oldSegmentWriter)
-                .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
-
-                    @Override
-                    public void onSuccess(LogSegmentMetadata value) {
-                        cacheLogWriter(newSegmentWriter);
-                        removeAllocatedLogWriter();
-                        FutureUtils.setValue(completePromise, newSegmentWriter);
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        FutureUtils.setException(completePromise, cause);
-                    }
-                });
-        return completePromise;
-    }
-
-    synchronized protected Future<BKLogSegmentWriter> rollLogSegmentIfNecessary(
-            final BKLogSegmentWriter segmentWriter,
-            long startTxId,
-            boolean bestEffort,
-            boolean allowMaxTxID) {
-        final BKLogWriteHandler writeHandler;
-        try {
-            writeHandler = getWriteHandler();
-        } catch (IOException e) {
-            return Future.exception(e);
-        }
-        Future<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);
-        }
-        return rollPromise.map(new AbstractFunction1<BKLogSegmentWriter, BKLogSegmentWriter>() {
-            @Override
-            public BKLogSegmentWriter apply(BKLogSegmentWriter newSegmentWriter) {
-                if (segmentWriter == newSegmentWriter) {
-                    return newSegmentWriter;
-                }
-                truncateLogSegmentsIfNecessary(writeHandler);
-                return newSegmentWriter;
-            }
-        });
-    }
-
-    protected synchronized void checkClosedOrInError(String operation) throws AlreadyClosedException {
-        if (null != closePromise) {
-            LOG.error("Executing " + operation + " on already closed Log Writer");
-            throw new AlreadyClosedException("Executing " + operation + " on already closed Log Writer");
-        }
-    }
-
-    @VisibleForTesting
-    public void setForceRolling(boolean forceRolling) {
-        this.forceRolling = forceRolling;
-    }
-
-    @VisibleForTesting
-    public synchronized void overRideMinTimeStampToKeep(Long minTimestampToKeepOverride) {
-        this.minTimestampToKeepOverride = minTimestampToKeepOverride;
-    }
-
-    protected synchronized void cancelTruncation() {
-        if (null != lastTruncationAttempt) {
-            FutureUtils.cancel(lastTruncationAttempt);
-            lastTruncationAttempt = null;
-        }
-    }
-
-    @VisibleForTesting
-    public synchronized void setForceRecovery(boolean forceRecovery) {
-        this.forceRecovery = forceRecovery;
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java
deleted file mode 100644
index aee4103..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java
+++ /dev/null
@@ -1,751 +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 com.twitter.distributedlog;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Stopwatch;
-import com.google.common.base.Ticker;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.distributedlog.exceptions.IdleReaderException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.ReadCancelledException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.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.
- *
- * <h3>Metrics</h3>
- * All the metrics are exposed under `async_reader`.
- * <ul>
- * <li> `async_reader`/future_set: opstats. time spent on satisfying futures of read requests.
- * if it is high, it means that the caller takes time on processing the result of read requests.
- * The side effect is blocking consequent reads.
- * <li> `async_reader`/schedule: opstats. time spent on scheduling next reads.
- * <li> `async_reader`/background_read: opstats. time spent on background reads.
- * <li> `async_reader`/read_next_exec: opstats. time spent on executing {@link #readNext()}.
- * <li> `async_reader`/time_between_read_next: opstats. time spent on between two consequent {@link #readNext()}.
- * if it is high, it means that the caller is slowing down on calling {@link #readNext()}.
- * <li> `async_reader`/delay_until_promise_satisfied: opstats. total latency for the read requests.
- * <li> `async_reader`/idle_reader_error: counter. the number idle reader errors.
- * </ul>
- */
-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 final String streamName;
-    protected final BKDistributedLogManager bkDistributedLogManager;
-    protected final BKLogReadHandler readHandler;
-    private final AtomicReference<Throwable> lastException = new AtomicReference<Throwable>();
-    private final OrderedScheduler scheduler;
-    private final ConcurrentLinkedQueue<PendingReadRequest> pendingRequests = new ConcurrentLinkedQueue<PendingReadRequest>();
-    private final Object scheduleLock = new Object();
-    private final AtomicLong scheduleCount = new AtomicLong(0);
-    final private Stopwatch scheduleDelayStopwatch;
-    final private Stopwatch readNextDelayStopwatch;
-    private DLSN startDLSN;
-    private ReadAheadEntryReader readAheadReader = null;
-    private int lastPosition = 0;
-    private final boolean positionGapDetectionEnabled;
-    private final int idleErrorThresholdMillis;
-    final ScheduledFuture<?> idleReaderTimeoutTask;
-    private ScheduledFuture<?> backgroundScheduleTask = null;
-    // last process time
-    private final Stopwatch lastProcessTime;
-
-    protected Promise<Void> closeFuture = null;
-
-    private boolean lockStream = false;
-
-    private final boolean returnEndOfStreamRecord;
-
-    private final Runnable BACKGROUND_READ_SCHEDULER = new Runnable() {
-        @Override
-        public void run() {
-            synchronized (scheduleLock) {
-                backgroundScheduleTask = null;
-            }
-            scheduleBackgroundRead();
-        }
-    };
-
-    // State
-    private Entry.Reader currentEntry = null;
-    private LogRecordWithDLSN nextRecord = null;
-
-    // Failure Injector
-    private boolean disableProcessingReadRequests = false;
-
-    // Stats
-    private final OpStatsLogger readNextExecTime;
-    private final OpStatsLogger delayUntilPromiseSatisfied;
-    private final OpStatsLogger timeBetweenReadNexts;
-    private final OpStatsLogger futureSetLatency;
-    private final OpStatsLogger scheduleLatency;
-    private final OpStatsLogger backgroundReaderRunTime;
-    private final Counter idleReaderCheckCount;
-    private final Counter idleReaderCheckIdleReadRequestCount;
-    private final Counter idleReaderCheckIdleReadAheadCount;
-    private final Counter idleReaderError;
-
-    private class PendingReadRequest {
-        private final Stopwatch enqueueTime;
-        private final int numEntries;
-        private final List<LogRecordWithDLSN> records;
-        private final Promise<List<LogRecordWithDLSN>> promise;
-        private final long deadlineTime;
-        private final TimeUnit deadlineTimeUnit;
-
-        PendingReadRequest(int numEntries,
-                           long deadlineTime,
-                           TimeUnit deadlineTimeUnit) {
-            this.numEntries = numEntries;
-            this.enqueueTime = Stopwatch.createStarted();
-            // optimize the space usage for single read.
-            if (numEntries == 1) {
-                this.records = new ArrayList<LogRecordWithDLSN>(1);
-            } else {
-                this.records = new ArrayList<LogRecordWithDLSN>();
-            }
-            this.promise = new Promise<List<LogRecordWithDLSN>>();
-            this.deadlineTime = deadlineTime;
-            this.deadlineTimeUnit = deadlineTimeUnit;
-        }
-
-        Promise<List<LogRecordWithDLSN>> getPromise() {
-            return promise;
-        }
-
-        long elapsedSinceEnqueue(TimeUnit timeUnit) {
-            return enqueueTime.elapsed(timeUnit);
-        }
-
-        void setException(Throwable throwable) {
-            Stopwatch stopwatch = Stopwatch.createStarted();
-            if (promise.updateIfEmpty(new Throw<List<LogRecordWithDLSN>>(throwable))) {
-                futureSetLatency.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-                delayUntilPromiseSatisfied.registerFailedEvent(enqueueTime.elapsed(TimeUnit.MICROSECONDS));
-            }
-        }
-
-        boolean hasReadRecords() {
-            return records.size() > 0;
-        }
-
-        boolean hasReadEnoughRecords() {
-            return records.size() >= numEntries;
-        }
-
-        long getRemainingWaitTime() {
-            if (deadlineTime <= 0L) {
-                return 0L;
-            }
-            return deadlineTime - elapsedSinceEnqueue(deadlineTimeUnit);
-        }
-
-        void addRecord(LogRecordWithDLSN record) {
-            records.add(record);
-        }
-
-        void complete() {
-            if (LOG.isTraceEnabled()) {
-                LOG.trace("{} : Satisfied promise with {} records", readHandler.getFullyQualifiedName(), records.size());
-            }
-            delayUntilPromiseSatisfied.registerSuccessfulEvent(enqueueTime.stop().elapsed(TimeUnit.MICROSECONDS));
-            Stopwatch stopwatch = Stopwatch.createStarted();
-            promise.setValue(records);
-            futureSetLatency.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-        }
-    }
-
-    BKAsyncLogReader(BKDistributedLogManager bkdlm,
-                     OrderedScheduler scheduler,
-                     DLSN startDLSN,
-                     Optional<String> subscriberId,
-                     boolean returnEndOfStreamRecord,
-                     StatsLogger statsLogger) {
-        this.streamName = bkdlm.getStreamName();
-        this.bkDistributedLogManager = bkdlm;
-        this.scheduler = scheduler;
-        this.readHandler = bkDistributedLogManager.createReadHandler(subscriberId,
-                this, true);
-        LOG.debug("Starting async reader at {}", startDLSN);
-        this.startDLSN = startDLSN;
-        this.scheduleDelayStopwatch = Stopwatch.createUnstarted();
-        this.readNextDelayStopwatch = Stopwatch.createStarted();
-        this.positionGapDetectionEnabled = bkdlm.getConf().getPositionGapDetectionEnabled();
-        this.idleErrorThresholdMillis = bkdlm.getConf().getReaderIdleErrorThresholdMillis();
-        this.returnEndOfStreamRecord = returnEndOfStreamRecord;
-
-        // Stats
-        StatsLogger asyncReaderStatsLogger = statsLogger.scope("async_reader");
-        futureSetLatency = asyncReaderStatsLogger.getOpStatsLogger("future_set");
-        scheduleLatency = asyncReaderStatsLogger.getOpStatsLogger("schedule");
-        backgroundReaderRunTime = asyncReaderStatsLogger.getOpStatsLogger("background_read");
-        readNextExecTime = asyncReaderStatsLogger.getOpStatsLogger("read_next_exec");
-        timeBetweenReadNexts = asyncReaderStatsLogger.getOpStatsLogger("time_between_read_next");
-        delayUntilPromiseSatisfied = asyncReaderStatsLogger.getOpStatsLogger("delay_until_promise_satisfied");
-        idleReaderError = asyncReaderStatsLogger.getCounter("idle_reader_error");
-        idleReaderCheckCount = asyncReaderStatsLogger.getCounter("idle_reader_check_total");
-        idleReaderCheckIdleReadRequestCount = asyncReaderStatsLogger.getCounter("idle_reader_check_idle_read_requests");
-        idleReaderCheckIdleReadAheadCount = asyncReaderStatsLogger.getCounter("idle_reader_check_idle_readahead");
-
-        // Lock the stream if requested. The lock will be released when the reader is closed.
-        this.lockStream = false;
-        this.idleReaderTimeoutTask = scheduleIdleReaderTaskIfNecessary();
-        this.lastProcessTime = Stopwatch.createStarted();
-    }
-
-    private ScheduledFuture<?> scheduleIdleReaderTaskIfNecessary() {
-        if (idleErrorThresholdMillis < Integer.MAX_VALUE) {
-            // Dont run the task more than once every seconds (for sanity)
-            long period = Math.max(idleErrorThresholdMillis / 10, 1000);
-            // Except when idle reader threshold is less than a second (tests?)
-            period = Math.min(period, idleErrorThresholdMillis / 5);
-
-            return scheduler.scheduleAtFixedRate(streamName, new Runnable() {
-                @Override
-                public void run() {
-                    PendingReadRequest nextRequest = pendingRequests.peek();
-
-                    idleReaderCheckCount.inc();
-                    if (null == nextRequest) {
-                        return;
-                    }
-
-                    idleReaderCheckIdleReadRequestCount.inc();
-                    if (nextRequest.elapsedSinceEnqueue(TimeUnit.MILLISECONDS) < idleErrorThresholdMillis) {
-                        return;
-                    }
-
-                    ReadAheadEntryReader readAheadReader = getReadAheadReader();
-
-                    // read request has been idle
-                    //   - cache has records but read request are idle,
-                    //     that means notification was missed between readahead and reader.
-                    //   - cache is empty and readahead is idle (no records added for a long time)
-                    idleReaderCheckIdleReadAheadCount.inc();
-                    try {
-                        if (null == readAheadReader || (!hasMoreRecords() &&
-                                readAheadReader.isReaderIdle(idleErrorThresholdMillis, TimeUnit.MILLISECONDS))) {
-                            markReaderAsIdle();
-                            return;
-                        } else if (lastProcessTime.elapsed(TimeUnit.MILLISECONDS) > idleErrorThresholdMillis) {
-                            markReaderAsIdle();;
-                        }
-                    } catch (IOException e) {
-                        setLastException(e);
-                        return;
-                    }
-                }
-            }, period, period, TimeUnit.MILLISECONDS);
-        }
-        return null;
-    }
-
-    synchronized ReadAheadEntryReader getReadAheadReader() {
-        return readAheadReader;
-    }
-
-    void cancelIdleReaderTask() {
-        // Do this after we have checked that the reader was not previously closed
-        try {
-            if (null != idleReaderTimeoutTask) {
-                idleReaderTimeoutTask.cancel(true);
-            }
-        } catch (Exception exc) {
-            LOG.info("{}: Failed to cancel the background idle reader timeout task", readHandler.getFullyQualifiedName());
-        }
-    }
-
-    private void markReaderAsIdle() {
-        idleReaderError.inc();
-        IdleReaderException ire = new IdleReaderException("Reader on stream "
-                + readHandler.getFullyQualifiedName()
-                + " is idle for " + idleErrorThresholdMillis +" ms");
-        setLastException(ire);
-        // cancel all pending reads directly rather than notifying on error
-        // because idle reader could happen on idle read requests that usually means something wrong
-        // in scheduling reads
-        cancelAllPendingReads(ire);
-    }
-
-    protected synchronized void setStartDLSN(DLSN fromDLSN) throws UnexpectedException {
-        if (null != readAheadReader) {
-            throw new UnexpectedException("Could't reset from dlsn after reader already starts reading.");
-        }
-        startDLSN = fromDLSN;
-    }
-
-    @VisibleForTesting
-    public synchronized DLSN getStartDLSN() {
-        return startDLSN;
-    }
-
-    public Future<Void> lockStream() {
-        this.lockStream = true;
-        return readHandler.lockStream();
-    }
-
-    private boolean checkClosedOrInError(String operation) {
-        if (null == lastException.get()) {
-            try {
-                if (null != readHandler && null != getReadAheadReader()) {
-                    getReadAheadReader().checkLastException();
-                }
-
-                bkDistributedLogManager.checkClosedOrInError(operation);
-            } catch (IOException exc) {
-                setLastException(exc);
-            }
-        }
-
-        if (lockStream) {
-            try {
-                readHandler.checkReadLock();
-            } catch (IOException ex) {
-                setLastException(ex);
-            }
-        }
-
-        if (null != lastException.get()) {
-            LOG.trace("Cancelling pending reads");
-            cancelAllPendingReads(lastException.get());
-            return true;
-        }
-
-        return false;
-    }
-
-    private void setLastException(IOException exc) {
-        lastException.compareAndSet(null, exc);
-    }
-
-    @Override
-    public String getStreamName() {
-        return streamName;
-    }
-
-    /**
-     * @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 Future<List<LogRecordWithDLSN>> readBulk(int numEntries) {
-        return readInternal(numEntries, 0, TimeUnit.MILLISECONDS);
-    }
-
-    @Override
-    public synchronized Future<List<LogRecordWithDLSN>> readBulk(int numEntries,
-                                                                 long waitTime,
-                                                                 TimeUnit timeUnit) {
-        return readInternal(numEntries, waitTime, timeUnit);
-    }
-
-    /**
-     * Read up to <i>numEntries</i> entries. The future will be satisfied when any number of entries are
-     * ready (1 to <i>numEntries</i>).
-     *
-     * @param numEntries
-     *          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,
-                                                                      long deadlineTime,
-                                                                      TimeUnit deadlineTimeUnit) {
-        timeBetweenReadNexts.registerSuccessfulEvent(readNextDelayStopwatch.elapsed(TimeUnit.MICROSECONDS));
-        readNextDelayStopwatch.reset().start();
-        final PendingReadRequest readRequest = new PendingReadRequest(numEntries, deadlineTime, deadlineTimeUnit);
-
-        if (null == readAheadReader) {
-            final ReadAheadEntryReader readAheadEntryReader = this.readAheadReader = new ReadAheadEntryReader(
-                    getStreamName(),
-                    getStartDLSN(),
-                    bkDistributedLogManager.getConf(),
-                    readHandler,
-                    bkDistributedLogManager.getReaderEntryStore(),
-                    bkDistributedLogManager.getScheduler(),
-                    Ticker.systemTicker(),
-                    bkDistributedLogManager.alertStatsLogger);
-            readHandler.checkLogStreamExists().addEventListener(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;
-                                    }
-                                });
-                    } catch (Exception exc) {
-                        notifyOnError(exc);
-                    }
-                }
-
-                @Override
-                public void onFailure(Throwable cause) {
-                    notifyOnError(cause);
-                }
-            });
-        }
-
-        if (checkClosedOrInError("readNext")) {
-            readRequest.setException(lastException.get());
-        } else {
-            boolean queueEmpty = pendingRequests.isEmpty();
-            pendingRequests.add(readRequest);
-
-            if (queueEmpty) {
-                scheduleBackgroundRead();
-            }
-        }
-
-        readNextExecTime.registerSuccessfulEvent(readNextDelayStopwatch.elapsed(TimeUnit.MICROSECONDS));
-        readNextDelayStopwatch.reset().start();
-
-        return readRequest.getPromise();
-    }
-
-    public synchronized void scheduleBackgroundRead() {
-        // if the reader is already closed, we don't need to schedule background read again.
-        if (null != closeFuture) {
-            return;
-        }
-
-        long prevCount = scheduleCount.getAndIncrement();
-        if (0 == prevCount) {
-            scheduleDelayStopwatch.reset().start();
-            scheduler.submit(streamName, this);
-        }
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        // Cancel the idle reader timeout task, interrupting if necessary
-        ReadCancelledException exception;
-        Promise<Void> closePromise;
-        synchronized (this) {
-            if (null != closeFuture) {
-                return closeFuture;
-            }
-            closePromise = closeFuture = new Promise<Void>();
-            exception = new ReadCancelledException(readHandler.getFullyQualifiedName(), "Reader was closed");
-            setLastException(exception);
-        }
-
-        // Do this after we have checked that the reader was not previously closed
-        cancelIdleReaderTask();
-
-        synchronized (scheduleLock) {
-            if (null != backgroundScheduleTask) {
-                backgroundScheduleTask.cancel(true);
-            }
-        }
-
-        cancelAllPendingReads(exception);
-
-        ReadAheadEntryReader readAheadReader = getReadAheadReader();
-        if (null != readAheadReader) {
-            readHandler.unregisterListener(readAheadReader);
-            readAheadReader.removeStateChangeNotification(this);
-        }
-        Utils.closeSequence(bkDistributedLogManager.getScheduler(), true,
-                readAheadReader,
-                readHandler
-        ).proxyTo(closePromise);
-        return closePromise;
-    }
-
-    private void cancelAllPendingReads(Throwable throwExc) {
-        for (PendingReadRequest promise : pendingRequests) {
-            promise.setException(throwExc);
-        }
-        pendingRequests.clear();
-    }
-
-    synchronized boolean hasMoreRecords() throws IOException {
-        if (null == readAheadReader) {
-            return false;
-        }
-        if (readAheadReader.getNumCachedEntries() > 0 || null != nextRecord) {
-            return true;
-        } else if (null != currentEntry) {
-            nextRecord = currentEntry.nextRecord();
-            return null != nextRecord;
-        }
-        return false;
-    }
-
-    private synchronized LogRecordWithDLSN readNextRecord() throws IOException {
-        if (null == readAheadReader) {
-            return null;
-        }
-        if (null == currentEntry) {
-            currentEntry = readAheadReader.getNextReadAheadEntry(0L, TimeUnit.MILLISECONDS);
-            // no entry after reading from read ahead then return null
-            if (null == currentEntry) {
-                return null;
-            }
-        }
-
-        LogRecordWithDLSN recordToReturn;
-        if (null == nextRecord) {
-            nextRecord = currentEntry.nextRecord();
-            // no more records in current entry
-            if (null == nextRecord) {
-                currentEntry = null;
-                return readNextRecord();
-            }
-        }
-
-        // found a record to return and prefetch the next one
-        recordToReturn = nextRecord;
-        nextRecord = currentEntry.nextRecord();
-        return recordToReturn;
-    }
-
-    @Override
-    public void run() {
-        synchronized(scheduleLock) {
-            if (scheduleDelayStopwatch.isRunning()) {
-                scheduleLatency.registerSuccessfulEvent(scheduleDelayStopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            }
-
-            Stopwatch runTime = Stopwatch.createStarted();
-            int iterations = 0;
-            long scheduleCountLocal = scheduleCount.get();
-            LOG.debug("{}: Scheduled Background Reader", readHandler.getFullyQualifiedName());
-            while(true) {
-                if (LOG.isTraceEnabled()) {
-                    LOG.trace("{}: Executing Iteration: {}", readHandler.getFullyQualifiedName(), iterations++);
-                }
-
-                PendingReadRequest nextRequest = null;
-                synchronized(this) {
-                    nextRequest = pendingRequests.peek();
-
-                    // Queue is empty, nothing to read, return
-                    if (null == nextRequest) {
-                        LOG.trace("{}: Queue Empty waiting for Input", readHandler.getFullyQualifiedName());
-                        scheduleCount.set(0);
-                        backgroundReaderRunTime.registerSuccessfulEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
-                        return;
-                    }
-
-                    if (disableProcessingReadRequests) {
-                        LOG.info("Reader of {} is forced to stop processing read requests", readHandler.getFullyQualifiedName());
-                        return;
-                    }
-                }
-                lastProcessTime.reset().start();
-
-                // If the oldest pending promise is interrupted then we must mark
-                // 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 (checkClosedOrInError("readNext")) {
-                    if (!(lastException.get().getCause() instanceof LogNotFoundException)) {
-                        LOG.warn("{}: Exception", readHandler.getFullyQualifiedName(), lastException.get());
-                    }
-                    backgroundReaderRunTime.registerFailedEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
-                    return;
-                }
-
-                try {
-                    // Fail 10% of the requests when asked to simulate errors
-                    if (bkDistributedLogManager.getFailureInjector().shouldInjectErrors()) {
-                        throw new IOException("Reader Simulated Exception");
-                    }
-                    LogRecordWithDLSN record;
-                    while (!nextRequest.hasReadEnoughRecords()) {
-                        // read single record
-                        do {
-                            record = readNextRecord();
-                        } while (null != record && (record.isControl() || (record.getDlsn().compareTo(getStartDLSN()) < 0)));
-                        if (null == record) {
-                            break;
-                        } else {
-                            if (record.isEndOfStream() && !returnEndOfStreamRecord) {
-                                setLastException(new EndOfStreamException("End of Stream Reached for "
-                                        + readHandler.getFullyQualifiedName()));
-                                break;
-                            }
-
-                            // gap detection
-                            if (recordPositionsContainsGap(record, lastPosition)) {
-                                bkDistributedLogManager.raiseAlert("Gap detected between records at record = {}", record);
-                                if (positionGapDetectionEnabled) {
-                                    throw new DLIllegalStateException("Gap detected between records at record = " + record);
-                                }
-                            }
-                            lastPosition = record.getLastPositionWithinLogSegment();
-
-                            nextRequest.addRecord(record);
-                        }
-                    };
-                } catch (IOException exc) {
-                    setLastException(exc);
-                    if (!(exc instanceof LogNotFoundException)) {
-                        LOG.warn("{} : read with skip Exception", readHandler.getFullyQualifiedName(), lastException.get());
-                    }
-                    continue;
-                }
-
-                if (nextRequest.hasReadRecords()) {
-                    long remainingWaitTime = nextRequest.getRemainingWaitTime();
-                    if (remainingWaitTime > 0 && !nextRequest.hasReadEnoughRecords()) {
-                        backgroundReaderRunTime.registerSuccessfulEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
-                        scheduleDelayStopwatch.reset().start();
-                        scheduleCount.set(0);
-                        // the request could still wait for more records
-                        backgroundScheduleTask = scheduler.schedule(
-                                streamName,
-                                BACKGROUND_READ_SCHEDULER,
-                                remainingWaitTime,
-                                nextRequest.deadlineTimeUnit);
-                        return;
-                    }
-
-                    PendingReadRequest request = pendingRequests.poll();
-                    if (null != request && nextRequest == request) {
-                        request.complete();
-                        if (null != backgroundScheduleTask) {
-                            backgroundScheduleTask.cancel(true);
-                            backgroundScheduleTask = null;
-                        }
-                    } else {
-                        DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at dlsn = "
-                                + nextRequest.records.get(0).getDlsn());
-                        nextRequest.setException(ise);
-                        if (null != request) {
-                            request.setException(ise);
-                        }
-                        // We should never get here as we should have exited the loop if
-                        // pendingRequests were empty
-                        bkDistributedLogManager.raiseAlert("Unexpected condition at dlsn = {}",
-                                nextRequest.records.get(0).getDlsn());
-                        setLastException(ise);
-                    }
-                } else {
-                    if (0 == scheduleCountLocal) {
-                        LOG.trace("Schedule count dropping to zero", lastException.get());
-                        backgroundReaderRunTime.registerSuccessfulEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
-                        return;
-                    }
-                    scheduleCountLocal = scheduleCount.decrementAndGet();
-                }
-            }
-        }
-    }
-
-    private boolean recordPositionsContainsGap(LogRecordWithDLSN record, long lastPosition) {
-        final boolean firstLogRecord = (1 == record.getPositionWithinLogSegment());
-        final boolean endOfStreamRecord = record.isEndOfStream();
-        final boolean emptyLogSegment = (0 == lastPosition);
-        final boolean positionIncreasedByOne = (record.getPositionWithinLogSegment() == (lastPosition + 1));
-
-        return !firstLogRecord && !endOfStreamRecord && !emptyLogSegment &&
-               !positionIncreasedByOne;
-    }
-
-    /**
-     * Triggered when the background activity encounters an exception
-     */
-    @Override
-    public void notifyOnError(Throwable cause) {
-        if (cause instanceof IOException) {
-            setLastException((IOException) cause);
-        } else {
-            setLastException(new IOException(cause));
-        }
-        scheduleBackgroundRead();
-    }
-
-    /**
-     * Triggered when the background activity completes an operation
-     */
-    @Override
-    public void notifyOnOperationComplete() {
-        scheduleBackgroundRead();
-    }
-
-    @VisibleForTesting
-    void simulateErrors() {
-        bkDistributedLogManager.getFailureInjector().injectErrors(true);
-    }
-
-    @VisibleForTesting
-    synchronized void disableReadAheadLogSegmentsNotification() {
-        readHandler.disableReadAheadLogSegmentsNotification();
-    }
-
-    @VisibleForTesting
-    synchronized void disableProcessingReadRequests() {
-        disableProcessingReadRequests = true;
-    }
-}
-
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java
deleted file mode 100644
index 9432e8a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java
+++ /dev/null
@@ -1,559 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Stopwatch;
-import com.google.common.annotations.VisibleForTesting;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.StreamNotReadyException;
-import com.twitter.distributedlog.exceptions.WriteCancelledException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.feature.CoreFeatureKeys;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.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.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.
- *
- * <h3>Metrics</h3>
- * All the metrics are exposed under `log_writer`.
- * <ul>
- * <li> `log_writer/write`: opstats. latency characteristics about the time that write operations spent.
- * <li> `log_writer/bulk_write`: opstats. latency characteristics about the time that bulk_write
- * operations spent.
- * are pending in the queue for long time due to log segment rolling.
- * <li> `log_writer/get_writer`: opstats. the time spent on getting the writer. it could spike when there
- * is log segment rolling happened during getting the writer. it is a good stat to look into when the latency
- * is caused by queuing time.
- * <li> `log_writer/pending_request_dispatch`: counter. the number of queued operations that are dispatched
- * after log segment is rolled. it is an metric on measuring how many operations has been queued because of
- * log segment rolling.
- * </ul>
- * See {@link BKLogSegmentWriter} for segment writer stats.
- */
-public class BKAsyncLogWriter extends BKAbstractLogWriter implements AsyncLogWriter {
-
-    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;
-                }
-            };
-
-    // Records pending for roll log segment.
-    class PendingLogRecord implements FutureEventListener<DLSN> {
-
-        final LogRecord record;
-        final Promise<DLSN> promise;
-        final boolean flush;
-
-        PendingLogRecord(LogRecord record, boolean flush) {
-            this.record = record;
-            this.promise = new Promise<DLSN>();
-            this.flush = flush;
-        }
-
-        @Override
-        public void onSuccess(DLSN value) {
-            promise.setValue(value);
-        }
-
-        @Override
-        public void onFailure(Throwable cause) {
-            promise.setException(cause);
-            encounteredError = true;
-        }
-    }
-
-    /**
-     * Last pending record in current log segment. After it is satisified, it would
-     * roll log segment.
-     *
-     * This implementation is based on the assumption that all future satisified in same
-     * order future pool.
-     */
-    class LastPendingLogRecord extends PendingLogRecord {
-
-        LastPendingLogRecord(LogRecord record, boolean flush) {
-            super(record, flush);
-        }
-
-        @Override
-        public void onSuccess(DLSN value) {
-            super.onSuccess(value);
-            // roll log segment and issue all pending requests.
-            rollLogSegmentAndIssuePendingRequests(record.getTransactionId());
-        }
-
-        @Override
-        public void onFailure(Throwable cause) {
-            super.onFailure(cause);
-            // error out pending requests.
-            errorOutPendingRequestsAndWriter(cause);
-        }
-    }
-
-    private final boolean streamFailFast;
-    private final boolean disableRollOnSegmentError;
-    private LinkedList<PendingLogRecord> pendingRequests = null;
-    private volatile boolean encounteredError = false;
-    private Promise<BKLogSegmentWriter> rollingFuture = null;
-    private long lastTxId = DistributedLogConstants.INVALID_TXID;
-
-    private final StatsLogger statsLogger;
-    private final OpStatsLogger writeOpStatsLogger;
-    private final OpStatsLogger markEndOfStreamOpStatsLogger;
-    private final OpStatsLogger bulkWriteOpStatsLogger;
-    private final OpStatsLogger getWriterOpStatsLogger;
-    private final Counter pendingRequestDispatch;
-
-    private final Feature disableLogSegmentRollingFeature;
-
-    BKAsyncLogWriter(DistributedLogConfiguration conf,
-                     DynamicDistributedLogConfiguration dynConf,
-                     BKDistributedLogManager bkdlm,
-                     BKLogWriteHandler writeHandler, /** log writer owns the handler **/
-                     FeatureProvider featureProvider,
-                     StatsLogger dlmStatsLogger) {
-        super(conf, dynConf, bkdlm);
-        this.writeHandler = writeHandler;
-        this.streamFailFast = conf.getFailFastOnStreamNotReady();
-        this.disableRollOnSegmentError = conf.getDisableRollingOnLogSegmentError();
-
-        // features
-        disableLogSegmentRollingFeature = featureProvider.getFeature(CoreFeatureKeys.DISABLE_LOGSEGMENT_ROLLING.name().toLowerCase());
-        // stats
-        this.statsLogger = dlmStatsLogger.scope("log_writer");
-        this.writeOpStatsLogger = statsLogger.getOpStatsLogger("write");
-        this.markEndOfStreamOpStatsLogger = statsLogger.getOpStatsLogger("mark_end_of_stream");
-        this.bulkWriteOpStatsLogger = statsLogger.getOpStatsLogger("bulk_write");
-        this.getWriterOpStatsLogger = statsLogger.getOpStatsLogger("get_writer");
-        this.pendingRequestDispatch = statsLogger.getCounter("pending_request_dispatch");
-    }
-
-    @VisibleForTesting
-    synchronized void setLastTxId(long txId) {
-        lastTxId = Math.max(lastTxId, txId);
-    }
-
-    @Override
-    public synchronized long getLastTxId() {
-        return lastTxId;
-    }
-
-    /**
-     * Write a log record as control record. The method will be used by Monitor Service to enforce a new inprogress segment.
-     *
-     * @param record
-     *          log record
-     * @return future of the write
-     */
-    public Future<DLSN> writeControlRecord(final LogRecord record) {
-        record.setControl();
-        return write(record);
-    }
-
-    private BKLogSegmentWriter getCachedLogSegmentWriter() throws WriteException {
-        if (encounteredError) {
-            throw new WriteException(bkDistributedLogManager.getStreamName(),
-                    "writer has been closed due to error.");
-        }
-        BKLogSegmentWriter segmentWriter = getCachedLogWriter();
-        if (null != segmentWriter
-                && segmentWriter.isLogSegmentInError()
-                && !disableRollOnSegmentError) {
-            return null;
-        } else {
-            return segmentWriter;
-        }
-    }
-
-    private Future<BKLogSegmentWriter> getLogSegmentWriter(long firstTxid,
-                                                           boolean bestEffort,
-                                                           boolean rollLog,
-                                                           boolean allowMaxTxID) {
-        Stopwatch stopwatch = Stopwatch.createStarted();
-        return FutureUtils.stats(
-                doGetLogSegmentWriter(firstTxid, bestEffort, rollLog, allowMaxTxID),
-                getWriterOpStatsLogger,
-                stopwatch);
-    }
-
-    private Future<BKLogSegmentWriter> doGetLogSegmentWriter(final long firstTxid,
-                                                             final boolean bestEffort,
-                                                             final boolean rollLog,
-                                                             final boolean allowMaxTxID) {
-        if (encounteredError) {
-            return Future.exception(new WriteException(bkDistributedLogManager.getStreamName(),
-                    "writer has been closed due to error."));
-        }
-        Future<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);
-                }
-            });
-        } else {
-            return writerFuture;
-        }
-    }
-
-    /**
-     * 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() {
-        return getLogSegmentWriter(DistributedLogConstants.MAX_TXID,
-                                     false /* bestEffort */,
-                                     false /* roll log */,
-                                     true /* allow max txid */);
-    }
-
-    private Future<BKLogSegmentWriter> getLogSegmentWriter(long firstTxid,
-                                                           boolean bestEffort,
-                                                           boolean rollLog) {
-        return getLogSegmentWriter(firstTxid, bestEffort, rollLog, false /* allow max txid */);
-    }
-
-    Future<DLSN> queueRequest(LogRecord record, boolean flush) {
-        PendingLogRecord pendingLogRecord = new PendingLogRecord(record, flush);
-        pendingRequests.add(pendingLogRecord);
-        return pendingLogRecord.promise;
-    }
-
-    boolean shouldRollLog(BKLogSegmentWriter w) {
-        try {
-            return null == w ||
-                    (!disableLogSegmentRollingFeature.isAvailable() &&
-                    shouldStartNewSegment(w));
-        } catch (IOException ioe) {
-            return false;
-        }
-    }
-
-    void startQueueingRequests() {
-        assert(null == pendingRequests && null == rollingFuture);
-        pendingRequests = new LinkedList<PendingLogRecord>();
-        rollingFuture = new Promise<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,
-                                                 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;
-        BKLogSegmentWriter w;
-        try {
-            w = getCachedLogSegmentWriter();
-        } catch (WriteException we) {
-            return Future.exception(we);
-        }
-        if (null != rollingFuture) {
-            if (streamFailFast) {
-                result = Future.exception(new StreamNotReadyException("Rolling log segment"));
-            } else {
-                result = queueRequest(record, flush);
-            }
-        } else if (shouldRollLog(w)) {
-            // insert a last record, so when it called back, we will trigger a log segment rolling
-            startQueueingRequests();
-            if (null != w) {
-                LastPendingLogRecord lastLogRecordInCurrentSegment = new LastPendingLogRecord(record, flush);
-                w.asyncWrite(record, true).addEventListener(lastLogRecordInCurrentSegment);
-                result = lastLogRecordInCurrentSegment.promise;
-            } else { // no log segment yet. roll the log segment and issue pending requests.
-                result = queueRequest(record, flush);
-                rollLogSegmentAndIssuePendingRequests(record.getTransactionId());
-            }
-        } else {
-            result = w.asyncWrite(record, flush);
-        }
-        // 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;
-            }
-        });
-    }
-
-    private List<Future<DLSN>> asyncWriteBulk(List<LogRecord> records) {
-        final ArrayList<Future<DLSN>> results = new ArrayList<Future<DLSN>>(records.size());
-        Iterator<LogRecord> iterator = records.iterator();
-        while (iterator.hasNext()) {
-            LogRecord record = iterator.next();
-            Future<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()) {
-                break;
-            }
-        }
-        if (records.size() > results.size()) {
-            appendCancelledFutures(results, records.size() - results.size());
-        }
-        return results;
-    }
-
-    private void appendCancelledFutures(List<Future<DLSN>> futures, int numToAdd) {
-        final WriteCancelledException cre =
-            new WriteCancelledException(getStreamName());
-        for (int i = 0; i < numToAdd; i++) {
-            Future<DLSN> cancelledFuture = Future.exception(cre);
-            futures.add(cancelledFuture);
-        }
-    }
-
-    private void rollLogSegmentAndIssuePendingRequests(final long firstTxId) {
-        getLogSegmentWriter(firstTxId, true, true)
-                .addEventListener(new FutureEventListener<BKLogSegmentWriter>() {
-            @Override
-            public void onSuccess(BKLogSegmentWriter writer) {
-                try {
-                    synchronized (BKAsyncLogWriter.this) {
-                        for (PendingLogRecord pendingLogRecord : pendingRequests) {
-                            FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_LogWriterIssuePending);
-                            writer.asyncWrite(pendingLogRecord.record, pendingLogRecord.flush)
-                                    .addEventListener(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
-                        // the corresponding bookies would be able to create its ledger.
-                        if (pendingRequests.isEmpty()) {
-                            LogRecord controlRecord = new LogRecord(firstTxId,
-                                    DistributedLogConstants.CONTROL_RECORD_CONTENT);
-                            controlRecord.setControl();
-                            PendingLogRecord controlReq = new PendingLogRecord(controlRecord, false);
-                            writer.asyncWrite(controlReq.record, controlReq.flush)
-                                    .addEventListener(controlReq);
-                        }
-                        if (null != rollingFuture) {
-                            FutureUtils.setValue(rollingFuture, writer);
-                        }
-                        rollingFuture = null;
-                        pendingRequestDispatch.add(pendingRequests.size());
-                        pendingRequests = null;
-                    }
-                } catch (IOException ioe) {
-                    errorOutPendingRequestsAndWriter(ioe);
-                }
-            }
-            @Override
-            public void onFailure(Throwable cause) {
-                errorOutPendingRequestsAndWriter(cause);
-            }
-        });
-    }
-
-    @VisibleForTesting
-    void errorOutPendingRequests(Throwable cause, boolean errorOutWriter) {
-        final List<PendingLogRecord> pendingRequestsSnapshot;
-        synchronized (this) {
-            pendingRequestsSnapshot = pendingRequests;
-            encounteredError = errorOutWriter;
-            pendingRequests = null;
-            if (null != rollingFuture) {
-                FutureUtils.setException(rollingFuture, cause);
-            }
-            rollingFuture = null;
-        }
-
-        pendingRequestDispatch.add(pendingRequestsSnapshot.size());
-
-        // After erroring out the writer above, no more requests
-        // will be enqueued to pendingRequests
-        for (PendingLogRecord pendingLogRecord : pendingRequestsSnapshot) {
-            pendingLogRecord.promise.setException(cause);
-        }
-    }
-
-    void errorOutPendingRequestsAndWriter(Throwable cause) {
-        errorOutPendingRequests(cause, true /* error out writer */);
-    }
-
-    /**
-     * Write a log record to the stream.
-     *
-     * @param record single log record
-     */
-    @Override
-    public Future<DLSN> write(final LogRecord record) {
-        final Stopwatch stopwatch = Stopwatch.createStarted();
-        return FutureUtils.stats(
-                asyncWrite(record, true),
-                writeOpStatsLogger,
-                stopwatch);
-    }
-
-    /**
-     * Write many log records to the stream. The return type here is unfortunate but its a direct result
-     * of having to combine FuturePool and the asyncWriteBulk method which returns a future as well. The
-     * problem is the List that asyncWriteBulk returns can't be materialized until getLogSegmentWriter
-     * completes, so it has to be wrapped in a future itself.
-     *
-     * @param records list of records
-     */
-    @Override
-    public Future<List<Future<DLSN>>> writeBulk(final List<LogRecord> records) {
-        final Stopwatch stopwatch = Stopwatch.createStarted();
-        return FutureUtils.stats(
-                Future.value(asyncWriteBulk(records)),
-                bulkWriteOpStatsLogger,
-                stopwatch);
-    }
-
-    @Override
-    public Future<Boolean> truncate(final DLSN dlsn) {
-        if (DLSN.InvalidDLSN == dlsn) {
-            return Future.value(false);
-        }
-        BKLogWriteHandler writeHandler;
-        try {
-            writeHandler = getWriteHandler();
-        } catch (IOException e) {
-            return Future.exception(e);
-        }
-        return writeHandler.setLogSegmentsOlderThanDLSNTruncated(dlsn).map(TruncationResultConverter);
-    }
-
-    Future<Long> flushAndCommit() {
-        Future<BKLogSegmentWriter> writerFuture;
-        synchronized (this) {
-            if (null != this.rollingFuture) {
-                writerFuture = this.rollingFuture;
-            } else {
-                writerFuture = getCachedLogWriterFuture();
-            }
-        }
-        if (null == writerFuture) {
-            return Future.value(getLastTxId());
-        }
-        return writerFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Long>>() {
-            @Override
-            public Future<Long> apply(BKLogSegmentWriter writer) {
-                return writer.flushAndCommit();
-            }
-        });
-    }
-
-    Future<Long> markEndOfStream() {
-        final Stopwatch stopwatch = Stopwatch.createStarted();
-        Future<BKLogSegmentWriter> logSegmentWriterFuture;
-        synchronized (this) {
-            logSegmentWriterFuture = this.rollingFuture;
-        }
-        if (null == logSegmentWriterFuture) {
-            logSegmentWriterFuture = getLogSegmentWriterForEndOfStream();
-        }
-
-        return FutureUtils.stats(
-                logSegmentWriterFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Long>>() {
-                    @Override
-                    public Future<Long> apply(BKLogSegmentWriter w) {
-                        return w.markEndOfStream();
-                    }
-                }),
-                markEndOfStreamOpStatsLogger,
-                stopwatch);
-    }
-
-    @Override
-    protected Future<Void> asyncCloseAndComplete() {
-        Future<BKLogSegmentWriter> logSegmentWriterFuture;
-        synchronized (this) {
-            logSegmentWriterFuture = this.rollingFuture;
-        }
-
-        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();
-                }
-            });
-        }
-    }
-
-    @Override
-    void closeAndComplete() throws IOException {
-        FutureUtils.result(asyncCloseAndComplete());
-    }
-
-    /**
-     * *TEMP HACK*
-     * Get the name of the stream this writer writes data to
-     */
-    @Override
-    public String getStreamName() {
-        return bkDistributedLogManager.getStreamName();
-    }
-
-    @Override
-    public Future<Void> asyncAbort() {
-        Future<Void> result = super.asyncAbort();
-        synchronized (this) {
-            if (pendingRequests != null) {
-                for (PendingLogRecord pendingLogRecord : pendingRequests) {
-                    pendingLogRecord.promise.setException(new WriteException(bkDistributedLogManager.getStreamName(),
-                            "abort wring: writer has been closed due to error."));
-                }
-            }
-        }
-        return result;
-    }
-
-    @Override
-    public String toString() {
-        return String.format("AsyncLogWriter:%s", getStreamName());
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java
deleted file mode 100644
index a3959b0..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java
+++ /dev/null
@@ -1,1106 +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 com.twitter.distributedlog;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.twitter.distributedlog.callback.LogSegmentListener;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.AlreadyClosedException;
-import com.twitter.distributedlog.exceptions.LogEmptyException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.function.CloseAsyncCloseableFunction;
-import com.twitter.distributedlog.function.GetVersionedValueFunction;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
-import com.twitter.distributedlog.metadata.LogMetadataForReader;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.lock.DistributedLock;
-import com.twitter.distributedlog.lock.NopDistributedLock;
-import com.twitter.distributedlog.lock.ZKDistributedLock;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
-import com.twitter.distributedlog.namespace.NamespaceDriver;
-import com.twitter.distributedlog.stats.BroadCastStatsLogger;
-import com.twitter.distributedlog.subscription.SubscriptionsStore;
-import com.twitter.distributedlog.util.Allocator;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.MonitoredFuturePool;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.PermitLimiter;
-import com.twitter.distributedlog.util.PermitManager;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.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 com.twitter.distributedlog.namespace.NamespaceDriver.Role.READER;
-import static com.twitter.distributedlog.namespace.NamespaceDriver.Role.WRITER;
-
-/**
- * <h3>Metrics</h3>
- * <ul>
- * <li> `log_writer/*`: all asynchronous writer related metrics are exposed under scope `log_writer`.
- * See {@link BKAsyncLogWriter} for detail stats.
- * <li> `async_reader/*`: all asyncrhonous reader related metrics are exposed under scope `async_reader`.
- * See {@link BKAsyncLogReader} for detail stats.
- * <li> `writer_future_pool/*`: metrics about the future pools that used by writers are exposed under
- * scope `writer_future_pool`. See {@link MonitoredFuturePool} for detail stats.
- * <li> `reader_future_pool/*`: metrics about the future pools that used by readers are exposed under
- * scope `reader_future_pool`. See {@link MonitoredFuturePool} for detail stats.
- * <li> `lock/*`: metrics about the locks used by writers. See {@link ZKDistributedLock} for detail
- * stats.
- * <li> `read_lock/*`: metrics about the locks used by readers. See {@link ZKDistributedLock} for
- * detail stats.
- * <li> `logsegments/*`: metrics about basic operations on log segments. See {@link BKLogHandler} for details.
- * <li> `segments/*`: metrics about write operations on log segments. See {@link BKLogWriteHandler} for details.
- * <li> `readahead_worker/*`: metrics about readahead workers used by readers. See {@link BKLogReadHandler}
- * for details.
- * </ul>
- */
-class BKDistributedLogManager implements DistributedLogManager {
-    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();
-                }
-            };
-
-    static final Function<LogRecordWithDLSN, DLSN> RECORD_2_DLSN_FUNCTION =
-            new Function<LogRecordWithDLSN, DLSN>() {
-                @Override
-                public DLSN apply(LogRecordWithDLSN record) {
-                    return record.getDlsn();
-                }
-            };
-
-    private final URI uri;
-    private final String name;
-    private final String clientId;
-    private final int regionId;
-    private final String streamIdentifier;
-    private final DistributedLogConfiguration conf;
-    private final DynamicDistributedLogConfiguration dynConf;
-    private final NamespaceDriver driver;
-    private Promise<Void> closePromise;
-    private final OrderedScheduler scheduler;
-    private final FeatureProvider featureProvider;
-    private final AsyncFailureInjector failureInjector;
-    private final StatsLogger statsLogger;
-    private final StatsLogger perLogStatsLogger;
-    final AlertStatsLogger alertStatsLogger;
-
-    // log segment metadata cache
-    private final LogSegmentMetadataCache logSegmentMetadataCache;
-
-    //
-    // Writer Related Variables
-    //
-    private final PermitLimiter writeLimiter;
-
-    //
-    // Reader Related Variables
-    ///
-    // read handler for listener.
-    private BKLogReadHandler readHandlerForListener = null;
-    private final PendingReaders pendingReaders;
-
-    // resource to close
-    private final Optional<AsyncCloseable> resourcesCloseable;
-
-    /**
-     * Create a {@link DistributedLogManager} with supplied resources.
-     *
-     * @param name log name
-     * @param conf distributedlog configuration
-     * @param dynConf dynamic distributedlog configuration
-     * @param uri uri location for the log
-     * @param driver namespace driver
-     * @param logSegmentMetadataCache log segment metadata cache
-     * @param scheduler ordered scheduled used by readers and writers
-     * @param clientId client id that used to initiate the locks
-     * @param regionId region id that would be encrypted as part of log segment metadata
-     *                 to indicate which region that the log segment will be created
-     * @param writeLimiter write limiter
-     * @param featureProvider provider to offer features
-     * @param statsLogger stats logger to receive stats
-     * @param perLogStatsLogger stats logger to receive per log stats
-     * @throws IOException
-     */
-    BKDistributedLogManager(String name,
-                            DistributedLogConfiguration conf,
-                            DynamicDistributedLogConfiguration dynConf,
-                            URI uri,
-                            NamespaceDriver driver,
-                            LogSegmentMetadataCache logSegmentMetadataCache,
-                            OrderedScheduler scheduler,
-                            String clientId,
-                            Integer regionId,
-                            PermitLimiter writeLimiter,
-                            FeatureProvider featureProvider,
-                            AsyncFailureInjector failureInjector,
-                            StatsLogger statsLogger,
-                            StatsLogger perLogStatsLogger,
-                            Optional<AsyncCloseable> resourcesCloseable) {
-        this.name = name;
-        this.conf = conf;
-        this.dynConf = dynConf;
-        this.uri = uri;
-        this.driver = driver;
-        this.logSegmentMetadataCache = logSegmentMetadataCache;
-        this.scheduler = scheduler;
-        this.statsLogger = statsLogger;
-        this.perLogStatsLogger = BroadCastStatsLogger.masterslave(perLogStatsLogger, statsLogger);
-        this.pendingReaders = new PendingReaders(scheduler);
-        this.regionId = regionId;
-        this.clientId = clientId;
-        this.streamIdentifier = conf.getUnpartitionedStreamName();
-        this.writeLimiter = writeLimiter;
-        // Feature Provider
-        this.featureProvider = featureProvider;
-        // Failure Injector
-        this.failureInjector = failureInjector;
-        // Stats
-        this.alertStatsLogger = new AlertStatsLogger(this.perLogStatsLogger, "dl_alert");
-        this.resourcesCloseable = resourcesCloseable;
-    }
-
-    @Override
-    public String getStreamName() {
-        return name;
-    }
-
-    @Override
-    public NamespaceDriver getNamespaceDriver() {
-        return driver;
-    }
-
-    URI getUri() {
-        return uri;
-    }
-
-    DistributedLogConfiguration getConf() {
-        return conf;
-    }
-
-    OrderedScheduler getScheduler() {
-        return scheduler;
-    }
-
-    AsyncFailureInjector getFailureInjector() {
-        return failureInjector;
-    }
-
-    //
-    // Test Methods
-    //
-
-    @VisibleForTesting
-    LogStreamMetadataStore getWriterMetadataStore() {
-        return driver.getLogStreamMetadataStore(WRITER);
-    }
-
-    @VisibleForTesting
-    LogSegmentEntryStore getReaderEntryStore() {
-        return driver.getLogSegmentEntryStore(READER);
-    }
-
-    @VisibleForTesting
-    FeatureProvider getFeatureProvider() {
-        return this.featureProvider;
-    }
-
-    private synchronized BKLogReadHandler getReadHandlerAndRegisterListener(
-            boolean create, LogSegmentListener listener) {
-        if (null == readHandlerForListener && create) {
-            readHandlerForListener = createReadHandler();
-            readHandlerForListener.registerListener(listener);
-            // start fetch the log segments after created the listener
-            readHandlerForListener.asyncStartFetchLogSegments();
-            return readHandlerForListener;
-        }
-        if (null != readHandlerForListener && null != listener) {
-            readHandlerForListener.registerListener(listener);
-        }
-        return readHandlerForListener;
-    }
-
-    @Override
-    public List<LogSegmentMetadata> getLogSegments() throws IOException {
-        return FutureUtils.result(getLogSegmentsAsync());
-    }
-
-    protected Future<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));
-    }
-
-    @Override
-    public void registerListener(LogSegmentListener listener) throws IOException {
-        getReadHandlerAndRegisterListener(true, listener);
-    }
-
-    @Override
-    public synchronized void unregisterListener(LogSegmentListener listener) {
-        if (null != readHandlerForListener) {
-            readHandlerForListener.unregisterListener(listener);
-        }
-    }
-
-    public void checkClosedOrInError(String operation) throws AlreadyClosedException {
-        synchronized (this) {
-            if (null != closePromise) {
-                throw new AlreadyClosedException("Executing " + operation + " on already closed DistributedLogManager");
-            }
-        }
-    }
-
-    // Create Read Handler
-
-    synchronized BKLogReadHandler createReadHandler() {
-        Optional<String> subscriberId = Optional.absent();
-        return createReadHandler(subscriberId, false);
-    }
-
-    synchronized BKLogReadHandler createReadHandler(Optional<String> subscriberId) {
-        return createReadHandler(subscriberId, false);
-    }
-
-    synchronized BKLogReadHandler createReadHandler(Optional<String> subscriberId,
-                                                    boolean isHandleForReading) {
-        return createReadHandler(
-                subscriberId,
-                null,
-                isHandleForReading);
-    }
-
-    synchronized BKLogReadHandler createReadHandler(Optional<String> subscriberId,
-                                                    AsyncNotification notification,
-                                                    boolean isHandleForReading) {
-        LogMetadataForReader logMetadata = LogMetadataForReader.of(uri, name, streamIdentifier);
-        return new BKLogReadHandler(
-                logMetadata,
-                subscriberId,
-                conf,
-                dynConf,
-                driver.getLogStreamMetadataStore(READER),
-                logSegmentMetadataCache,
-                driver.getLogSegmentEntryStore(READER),
-                scheduler,
-                alertStatsLogger,
-                statsLogger,
-                perLogStatsLogger,
-                clientId,
-                notification,
-                isHandleForReading);
-    }
-
-    // Create Ledger Allocator
-
-
-
-    // Create Write Handler
-
-    public BKLogWriteHandler createWriteHandler(boolean lockHandler)
-            throws IOException {
-        return FutureUtils.result(asyncCreateWriteHandler(lockHandler));
-    }
-
-    Future<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;
-            }
-        });
-    }
-
-    private void createWriteHandler(LogMetadataForWriter logMetadata,
-                                    boolean lockHandler,
-                                    final Promise<BKLogWriteHandler> createPromise) {
-        // Build the locks
-        DistributedLock lock;
-        if (conf.isWriteLockEnabled()) {
-            lock = driver.getLogStreamMetadataStore(WRITER).createWriteLock(logMetadata);
-        } else {
-            lock = NopDistributedLock.INSTANCE;
-        }
-
-        Allocator<LogSegmentEntryWriter, Object> segmentAllocator;
-        try {
-            segmentAllocator = driver.getLogSegmentEntryStore(WRITER)
-                    .newLogSegmentAllocator(logMetadata, dynConf);
-        } catch (IOException ioe) {
-            FutureUtils.setException(createPromise, ioe);
-            return;
-        }
-
-        // Make sure writer handler created before resources are initialized
-        final BKLogWriteHandler writeHandler = new BKLogWriteHandler(
-                logMetadata,
-                conf,
-                driver.getLogStreamMetadataStore(WRITER),
-                logSegmentMetadataCache,
-                driver.getLogSegmentEntryStore(WRITER),
-                scheduler,
-                segmentAllocator,
-                statsLogger,
-                perLogStatsLogger,
-                alertStatsLogger,
-                clientId,
-                regionId,
-                writeLimiter,
-                featureProvider,
-                dynConf,
-                lock);
-        if (lockHandler) {
-            writeHandler.lockHandler().addEventListener(new FutureEventListener<DistributedLock>() {
-                @Override
-                public void onSuccess(DistributedLock lock) {
-                    FutureUtils.setValue(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;
-                        }
-                    });
-                }
-            });
-        } else {
-            FutureUtils.setValue(createPromise, writeHandler);
-        }
-    }
-
-    PermitManager getLogSegmentRollingPermitManager() {
-        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);
-            }
-        });
-    }
-
-    /**
-     * Check if an end of stream marker was added to the stream
-     * A stream with an end of stream marker cannot be appended to
-     *
-     * @return true if the marker was added to the stream, false otherwise
-     */
-    @Override
-    public boolean isEndOfStreamMarked() throws IOException {
-        checkClosedOrInError("isEndOfStreamMarked");
-        long lastTxId = FutureUtils.result(getLastLogRecordAsyncInternal(false, true)).getTransactionId();
-        return lastTxId == DistributedLogConstants.MAX_TXID;
-    }
-
-    /**
-     * 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 {
-        long position;
-        try {
-            position = FutureUtils.result(getLastLogRecordAsyncInternal(true, false)).getTransactionId();
-            if (DistributedLogConstants.INVALID_TXID == position ||
-                DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID == position) {
-                position = 0;
-            }
-        } catch (LogEmptyException ex) {
-            position = 0;
-        } catch (LogNotFoundException ex) {
-            position = 0;
-        }
-        return new AppendOnlyStreamWriter(startAsyncLogSegmentNonPartitioned(), position);
-    }
-
-    /**
-     * 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 {
-        return new AppendOnlyStreamReader(this);
-    }
-
-    /**
-     * Begin writing to the log stream identified by the name
-     *
-     * @return the writer interface to generate log records
-     */
-    @Override
-    public BKSyncLogWriter startLogSegmentNonPartitioned() throws IOException {
-        checkClosedOrInError("startLogSegmentNonPartitioned");
-        BKSyncLogWriter writer = new BKSyncLogWriter(conf, dynConf, this);
-        boolean success = false;
-        try {
-            writer.createAndCacheWriteHandler();
-            BKLogWriteHandler writeHandler = writer.getWriteHandler();
-            FutureUtils.result(writeHandler.lockHandler());
-            success = true;
-            return writer;
-        } finally {
-            if (!success) {
-                writer.abort();
-            }
-        }
-    }
-
-    /**
-     * Begin writing to the log stream identified by the name
-     *
-     * @return the writer interface to generate log records
-     */
-    @Override
-    public BKAsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException {
-        return (BKAsyncLogWriter) FutureUtils.result(openAsyncLogWriter());
-    }
-
-    @Override
-    public Future<AsyncLogWriter> openAsyncLogWriter() {
-        try {
-            checkClosedOrInError("startLogSegmentNonPartitioned");
-        } catch (AlreadyClosedException e) {
-            return Future.exception(e);
-        }
-
-        Future<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;
-                            }
-                        });
-            }
-        });
-    }
-
-    @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);
-            }
-        });
-    }
-
-    private Future<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 getDLSNNotLessThanTxIdInSegment(
-                fromTxnId,
-                segmentIdx,
-                segments,
-                driver.getLogSegmentEntryStore(READER)
-        );
-    }
-
-    private Future<DLSN> getDLSNNotLessThanTxIdInSegment(final long fromTxnId,
-                                                         final int segmentIdx,
-                                                         final List<LogSegmentMetadata> segments,
-                                                         final LogSegmentEntryStore entryStore) {
-        final LogSegmentMetadata segment = segments.get(segmentIdx);
-        return ReadUtils.getLogRecordNotLessThanTxId(
-                name,
-                segment,
-                fromTxnId,
-                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);
-                }
-            }
-        });
-    }
-
-    /**
-     * 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.
-     */
-    @Override
-    public LogReader getInputStream(long fromTxnId)
-        throws IOException {
-        return getInputStreamInternal(fromTxnId);
-    }
-
-    @Override
-    public LogReader getInputStream(DLSN fromDLSN) throws IOException {
-        return getInputStreamInternal(fromDLSN, Optional.<Long>absent());
-    }
-
-    @Override
-    public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException {
-        return FutureUtils.result(openAsyncLogReader(fromTxnId));
-    }
-
-    /**
-     * Opening a log reader positioning by transaction id <code>fromTxnId</code>.
-     *
-     * <p>
-     * - retrieve log segments for the stream
-     * - if the log segment list is empty, positioning by the last dlsn
-     * - otherwise, find the first log segment that contains the records whose transaction ids are not less than
-     *   the provided transaction id <code>fromTxnId</code>
-     *   - if all log segments' records' transaction ids are more than <code>fromTxnId</code>, positioning
-     *     on the first record.
-     *   - otherwise, search the log segment to find the log record
-     *     - if the log record is found, positioning the reader by that found record's dlsn
-     *     - otherwise, positioning by the last dlsn
-     * </p>
-     *
-     * @see DLUtils#findLogSegmentNotLessThanTxnId(List, long)
-     * @see ReadUtils#getLogRecordNotLessThanTxId(String, LogSegmentMetadata, long, ExecutorService, LogSegmentEntryStore, int)
-     * @param fromTxnId
-     *          transaction id to start reading from
-     * @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>() {
-
-            @Override
-            public void onSuccess(DLSN dlsn) {
-                dlsnPromise.setValue(dlsn);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                if (cause instanceof LogEmptyException) {
-                    dlsnPromise.setValue(DLSN.InitialDLSN);
-                } else {
-                    dlsnPromise.setException(cause);
-                }
-            }
-        });
-        return dlsnPromise.flatMap(new AbstractFunction1<DLSN, Future<AsyncLogReader>>() {
-            @Override
-            public Future<AsyncLogReader> apply(DLSN dlsn) {
-                return openAsyncLogReader(dlsn);
-            }
-        });
-    }
-
-    @Override
-    public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException {
-        return FutureUtils.result(openAsyncLogReader(fromDLSN));
-    }
-
-    @Override
-    public Future<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN) {
-        Optional<String> subscriberId = Optional.absent();
-        AsyncLogReader reader = new BKAsyncLogReader(
-                this,
-                scheduler,
-                fromDLSN,
-                subscriberId,
-                false,
-                statsLogger);
-        pendingReaders.add(reader);
-        return Future.value(reader);
-    }
-
-    /**
-     * Note the lock here is a sort of elective exclusive lock. I.e. acquiring this lock will only prevent other
-     * people who try to acquire the lock from reading from the stream. Normal readers (and writers) will not be
-     * blocked.
-     */
-    @Override
-    public Future<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) {
-        return getAsyncLogReaderWithLock(Optional.of(fromDLSN), Optional.of(subscriberId));
-    }
-
-    @Override
-    public Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId) {
-        Optional<DLSN> fromDLSN = Optional.absent();
-        return getAsyncLogReaderWithLock(fromDLSN, Optional.of(subscriberId));
-    }
-
-    protected Future<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."));
-        }
-        final BKAsyncLogReader reader = new BKAsyncLogReader(
-                BKDistributedLogManager.this,
-                scheduler,
-                fromDLSN.isPresent() ? fromDLSN.get() : DLSN.InitialDLSN,
-                subscriberId,
-                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) {
-                // cancel the lock when the creation future is cancelled
-                lockFuture.cancel();
-                return BoxedUnit.UNIT;
-            }
-        });
-        // lock the stream - fetch the last commit position on success
-        lockFuture.flatMap(new Function<Void, Future<AsyncLogReader>>() {
-            @Override
-            public Future<AsyncLogReader> apply(Void complete) {
-                if (fromDLSN.isPresent()) {
-                    return Future.value((AsyncLogReader) 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;
-                    }
-                });
-            }
-        }).addEventListener(new FutureEventListener<AsyncLogReader>() {
-            @Override
-            public void onSuccess(AsyncLogReader r) {
-                pendingReaders.remove(reader);
-                FutureUtils.setValue(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;
-                    }
-                });
-            }
-        });
-        return createPromise;
-    }
-
-    /**
-     * Get the input stream starting with fromTxnId for the specified log
-     *
-     * @param fromTxnId
-     *          transaction id to start reading from
-     * @return log reader
-     * @throws IOException
-     */
-    LogReader getInputStreamInternal(long fromTxnId)
-        throws IOException {
-        DLSN fromDLSN;
-        try {
-            fromDLSN = FutureUtils.result(getDLSNNotLessThanTxId(fromTxnId));
-        } catch (LogEmptyException lee) {
-            fromDLSN = DLSN.InitialDLSN;
-        }
-        return getInputStreamInternal(fromDLSN, Optional.of(fromTxnId));
-    }
-
-    LogReader getInputStreamInternal(DLSN fromDLSN, Optional<Long> fromTxnId)
-            throws IOException {
-        LOG.info("Create sync reader starting from {}", fromDLSN);
-        checkClosedOrInError("getInputStream");
-        return new BKSyncLogReader(
-                conf,
-                this,
-                fromDLSN,
-                fromTxnId,
-                statsLogger);
-    }
-
-    /**
-     * Get the last log record in the stream
-     *
-     * @return the last log record in the stream
-     * @throws java.io.IOException if a stream cannot be found.
-     */
-    @Override
-    public LogRecordWithDLSN getLastLogRecord() throws IOException {
-        checkClosedOrInError("getLastLogRecord");
-        return FutureUtils.result(getLastLogRecordAsync());
-    }
-
-    @Override
-    public long getFirstTxId() throws IOException {
-        checkClosedOrInError("getFirstTxId");
-        return FutureUtils.result(getFirstRecordAsyncInternal()).getTransactionId();
-    }
-
-    @Override
-    public long getLastTxId() throws IOException {
-        checkClosedOrInError("getLastTxId");
-        return FutureUtils.result(getLastTxIdAsync());
-    }
-
-    @Override
-    public DLSN getLastDLSN() throws IOException {
-        checkClosedOrInError("getLastDLSN");
-        return FutureUtils.result(getLastLogRecordAsyncInternal(false, false)).getDlsn();
-    }
-
-    /**
-     * Get Latest log record in the log
-     *
-     * @return latest log record
-     */
-    @Override
-    public Future<LogRecordWithDLSN> getLastLogRecordAsync() {
-        return getLastLogRecordAsyncInternal(false, false);
-    }
-
-    private Future<LogRecordWithDLSN> getLastLogRecordAsyncInternal(final boolean recover,
-                                                                    final boolean includeEndOfStream) {
-        return processReaderOperation(new Function<BKLogReadHandler, Future<LogRecordWithDLSN>>() {
-            @Override
-            public Future<LogRecordWithDLSN> apply(final BKLogReadHandler ledgerHandler) {
-                return ledgerHandler.getLastLogRecordAsync(recover, includeEndOfStream);
-            }
-        });
-    }
-
-    /**
-     * Get Latest Transaction Id in the log
-     *
-     * @return latest transaction id
-     */
-    @Override
-    public Future<Long> getLastTxIdAsync() {
-        return getLastLogRecordAsyncInternal(false, false)
-                .map(RECORD_2_TXID_FUNCTION);
-    }
-
-    /**
-     * Get first DLSN in the log.
-     *
-     * @return first dlsn in the stream
-     */
-    @Override
-    public Future<DLSN> getFirstDLSNAsync() {
-        return getFirstRecordAsyncInternal().map(RECORD_2_DLSN_FUNCTION);
-    }
-
-    private Future<LogRecordWithDLSN> getFirstRecordAsyncInternal() {
-        return processReaderOperation(new Function<BKLogReadHandler, Future<LogRecordWithDLSN>>() {
-            @Override
-            public Future<LogRecordWithDLSN> apply(final BKLogReadHandler ledgerHandler) {
-                return ledgerHandler.asyncGetFirstLogRecord();
-            }
-        });
-    }
-
-    /**
-     * Get Latest DLSN in the log.
-     *
-     * @return latest transaction id
-     */
-    @Override
-    public Future<DLSN> getLastDLSNAsync() {
-        return getLastLogRecordAsyncInternal(false, false)
-                .map(RECORD_2_DLSN_FUNCTION);
-    }
-
-    /**
-     * 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
-     *
-     * @return number of log records
-     * @throws IOException
-     */
-    @Override
-    public long getLogRecordCount() throws IOException {
-        checkClosedOrInError("getLogRecordCount");
-        return FutureUtils.result(getLogRecordCountAsync(DLSN.InitialDLSN));
-    }
-
-    /**
-     * Get the number of log records in the active portion of the log asynchronously.
-     * Any log segments that have already been truncated will not be included
-     *
-     * @return future number of log records
-     * @throws IOException
-     */
-    @Override
-    public Future<Long> getLogRecordCountAsync(final DLSN beginDLSN) {
-        return processReaderOperation(new Function<BKLogReadHandler, Future<Long>>() {
-                    @Override
-                    public Future<Long> apply(BKLogReadHandler ledgerHandler) {
-                        return ledgerHandler.asyncGetLogRecordCount(beginDLSN);
-                    }
-                });
-    }
-
-    @Override
-    public void recover() throws IOException {
-        recoverInternal(conf.getUnpartitionedStreamName());
-    }
-
-    /**
-     * Recover a specified stream within the log container
-     * The writer implicitly recovers a topic when it resumes writing.
-     * This allows applications to recover a container explicitly so
-     * that application may read a fully recovered log before resuming
-     * the writes
-     *
-     * @throws IOException if the recovery fails
-     */
-    private void recoverInternal(String streamIdentifier) throws IOException {
-        checkClosedOrInError("recoverInternal");
-        BKLogWriteHandler ledgerHandler = createWriteHandler(true);
-        try {
-            FutureUtils.result(ledgerHandler.recoverIncompleteLogSegments());
-        } finally {
-            Utils.closeQuietly(ledgerHandler);
-        }
-    }
-
-    /**
-     * Delete all the partitions of the specified log
-     *
-     * @throws IOException if the deletion fails
-     */
-    @Override
-    public void delete() throws IOException {
-        FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
-                .deleteLog(uri, getStreamName()));
-    }
-
-    /**
-     * The DistributedLogManager may archive/purge any logs for transactionId
-     * less than or equal to minImageTxId.
-     * This is to be used only when the client explicitly manages deletion. If
-     * the cleanup policy is based on sliding time window, then this method need
-     * not be called.
-     *
-     * @param minTxIdToKeep the earliest txid that must be retained
-     * @throws IOException if purging fails
-     */
-    @Override
-    public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException {
-        Preconditions.checkArgument(minTxIdToKeep > 0, "Invalid transaction id " + minTxIdToKeep);
-        checkClosedOrInError("purgeLogSegmentsOlderThan");
-        BKLogWriteHandler ledgerHandler = createWriteHandler(true);
-        try {
-            LOG.info("Purging logs for {} older than {}", ledgerHandler.getFullyQualifiedName(), minTxIdToKeep);
-            FutureUtils.result(ledgerHandler.purgeLogSegmentsOlderThanTxnId(minTxIdToKeep));
-        } finally {
-            Utils.closeQuietly(ledgerHandler);
-        }
-    }
-
-    static class PendingReaders implements AsyncCloseable {
-
-        final ExecutorService executorService;
-        final Set<AsyncCloseable> readers = new HashSet<AsyncCloseable>();
-
-        PendingReaders(ExecutorService executorService) {
-            this.executorService = executorService;
-        }
-
-        public synchronized void remove(AsyncCloseable reader) {
-            readers.remove(reader);
-        }
-
-        public synchronized void add(AsyncCloseable reader) {
-            readers.add(reader);
-        }
-
-        @Override
-        public Future<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;
-                        }
-                    });
-        }
-    };
-
-    /**
-     * Close the distributed log manager, freeing any resources it may hold.
-     */
-    @Override
-    public Future<Void> asyncClose() {
-        Promise<Void> closeFuture;
-        BKLogReadHandler readHandlerToClose;
-        synchronized (this) {
-            if (null != closePromise) {
-                return closePromise;
-            }
-            closeFuture = closePromise = new Promise<Void>();
-            readHandlerToClose = readHandlerForListener;
-        }
-
-        Future<Void> closeResult = Utils.closeSequence(null, true,
-                readHandlerToClose,
-                pendingReaders,
-                resourcesCloseable.or(AsyncCloseable.NULL));
-        closeResult.proxyTo(closeFuture);
-        return closeFuture;
-    }
-
-    @Override
-    public void close() throws IOException {
-        FutureUtils.result(asyncClose());
-    }
-
-    @Override
-    public String toString() {
-        return String.format("DLM:%s:%s", getUri(), getStreamName());
-    }
-
-    public void raiseAlert(String msg, Object... args) {
-        alertStatsLogger.raise(msg, args);
-    }
-
-    @Override
-    public SubscriptionsStore getSubscriptionsStore() {
-        return driver.getSubscriptionsStore(getStreamName());
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java
deleted file mode 100644
index a8b1f77..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java
+++ /dev/null
@@ -1,320 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Ticker;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.AlreadyClosedException;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.NamespaceDriver;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.PermitLimiter;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.distributedlog.util.Utils;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Iterator;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static com.twitter.distributedlog.namespace.NamespaceDriver.Role.WRITER;
-import static com.twitter.distributedlog.util.DLUtils.validateName;
-
-/**
- * BKDistributedLogNamespace is the default implementation of {@link DistributedLogNamespace}. It uses
- * zookeeper for metadata storage and bookkeeper for data storage.
- * <h3>Metrics</h3>
- *
- * <h4>ZooKeeper Client</h4>
- * See {@link ZooKeeperClient} for detail sub-stats.
- * <ul>
- * <li> `scope`/dlzk_factory_writer_shared/* : stats about the zookeeper client shared by all DL writers.
- * <li> `scope`/dlzk_factory_reader_shared/* : stats about the zookeeper client shared by all DL readers.
- * <li> `scope`/bkzk_factory_writer_shared/* : stats about the zookeeper client used by bookkeeper client
- * shared by all DL writers.
- * <li> `scope`/bkzk_factory_reader_shared/* : stats about the zookeeper client used by bookkeeper client
- * shared by all DL readers.
- * </ul>
- *
- * <h4>BookKeeper Client</h4>
- * BookKeeper client stats are exposed directly to current scope. See {@link BookKeeperClient} for detail stats.
- *
- * <h4>Utils</h4>
- * <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>
- *
- * <h4>DistributedLogManager</h4>
- *
- * All the core stats about reader and writer are exposed under current scope via {@link BKDistributedLogManager}.
- */
-public class BKDistributedLogNamespace implements DistributedLogNamespace {
-    static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogNamespace.class);
-
-    private final String clientId;
-    private final int regionId;
-    private final DistributedLogConfiguration conf;
-    private final URI namespace;
-    // namespace driver
-    private final NamespaceDriver driver;
-    // resources
-    private final OrderedScheduler scheduler;
-    private final PermitLimiter writeLimiter;
-    private final AsyncFailureInjector failureInjector;
-    // log segment metadata store
-    private final LogSegmentMetadataCache logSegmentMetadataCache;
-    // feature provider
-    private final FeatureProvider featureProvider;
-    // Stats Loggers
-    private final StatsLogger statsLogger;
-    private final StatsLogger perLogStatsLogger;
-
-    protected final AtomicBoolean closed = new AtomicBoolean(false);
-
-    public BKDistributedLogNamespace(
-            DistributedLogConfiguration conf,
-            URI uri,
-            NamespaceDriver driver,
-            OrderedScheduler scheduler,
-            FeatureProvider featureProvider,
-            PermitLimiter writeLimiter,
-            AsyncFailureInjector failureInjector,
-            StatsLogger statsLogger,
-            StatsLogger perLogStatsLogger,
-            String clientId,
-            int regionId) {
-        this.conf = conf;
-        this.namespace = uri;
-        this.driver = driver;
-        this.scheduler = scheduler;
-        this.featureProvider = featureProvider;
-        this.writeLimiter = writeLimiter;
-        this.failureInjector = failureInjector;
-        this.statsLogger = statsLogger;
-        this.perLogStatsLogger = perLogStatsLogger;
-        this.clientId = clientId;
-        this.regionId = regionId;
-
-        // create a log segment metadata cache
-        this.logSegmentMetadataCache = new LogSegmentMetadataCache(conf, Ticker.systemTicker());
-    }
-
-    @Override
-    public NamespaceDriver getNamespaceDriver() {
-        return driver;
-    }
-
-    //
-    // Namespace Methods
-    //
-
-    @Override
-    public void createLog(String logName)
-            throws InvalidStreamNameException, IOException {
-        checkState();
-        validateName(logName);
-        URI uri = FutureUtils.result(driver.getLogMetadataStore().createLog(logName));
-        FutureUtils.result(driver.getLogStreamMetadataStore(WRITER).getLog(uri, logName, true, true));
-    }
-
-    @Override
-    public void deleteLog(String logName)
-            throws InvalidStreamNameException, LogNotFoundException, IOException {
-        checkState();
-        validateName(logName);
-        Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
-        if (!uri.isPresent()) {
-            throw new LogNotFoundException("Log " + logName + " isn't found.");
-        }
-        DistributedLogManager dlm = openLogInternal(
-                uri.get(),
-                logName,
-                Optional.<DistributedLogConfiguration>absent(),
-                Optional.<DynamicDistributedLogConfiguration>absent());
-        dlm.delete();
-    }
-
-    @Override
-    public DistributedLogManager openLog(String logName)
-            throws InvalidStreamNameException, IOException {
-        return openLog(logName,
-                Optional.<DistributedLogConfiguration>absent(),
-                Optional.<DynamicDistributedLogConfiguration>absent(),
-                Optional.<StatsLogger>absent());
-    }
-
-    @Override
-    public DistributedLogManager openLog(String logName,
-                                         Optional<DistributedLogConfiguration> logConf,
-                                         Optional<DynamicDistributedLogConfiguration> dynamicLogConf,
-                                         Optional<StatsLogger> perStreamStatsLogger)
-            throws InvalidStreamNameException, IOException {
-        checkState();
-        validateName(logName);
-        Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
-        if (!uri.isPresent()) {
-            throw new LogNotFoundException("Log " + logName + " isn't found.");
-        }
-        return openLogInternal(
-                uri.get(),
-                logName,
-                logConf,
-                dynamicLogConf);
-    }
-
-    @Override
-    public boolean logExists(String logName)
-        throws IOException, IllegalArgumentException {
-        checkState();
-        Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
-        if (uri.isPresent()) {
-            try {
-                FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
-                        .logExists(uri.get(), logName));
-                return true;
-            } catch (LogNotFoundException lnfe) {
-                return false;
-            }
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public Iterator<String> getLogs() throws IOException {
-        checkState();
-        return FutureUtils.result(driver.getLogMetadataStore().getLogs());
-    }
-
-    @Override
-    public void registerNamespaceListener(NamespaceListener listener) {
-        driver.getLogMetadataStore().registerNamespaceListener(listener);
-    }
-
-    @Override
-    public synchronized AccessControlManager createAccessControlManager() throws IOException {
-        checkState();
-        return driver.getAccessControlManager();
-    }
-
-    /**
-     * Open the log in location <i>uri</i>.
-     *
-     * @param uri
-     *          location to store the log
-     * @param nameOfLogStream
-     *          name of the log
-     * @param logConfiguration
-     *          optional stream configuration
-     * @param dynamicLogConfiguration
-     *          dynamic stream configuration overrides.
-     * @return distributedlog manager instance.
-     * @throws InvalidStreamNameException if the stream name is invalid
-     * @throws IOException
-     */
-    protected DistributedLogManager openLogInternal(
-            URI uri,
-            String nameOfLogStream,
-            Optional<DistributedLogConfiguration> logConfiguration,
-            Optional<DynamicDistributedLogConfiguration> dynamicLogConfiguration)
-        throws InvalidStreamNameException, IOException {
-        // Make sure the name is well formed
-        checkState();
-        validateName(nameOfLogStream);
-
-        DistributedLogConfiguration mergedConfiguration = new DistributedLogConfiguration();
-        mergedConfiguration.addConfiguration(conf);
-        mergedConfiguration.loadStreamConf(logConfiguration);
-        // If dynamic config was not provided, default to a static view of the global configuration.
-        DynamicDistributedLogConfiguration dynConf = null;
-        if (dynamicLogConfiguration.isPresent()) {
-            dynConf = dynamicLogConfiguration.get();
-        } else {
-            dynConf = ConfUtils.getConstDynConf(mergedConfiguration);
-        }
-
-        return new BKDistributedLogManager(
-                nameOfLogStream,                    /* Log Name */
-                mergedConfiguration,                /* Configuration */
-                dynConf,                            /* Dynamic Configuration */
-                uri,                                /* Namespace URI */
-                driver,                             /* Namespace Driver */
-                logSegmentMetadataCache,            /* Log Segment Metadata Cache */
-                scheduler,                          /* DL scheduler */
-                clientId,                           /* Client Id */
-                regionId,                           /* Region Id */
-                writeLimiter,                       /* Write Limiter */
-                featureProvider.scope("dl"),        /* Feature Provider */
-                failureInjector,                    /* Failure Injector */
-                statsLogger,                        /* Stats Logger */
-                perLogStatsLogger,                  /* Per Log Stats Logger */
-                Optional.<AsyncCloseable>absent()   /* shared resources, we don't need to close any resources in dlm */
-        );
-    }
-
-    /**
-     * Check the namespace state.
-     *
-     * @throws IOException
-     */
-    private void checkState() throws IOException {
-        if (closed.get()) {
-            LOG.error("BK namespace {} is already closed", namespace);
-            throw new AlreadyClosedException("BK namespace " + namespace + " is already closed");
-        }
-    }
-
-    /**
-     * Close the distributed log manager factory, freeing any resources it may hold.
-     */
-    @Override
-    public void close() {
-        if (!closed.compareAndSet(false, true)) {
-            return;
-        }
-        // shutdown the driver
-        Utils.close(driver);
-        // close the write limiter
-        this.writeLimiter.close();
-        // Shutdown the schedulers
-        SchedulerUtils.shutdownScheduler(scheduler, conf.getSchedulerShutdownTimeoutMs(),
-                TimeUnit.MILLISECONDS);
-        LOG.info("Executor Service Stopped.");
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogHandler.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogHandler.java
deleted file mode 100644
index 0cf8ed5..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogHandler.java
+++ /dev/null
@@ -1,715 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.callback.LogSegmentNamesListener;
-import com.twitter.distributedlog.exceptions.LogEmptyException;
-import com.twitter.distributedlog.exceptions.LogSegmentNotFoundException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.metadata.LogMetadata;
-import com.twitter.distributedlog.io.AsyncAbortable;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.logsegment.PerStreamLogSegmentCache;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.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.
- *
- * <h3>Metrics</h3>
- * The log handler is a base class on managing log segments. so all the metrics
- * here are related to log segments retrieval and exposed under `logsegments`.
- * These metrics are all OpStats, in the format of <code>`scope`/logsegments/`op`</code>.
- * <p>
- * Those operations are:
- * <ul>
- * <li>get_inprogress_segment: time between the inprogress log segment created and
- * the handler read it.
- * <li>get_completed_segment: time between a log segment is turned to completed and
- * the handler read it.
- * <li>negative_get_inprogress_segment: record the negative values for `get_inprogress_segment`.
- * <li>negative_get_completed_segment: record the negative values for `get_completed_segment`.
- * <li>recover_last_entry: recovering last entry from a log segment
- * <li>recover_scanned_entries: the number of entries that are scanned during recovering.
- * </ul>
- * @see BKLogWriteHandler
- * @see BKLogReadHandler
- */
-public abstract class BKLogHandler implements AsyncCloseable, AsyncAbortable {
-    static final Logger LOG = LoggerFactory.getLogger(BKLogHandler.class);
-
-    protected final LogMetadata logMetadata;
-    protected final DistributedLogConfiguration conf;
-    protected final LogStreamMetadataStore streamMetadataStore;
-    protected final LogSegmentMetadataStore metadataStore;
-    protected final LogSegmentMetadataCache metadataCache;
-    protected final LogSegmentEntryStore entryStore;
-    protected final int firstNumEntriesPerReadLastRecordScan;
-    protected final int maxNumEntriesPerReadLastRecordScan;
-    protected volatile long lastLedgerRollingTimeMillis = -1;
-    protected final OrderedScheduler scheduler;
-    protected final StatsLogger statsLogger;
-    protected final AlertStatsLogger alertStatsLogger;
-    protected volatile boolean reportGetSegmentStats = false;
-    private final String lockClientId;
-    protected final AtomicReference<IOException> metadataException = new AtomicReference<IOException>(null);
-
-    // Maintain the list of log segments per stream
-    protected final PerStreamLogSegmentCache logSegmentCache;
-
-    // trace
-    protected final long metadataLatencyWarnThresholdMillis;
-
-    // Stats
-    private final OpStatsLogger getInprogressSegmentStat;
-    private final OpStatsLogger getCompletedSegmentStat;
-    private final OpStatsLogger negativeGetInprogressSegmentStat;
-    private final OpStatsLogger negativeGetCompletedSegmentStat;
-    private final OpStatsLogger recoverLastEntryStats;
-    private final OpStatsLogger recoverScannedEntriesStats;
-
-    /**
-     * Construct a Bookkeeper journal manager.
-     */
-    BKLogHandler(LogMetadata metadata,
-                 DistributedLogConfiguration conf,
-                 LogStreamMetadataStore streamMetadataStore,
-                 LogSegmentMetadataCache metadataCache,
-                 LogSegmentEntryStore entryStore,
-                 OrderedScheduler scheduler,
-                 StatsLogger statsLogger,
-                 AlertStatsLogger alertStatsLogger,
-                 String lockClientId) {
-        this.logMetadata = metadata;
-        this.conf = conf;
-        this.scheduler = scheduler;
-        this.statsLogger = statsLogger;
-        this.alertStatsLogger = alertStatsLogger;
-        this.logSegmentCache = new PerStreamLogSegmentCache(
-                metadata.getLogName(),
-                conf.isLogSegmentSequenceNumberValidationEnabled());
-        firstNumEntriesPerReadLastRecordScan = conf.getFirstNumEntriesPerReadLastRecordScan();
-        maxNumEntriesPerReadLastRecordScan = conf.getMaxNumEntriesPerReadLastRecordScan();
-        this.streamMetadataStore = streamMetadataStore;
-        this.metadataStore = streamMetadataStore.getLogSegmentMetadataStore();
-        this.metadataCache = metadataCache;
-        this.entryStore = entryStore;
-        this.lockClientId = lockClientId;
-
-        // Traces
-        this.metadataLatencyWarnThresholdMillis = conf.getMetadataLatencyWarnThresholdMillis();
-
-        // Stats
-        StatsLogger segmentsLogger = statsLogger.scope("logsegments");
-        getInprogressSegmentStat = segmentsLogger.getOpStatsLogger("get_inprogress_segment");
-        getCompletedSegmentStat = segmentsLogger.getOpStatsLogger("get_completed_segment");
-        negativeGetInprogressSegmentStat = segmentsLogger.getOpStatsLogger("negative_get_inprogress_segment");
-        negativeGetCompletedSegmentStat = segmentsLogger.getOpStatsLogger("negative_get_completed_segment");
-        recoverLastEntryStats = segmentsLogger.getOpStatsLogger("recover_last_entry");
-        recoverScannedEntriesStats = segmentsLogger.getOpStatsLogger("recover_scanned_entries");
-    }
-
-    BKLogHandler checkMetadataException() throws IOException {
-        if (null != metadataException.get()) {
-            throw metadataException.get();
-        }
-        return this;
-    }
-
-    public void reportGetSegmentStats(boolean enabled) {
-        this.reportGetSegmentStats = enabled;
-    }
-
-    public String getLockClientId() {
-        return lockClientId;
-    }
-
-    public Future<LogRecordWithDLSN> asyncGetFirstLogRecord() {
-        final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
-        streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
-                .addEventListener(new FutureEventListener<Void>() {
-            @Override
-            public void onSuccess(Void value) {
-                readLogSegmentsFromStore(
-                        LogSegmentMetadata.COMPARATOR,
-                        LogSegmentFilter.DEFAULT_FILTER,
-                        null
-                ).addEventListener(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"));
-                            return;
-                        }
-                        Future<LogRecordWithDLSN> firstRecord = null;
-                        for (LogSegmentMetadata ledger : ledgerList.getValue()) {
-                            if (!ledger.isTruncated() && (ledger.getRecordCount() > 0 || ledger.isInProgress())) {
-                                firstRecord = asyncReadFirstUserRecord(ledger, DLSN.InitialDLSN);
-                                break;
-                            }
-                        }
-                        if (null != firstRecord) {
-                            promise.become(firstRecord);
-                        } else {
-                            promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
-                        }
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        promise.setException(cause);
-                    }
-                });
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                promise.setException(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>() {
-            @Override
-            public void onSuccess(Void value) {
-                readLogSegmentsFromStore(
-                        LogSegmentMetadata.DESC_COMPARATOR,
-                        LogSegmentFilter.DEFAULT_FILTER,
-                        null
-                ).addEventListener(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"));
-                            return;
-                        }
-                        asyncGetLastLogRecord(
-                                ledgerList.getValue().iterator(),
-                                promise,
-                                recover,
-                                false,
-                                includeEndOfStream);
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        promise.setException(cause);
-                    }
-                });
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                promise.setException(cause);
-            }
-        });
-        return promise;
-    }
-
-    private void asyncGetLastLogRecord(final Iterator<LogSegmentMetadata> ledgerIter,
-                                       final Promise<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);
-                        }
-                    }
-            );
-        } else {
-            promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
-        }
-    }
-
-    private Future<LogRecordWithDLSN> asyncReadFirstUserRecord(LogSegmentMetadata ledger, DLSN beginDLSN) {
-        return ReadUtils.asyncReadFirstUserRecord(
-                getFullyQualifiedName(),
-                ledger,
-                firstNumEntriesPerReadLastRecordScan,
-                maxNumEntriesPerReadLastRecordScan,
-                new AtomicInteger(0),
-                scheduler,
-                entryStore,
-                beginDLSN
-        );
-    }
-
-    /**
-     * This is a helper method to compactly return the record count between two records, the first denoted by
-     * 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;
-            }
-        });
-    }
-
-    /**
-     * Ledger metadata tells us how many records are in each completed segment, but for the first and last segments
-     * we may have to crack open the entry and count. For the first entry, we need to do so because beginDLSN may be
-     * 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) {
-        if (ledger.isInProgress() && ledger.isDLSNinThisSegment(beginDLSN)) {
-            return asyncReadLastUserRecord(ledger).flatMap(new Function<LogRecordWithDLSN, Future<Long>>() {
-                public Future<Long> apply(final LogRecordWithDLSN endRecord) {
-                    if (null != endRecord) {
-                        return asyncGetLogRecordCount(ledger, beginDLSN, endRecord.getLastPositionWithinLogSegment() /* end position */);
-                    } else {
-                        return Future.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;
-                    }
-                }
-            });
-        } else if (ledger.isDLSNinThisSegment(beginDLSN)) {
-            return asyncGetLogRecordCount(ledger, beginDLSN, ledger.getRecordCount() /* end position */);
-        } else {
-            return Future.value((long) ledger.getRecordCount());
-        }
-    }
-
-    /**
-     * Get a count of records between beginDLSN and the end of the stream.
-     *
-     * @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) {
-        return streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
-                .flatMap(new Function<Void, Future<Long>>() {
-            public Future<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) {
-
-                        List<Future<Long>> futureCounts = new ArrayList<Future<Long>>(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);
-                            }
-                        });
-                    }
-                });
-            }
-        });
-    }
-
-    private Long sum(List<Long> values) {
-        long sum = 0;
-        for (Long value : values) {
-            sum += value;
-        }
-        return sum;
-    }
-
-    @Override
-    public Future<Void> asyncAbort() {
-        return asyncClose();
-    }
-
-    public Future<LogRecordWithDLSN> asyncReadLastUserRecord(final LogSegmentMetadata l) {
-        return asyncReadLastRecord(l, false, false, false);
-    }
-
-    public Future<LogRecordWithDLSN> asyncReadLastRecord(final LogSegmentMetadata l,
-                                                         final boolean fence,
-                                                         final boolean includeControl,
-                                                         final boolean includeEndOfStream) {
-        final AtomicInteger numRecordsScanned = new AtomicInteger(0);
-        final Stopwatch stopwatch = Stopwatch.createStarted();
-        return ReadUtils.asyncReadLastRecord(
-                getFullyQualifiedName(),
-                l,
-                fence,
-                includeControl,
-                includeEndOfStream,
-                firstNumEntriesPerReadLastRecordScan,
-                maxNumEntriesPerReadLastRecordScan,
-                numRecordsScanned,
-                scheduler,
-                entryStore
-        ).addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
-            @Override
-            public void onSuccess(LogRecordWithDLSN value) {
-                recoverLastEntryStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-                recoverScannedEntriesStats.registerSuccessfulEvent(numRecordsScanned.get());
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                recoverLastEntryStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            }
-        });
-    }
-
-    protected void setLastLedgerRollingTimeMillis(long rollingTimeMillis) {
-        if (lastLedgerRollingTimeMillis < rollingTimeMillis) {
-            lastLedgerRollingTimeMillis = rollingTimeMillis;
-        }
-    }
-
-    public String getFullyQualifiedName() {
-        return logMetadata.getFullyQualifiedName();
-    }
-
-    // Log Segments Related Functions
-    //
-    // ***Note***
-    // Get log segment list should go through #getCachedLogSegments as we need to assign start sequence id
-    // for inprogress log segment so the reader could generate the right sequence id.
-    //
-    // ***PerStreamCache vs LogSegmentMetadataCache **
-    // The per stream cache maintains the list of segments per stream, while the metadata cache
-    // maintains log segments. The metadata cache is just to reduce the access to zookeeper, it is
-    // okay that some of the log segments are not in the cache; however the per stream cache can not
-    // have any gaps between log segment sequence numbers which it has to be accurate.
-
-    /**
-     * Get the cached log segments.
-     *
-     * @param comparator the comparator to sort the returned log segments.
-     * @return list of sorted log segments
-     * @throws UnexpectedException if unexpected condition detected.
-     */
-    protected List<LogSegmentMetadata> getCachedLogSegments(Comparator<LogSegmentMetadata> comparator)
-        throws UnexpectedException {
-        try {
-            return logSegmentCache.getLogSegments(comparator);
-        } catch (UnexpectedException ue) {
-            // the log segments cache went wrong
-            LOG.error("Unexpected exception on getting log segments from the cache for stream {}",
-                    getFullyQualifiedName(), ue);
-            metadataException.compareAndSet(null, ue);
-            throw ue;
-        }
-    }
-
-    /**
-     * Add the segment <i>metadata</i> for <i>name</i> in the cache.
-     *
-     * @param name
-     *          segment znode name.
-     * @param metadata
-     *          segment metadata.
-     */
-    protected void addLogSegmentToCache(String name, LogSegmentMetadata metadata) {
-        metadataCache.put(metadata.getZkPath(), metadata);
-        logSegmentCache.add(name, metadata);
-        // update the last ledger rolling time
-        if (!metadata.isInProgress() && (lastLedgerRollingTimeMillis < metadata.getCompletionTime())) {
-            lastLedgerRollingTimeMillis = metadata.getCompletionTime();
-        }
-
-        if (reportGetSegmentStats) {
-            // update stats
-            long ts = System.currentTimeMillis();
-            if (metadata.isInProgress()) {
-                // as we used timestamp as start tx id we could take it as start time
-                // NOTE: it is a hack here.
-                long elapsedMillis = ts - metadata.getFirstTxId();
-                long elapsedMicroSec = TimeUnit.MILLISECONDS.toMicros(elapsedMillis);
-                if (elapsedMicroSec > 0) {
-                    if (elapsedMillis > metadataLatencyWarnThresholdMillis) {
-                        LOG.warn("{} received inprogress log segment in {} millis: {}",
-                                 new Object[] { getFullyQualifiedName(), elapsedMillis, metadata });
-                    }
-                    getInprogressSegmentStat.registerSuccessfulEvent(elapsedMicroSec);
-                } else {
-                    negativeGetInprogressSegmentStat.registerSuccessfulEvent(-elapsedMicroSec);
-                }
-            } else {
-                long elapsedMillis = ts - metadata.getCompletionTime();
-                long elapsedMicroSec = TimeUnit.MILLISECONDS.toMicros(elapsedMillis);
-                if (elapsedMicroSec > 0) {
-                    if (elapsedMillis > metadataLatencyWarnThresholdMillis) {
-                        LOG.warn("{} received completed log segment in {} millis : {}",
-                                 new Object[] { getFullyQualifiedName(), elapsedMillis, metadata });
-                    }
-                    getCompletedSegmentStat.registerSuccessfulEvent(elapsedMicroSec);
-                } else {
-                    negativeGetCompletedSegmentStat.registerSuccessfulEvent(-elapsedMicroSec);
-                }
-            }
-        }
-    }
-
-    /**
-     * Read log segment <i>name</i> from the cache.
-     *
-     * @param name name of the log segment
-     * @return log segment metadata
-     */
-    protected LogSegmentMetadata readLogSegmentFromCache(String name) {
-        return logSegmentCache.get(name);
-    }
-
-    /**
-     * Remove the log segment <i>name</i> from the cache.
-     *
-     * @param name name of the log segment.
-     * @return log segment metadata
-     */
-    protected LogSegmentMetadata removeLogSegmentFromCache(String name) {
-        metadataCache.invalidate(name);
-        return logSegmentCache.remove(name);
-    }
-
-    /**
-     * Update the log segment cache with updated mapping
-     *
-     * @param logSegmentsRemoved log segments removed
-     * @param logSegmentsAdded log segments added
-     */
-    protected void updateLogSegmentCache(Set<String> logSegmentsRemoved,
-                                         Map<String, LogSegmentMetadata> logSegmentsAdded) {
-        for (String segmentName : logSegmentsRemoved) {
-            metadataCache.invalidate(segmentName);
-        }
-        for (Map.Entry<String, LogSegmentMetadata> entry : logSegmentsAdded.entrySet()) {
-            metadataCache.put(entry.getKey(), entry.getValue());
-        }
-        logSegmentCache.update(logSegmentsRemoved, logSegmentsAdded);
-    }
-
-    /**
-     * Read the log segments from the store and register a listener
-     * @param comparator
-     * @param segmentFilter
-     * @param logSegmentNamesListener
-     * @return future represents the result of log segments
-     */
-    public Future<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>>>();
-        metadataStore.getLogSegmentNames(logMetadata.getLogSegmentsPath(), logSegmentNamesListener)
-                .addEventListener(new FutureEventListener<Versioned<List<String>>>() {
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        FutureUtils.setException(readResult, cause);
-                    }
-
-                    @Override
-                    public void onSuccess(Versioned<List<String>> logSegmentNames) {
-                        readLogSegmentsFromStore(logSegmentNames, comparator, segmentFilter, readResult);
-                    }
-                });
-        return readResult;
-    }
-
-    protected void readLogSegmentsFromStore(final Versioned<List<String>> logSegmentNames,
-                                            final Comparator<LogSegmentMetadata> comparator,
-                                            final LogSegmentFilter segmentFilter,
-                                            final Promise<Versioned<List<LogSegmentMetadata>>> readResult) {
-        Set<String> segmentsReceived = new HashSet<String>();
-        segmentsReceived.addAll(segmentFilter.filter(logSegmentNames.getValue()));
-        Set<String> segmentsAdded;
-        final Set<String> removedSegments = Collections.synchronizedSet(new HashSet<String>());
-        final Map<String, LogSegmentMetadata> addedSegments =
-                Collections.synchronizedMap(new HashMap<String, LogSegmentMetadata>());
-        Pair<Set<String>, Set<String>> segmentChanges = logSegmentCache.diff(segmentsReceived);
-        segmentsAdded = segmentChanges.getLeft();
-        removedSegments.addAll(segmentChanges.getRight());
-
-        if (segmentsAdded.isEmpty()) {
-            if (LOG.isTraceEnabled()) {
-                LOG.trace("No segments added for {}.", getFullyQualifiedName());
-            }
-
-            // update the cache before #getCachedLogSegments to return
-            updateLogSegmentCache(removedSegments, addedSegments);
-
-            List<LogSegmentMetadata> segmentList;
-            try {
-                segmentList = getCachedLogSegments(comparator);
-            } catch (UnexpectedException e) {
-                FutureUtils.setException(readResult, e);
-                return;
-            }
-
-            FutureUtils.setValue(readResult,
-                    new Versioned<List<LogSegmentMetadata>>(segmentList, logSegmentNames.getVersion()));
-            return;
-        }
-
-        final AtomicInteger numChildren = new AtomicInteger(segmentsAdded.size());
-        final AtomicInteger numFailures = new AtomicInteger(0);
-        for (final String segment: segmentsAdded) {
-            String logSegmentPath = logMetadata.getLogSegmentPath(segment);
-            LogSegmentMetadata cachedSegment = metadataCache.get(logSegmentPath);
-            if (null != cachedSegment) {
-                addedSegments.put(segment, cachedSegment);
-                completeReadLogSegmentsFromStore(
-                        removedSegments,
-                        addedSegments,
-                        comparator,
-                        readResult,
-                        logSegmentNames.getVersion(),
-                        numChildren,
-                        numFailures);
-                continue;
-            }
-            metadataStore.getLogSegment(logSegmentPath)
-                    .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
-
-                        @Override
-                        public void onSuccess(LogSegmentMetadata result) {
-                            addedSegments.put(segment, result);
-                            complete();
-                        }
-
-                        @Override
-                        public void onFailure(Throwable cause) {
-                            // LogSegmentNotFoundException exception is possible in two cases
-                            // 1. A log segment was deleted by truncation between the call to getChildren and read
-                            // attempt on the znode corresponding to the segment
-                            // 2. In progress segment has been completed => inprogress ZNode does not exist
-                            if (cause instanceof LogSegmentNotFoundException) {
-                                removedSegments.add(segment);
-                                complete();
-                            } else {
-                                // fail fast
-                                if (1 == numFailures.incrementAndGet()) {
-                                    FutureUtils.setException(readResult, cause);
-                                    return;
-                                }
-                            }
-                        }
-
-                        private void complete() {
-                            completeReadLogSegmentsFromStore(
-                                    removedSegments,
-                                    addedSegments,
-                                    comparator,
-                                    readResult,
-                                    logSegmentNames.getVersion(),
-                                    numChildren,
-                                    numFailures);
-                        }
-                    });
-        }
-    }
-
-    private void completeReadLogSegmentsFromStore(final Set<String> removedSegments,
-                                                  final Map<String, LogSegmentMetadata> addedSegments,
-                                                  final Comparator<LogSegmentMetadata> comparator,
-                                                  final Promise<Versioned<List<LogSegmentMetadata>>> readResult,
-                                                  final Version logSegmentNamesVersion,
-                                                  final AtomicInteger numChildren,
-                                                  final AtomicInteger numFailures) {
-        if (0 != numChildren.decrementAndGet()) {
-            return;
-        }
-        if (numFailures.get() > 0) {
-            return;
-        }
-        // update the cache only when fetch completed and before #getCachedLogSegments
-        updateLogSegmentCache(removedSegments, addedSegments);
-        List<LogSegmentMetadata> segmentList;
-        try {
-            segmentList = getCachedLogSegments(comparator);
-        } catch (UnexpectedException e) {
-            FutureUtils.setException(readResult, e);
-            return;
-        }
-        FutureUtils.setValue(readResult,
-            new Versioned<List<LogSegmentMetadata>>(segmentList, logSegmentNamesVersion));
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogReadHandler.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogReadHandler.java
deleted file mode 100644
index 8aa00e7..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogReadHandler.java
+++ /dev/null
@@ -1,431 +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 com.twitter.distributedlog;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.callback.LogSegmentListener;
-import com.twitter.distributedlog.callback.LogSegmentNamesListener;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.LogSegmentNotFoundException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.metadata.LogMetadataForReader;
-import com.twitter.distributedlog.lock.DistributedLock;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.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.
- * <h3>Metrics</h3>
- *
- * <h4>ReadAhead Worker</h4>
- * Most of readahead stats are exposed under scope `readahead_worker`. Only readahead exceptions are exposed
- * in parent scope via <code>readAheadExceptionsLogger</code>.
- * <ul>
- * <li> `readahead_worker`/wait: counter. number of waits that readahead worker is waiting. If this keeps increasing,
- * it usually means readahead keep getting full because of reader slows down reading.
- * <li> `readahead_worker`/repositions: counter. number of repositions that readhead worker encounters. reposition
- * means that a readahead worker finds that it isn't advancing to a new log segment and force re-positioning.
- * <li> `readahead_worker`/entry_piggy_back_hits: counter. it increases when the last add confirmed being advanced
- * because of the piggy-back lac.
- * <li> `readahead_worker`/entry_piggy_back_misses: counter. it increases when the last add confirmed isn't advanced
- * by a read entry because it doesn't piggy back a newer lac.
- * <li> `readahead_worker`/read_entries: opstats. stats on number of entries read per readahead read batch.
- * <li> `readahead_worker`/read_lac_counter: counter. stats on the number of readLastConfirmed operations
- * <li> `readahead_worker`/read_lac_and_entry_counter: counter. stats on the number of readLastConfirmedAndEntry
- * operations.
- * <li> `readahead_worker`/cache_full: counter. it increases each time readahead worker finds cache become full.
- * If it keeps increasing, that means reader slows down reading.
- * <li> `readahead_worker`/resume: opstats. stats on readahead worker resuming reading from wait state.
- * <li> `readahead_worker`/read_lac_lag: opstats. stats on the number of entries diff between the lac reader knew
- * last time and the lac that it received. if `lag` between two subsequent lacs is high, that might means delay
- * might be high. because reader is only allowed to read entries after lac is advanced.
- * <li> `readahead_worker`/long_poll_interruption: opstats. stats on the number of interruptions happened to long
- * poll. the interruptions are usually because of receiving zookeeper notifications.
- * <li> `readahead_worker`/notification_execution: opstats. stats on executions over the notifications received from
- * zookeeper.
- * <li> `readahead_worker`/metadata_reinitialization: opstats. stats on metadata reinitialization after receiving
- * notifcation from log segments updates.
- * <li> `readahead_worker`/idle_reader_warn: counter. it increases each time the readahead worker detects itself
- * becoming idle.
- * </ul>
- * <h4>Read Lock</h4>
- * All read lock related stats are exposed under scope `read_lock`.
- * for detail stats.
- */
-class BKLogReadHandler extends BKLogHandler implements LogSegmentNamesListener {
-    static final Logger LOG = LoggerFactory.getLogger(BKLogReadHandler.class);
-
-    protected final LogMetadataForReader logMetadataForReader;
-
-    protected final DynamicDistributedLogConfiguration dynConf;
-
-    private final Optional<String> subscriberId;
-    private DistributedLock readLock;
-    private Future<Void> lockAcquireFuture;
-
-    // notify the state change about the read handler
-    protected final AsyncNotification readerStateNotification;
-
-    // log segments listener
-    protected boolean logSegmentsNotificationDisabled = false;
-    protected final CopyOnWriteArraySet<LogSegmentListener> listeners =
-            new CopyOnWriteArraySet<LogSegmentListener>();
-    protected Versioned<List<LogSegmentMetadata>> lastNotifiedLogSegments =
-            new Versioned<List<LogSegmentMetadata>>(null, Version.NEW);
-
-    // stats
-    private final StatsLogger perLogStatsLogger;
-
-    /**
-     * Construct a Bookkeeper journal manager.
-     */
-    BKLogReadHandler(LogMetadataForReader logMetadata,
-                     Optional<String> subscriberId,
-                     DistributedLogConfiguration conf,
-                     DynamicDistributedLogConfiguration dynConf,
-                     LogStreamMetadataStore streamMetadataStore,
-                     LogSegmentMetadataCache metadataCache,
-                     LogSegmentEntryStore entryStore,
-                     OrderedScheduler scheduler,
-                     AlertStatsLogger alertStatsLogger,
-                     StatsLogger statsLogger,
-                     StatsLogger perLogStatsLogger,
-                     String clientId,
-                     AsyncNotification readerStateNotification,
-                     boolean isHandleForReading) {
-        super(logMetadata,
-                conf,
-                streamMetadataStore,
-                metadataCache,
-                entryStore,
-                scheduler,
-                statsLogger,
-                alertStatsLogger,
-                clientId);
-        this.logMetadataForReader = logMetadata;
-        this.dynConf = dynConf;
-        this.perLogStatsLogger =
-                isHandleForReading ? perLogStatsLogger : NullStatsLogger.INSTANCE;
-        this.readerStateNotification = readerStateNotification;
-        this.subscriberId = subscriberId;
-    }
-
-    @VisibleForTesting
-    String getReadLockPath() {
-        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() {
-        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() {
-        if (null == lockAcquireFuture) {
-            lockAcquireFuture = streamMetadataStore.createReadLock(logMetadataForReader, subscriberId)
-                    .flatMap(new ExceptionalFunction<DistributedLock, Future<Void>>() {
-                        @Override
-                        public Future<Void> applyE(DistributedLock lock) throws Throwable {
-                            BKLogReadHandler.this.readLock = lock;
-                            LOG.info("acquiring readlock {} at {}", getLockClientId(), getReadLockPath());
-                            return acquireLockOnExecutorThread(lock);
-                        }
-                    });
-        }
-        return lockAcquireFuture;
-    }
-
-    /**
-     * 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();
-
-        // 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;
-            }
-        });
-        acquireFuture.addEventListener(new FutureEventListener<DistributedLock>() {
-            @Override
-            public void onSuccess(DistributedLock lock) {
-                LOG.info("acquired readlock {} at {}", getLockClientId(), getReadLockPath());
-                satisfyPromiseAsync(threadAcquirePromise, new Return<Void>(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));
-            }
-        });
-        return threadAcquirePromise;
-    }
-
-    /**
-     * Check ownership of elective stream lock.
-     */
-    void checkReadLock() throws DLIllegalStateException, LockingException {
-        synchronized (this) {
-            if ((null == lockAcquireFuture) ||
-                (!lockAcquireFuture.isDefined())) {
-                throw new DLIllegalStateException("Attempt to check for lock before it has been acquired successfully");
-            }
-        }
-
-        readLock.checkOwnership();
-    }
-
-    public Future<Void> asyncClose() {
-        DistributedLock lockToClose;
-        synchronized (this) {
-            if (null != lockAcquireFuture && !lockAcquireFuture.isDefined()) {
-                FutureUtils.cancel(lockAcquireFuture);
-            }
-            lockToClose = readLock;
-        }
-        return Utils.closeSequence(scheduler, lockToClose)
-                .flatMap(new AbstractFunction1<Void, Future<Void>>() {
-            @Override
-            public Future<Void> apply(Void result) {
-                // unregister the log segment listener
-                metadataStore.unregisterLogSegmentListener(logMetadata.getLogSegmentsPath(), BKLogReadHandler.this);
-                return Future.Void();
-            }
-        });
-    }
-
-    @Override
-    public Future<Void> asyncAbort() {
-        return asyncClose();
-    }
-
-    /**
-     * Start fetch the log segments and register the {@link LogSegmentNamesListener}.
-     * The future is satisfied only on a successful fetch or encountered a fatal failure.
-     *
-     * @return future represents the fetch result
-     */
-    Future<Versioned<List<LogSegmentMetadata>>> asyncStartFetchLogSegments() {
-        Promise<Versioned<List<LogSegmentMetadata>>> promise =
-                new Promise<Versioned<List<LogSegmentMetadata>>>();
-        asyncStartFetchLogSegments(promise);
-        return promise;
-    }
-
-    void asyncStartFetchLogSegments(final Promise<Versioned<List<LogSegmentMetadata>>> promise) {
-        readLogSegmentsFromStore(
-                LogSegmentMetadata.COMPARATOR,
-                LogSegmentFilter.DEFAULT_FILTER,
-                this).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                if (cause instanceof LogNotFoundException ||
-                        cause instanceof LogSegmentNotFoundException ||
-                        cause instanceof UnexpectedException) {
-                    // indicate some inconsistent behavior, abort
-                    metadataException.compareAndSet(null, (IOException) cause);
-                    // notify the reader that read handler is in error state
-                    notifyReaderOnError(cause);
-                    FutureUtils.setException(promise, cause);
-                    return;
-                }
-                scheduler.schedule(new Runnable() {
-                    @Override
-                    public void run() {
-                        asyncStartFetchLogSegments(promise);
-                    }
-                }, conf.getZKRetryBackoffMaxMillis(), TimeUnit.MILLISECONDS);
-            }
-
-            @Override
-            public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
-                // no-op
-                FutureUtils.setValue(promise, segments);
-            }
-        });
-    }
-
-    @VisibleForTesting
-    void disableReadAheadLogSegmentsNotification() {
-        logSegmentsNotificationDisabled = true;
-    }
-
-    @Override
-    public void onSegmentsUpdated(final Versioned<List<String>> segments) {
-        synchronized (this) {
-            if (lastNotifiedLogSegments.getVersion() != Version.NEW &&
-                    lastNotifiedLogSegments.getVersion().compare(segments.getVersion()) != Version.Occurred.BEFORE) {
-                // the log segments has been read, and it is possibly a retry from last segments update
-                return;
-            }
-        }
-
-        Promise<Versioned<List<LogSegmentMetadata>>> readLogSegmentsPromise =
-                new Promise<Versioned<List<LogSegmentMetadata>>>();
-        readLogSegmentsPromise.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                if (cause instanceof LogNotFoundException ||
-                        cause instanceof LogSegmentNotFoundException ||
-                        cause instanceof UnexpectedException) {
-                    // indicate some inconsistent behavior, abort
-                    metadataException.compareAndSet(null, (IOException) cause);
-                    // notify the reader that read handler is in error state
-                    notifyReaderOnError(cause);
-                    return;
-                }
-                scheduler.schedule(new Runnable() {
-                    @Override
-                    public void run() {
-                        onSegmentsUpdated(segments);
-                    }
-                }, conf.getZKRetryBackoffMaxMillis(), TimeUnit.MILLISECONDS);
-            }
-
-            @Override
-            public void onSuccess(Versioned<List<LogSegmentMetadata>> logSegments) {
-                List<LogSegmentMetadata> segmentsToNotify = null;
-                synchronized (BKLogReadHandler.this) {
-                    Versioned<List<LogSegmentMetadata>> lastLogSegments = lastNotifiedLogSegments;
-                    if (lastLogSegments.getVersion() == Version.NEW ||
-                            lastLogSegments.getVersion().compare(logSegments.getVersion()) == Version.Occurred.BEFORE) {
-                        lastNotifiedLogSegments = logSegments;
-                        segmentsToNotify = logSegments.getValue();
-                    }
-                }
-                if (null != segmentsToNotify) {
-                    notifyUpdatedLogSegments(segmentsToNotify);
-                }
-            }
-        });
-        // log segments list is updated, read their metadata
-        readLogSegmentsFromStore(
-                segments,
-                LogSegmentMetadata.COMPARATOR,
-                LogSegmentFilter.DEFAULT_FILTER,
-                readLogSegmentsPromise);
-    }
-
-    @Override
-    public void onLogStreamDeleted() {
-        notifyLogStreamDeleted();
-    }
-
-    //
-    // Listener for log segments
-    //
-
-    protected void registerListener(@Nullable LogSegmentListener listener) {
-        if (null != listener) {
-            listeners.add(listener);
-        }
-    }
-
-    protected void unregisterListener(@Nullable LogSegmentListener listener) {
-        if (null != listener) {
-            listeners.remove(listener);
-        }
-    }
-
-    protected void notifyUpdatedLogSegments(List<LogSegmentMetadata> segments) {
-        if (logSegmentsNotificationDisabled) {
-            return;
-        }
-
-        for (LogSegmentListener listener : listeners) {
-            List<LogSegmentMetadata> listToReturn =
-                    new ArrayList<LogSegmentMetadata>(segments);
-            Collections.sort(listToReturn, LogSegmentMetadata.COMPARATOR);
-            listener.onSegmentsUpdated(listToReturn);
-        }
-    }
-
-    protected void notifyLogStreamDeleted() {
-        if (logSegmentsNotificationDisabled) {
-            return;
-        }
-
-        for (LogSegmentListener listener : listeners) {
-            listener.onLogStreamDeleted();
-        }
-    }
-
-    // notify the errors
-    protected void notifyReaderOnError(Throwable cause) {
-        if (null != readerStateNotification) {
-            readerStateNotification.notifyOnError(cause);
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogSegmentWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogSegmentWriter.java
deleted file mode 100644
index df64505..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogSegmentWriter.java
+++ /dev/null
@@ -1,1348 +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 com.twitter.distributedlog;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.concurrent.locks.ReentrantLock;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.distributedlog.exceptions.FlushException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.TransactionIdOutOfOrderException;
-import com.twitter.distributedlog.exceptions.WriteCancelledException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.exceptions.InvalidEnvelopedEntryException;
-import com.twitter.distributedlog.feature.CoreFeatureKeys;
-import com.twitter.distributedlog.injector.FailureInjector;
-import com.twitter.distributedlog.injector.RandomDelayFailureInjector;
-import com.twitter.distributedlog.io.Buffer;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.distributedlog.io.CompressionUtils;
-import com.twitter.distributedlog.lock.DistributedLock;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
-import com.twitter.distributedlog.logsegment.LogSegmentWriter;
-import com.twitter.distributedlog.stats.BroadCastStatsLogger;
-import com.twitter.distributedlog.stats.OpStatsListener;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.PermitLimiter;
-import com.twitter.distributedlog.util.SafeQueueingFuturePool;
-import com.twitter.distributedlog.util.SimplePermitLimiter;
-import com.twitter.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.bookkeeper.client.AsyncCallback.AddCallback;
-import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.AlertStatsLogger;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.MathUtils;
-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 com.twitter.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
-
-/**
- * BookKeeper Based Log Segment Writer.
- *
- * Multiple log records are packed into a single bookkeeper
- * entry before sending it over the network. The fact that the log record entries
- * are complete in the bookkeeper entries means that each bookkeeper log entry
- * can be read as a complete edit log. This is useful for reading, as we don't
- * need to read through the entire log segment to get the last written entry.
- *
- * <h3>Metrics</h3>
- *
- * <ul>
- * <li> flush/periodic/{success,miss}: counters for periodic flushes.
- * <li> data/{success,miss}: counters for data transmits.
- * <li> transmit/packetsize: opstats. characteristics of packet size for transmits.
- * <li> control/success: counter of success transmit of control records
- * <li> seg_writer/write: opstats. latency characteristics of write operations in segment writer.
- * <li> seg_writer/add_complete/{callback,queued,deferred}: opstats. latency components of add completions.
- * <li> seg_writer/pendings: counter. the number of records pending by the segment writers.
- * <li> transmit/outstanding/requests: per stream gauge. the number of outstanding transmits each stream.
- * </ul>
- */
-class BKLogSegmentWriter implements LogSegmentWriter, AddCallback, Runnable, Sizable {
-    static final Logger LOG = LoggerFactory.getLogger(BKLogSegmentWriter.class);
-
-    private final String fullyQualifiedLogSegment;
-    private final String streamName;
-    private final int logSegmentMetadataVersion;
-    private BKTransmitPacket packetPrevious;
-    private Entry.Writer recordSetWriter;
-    private final AtomicInteger outstandingTransmits;
-    private final int transmissionThreshold;
-    protected final LogSegmentEntryWriter entryWriter;
-    private final CompressionCodec.Type compressionType;
-    private final ReentrantLock transmitLock = new ReentrantLock();
-    private final AtomicInteger transmitResult
-        = new AtomicInteger(BKException.Code.OK);
-    private final DistributedLock lock;
-    private final boolean isDurableWriteEnabled;
-    private DLSN lastDLSN = DLSN.InvalidDLSN;
-    private final long startTxId;
-    private long lastTxId = DistributedLogConstants.INVALID_TXID;
-    private long lastTxIdAcknowledged = DistributedLogConstants.INVALID_TXID;
-    private long outstandingBytes = 0;
-    private long numFlushesSinceRestart = 0;
-    private long numBytes = 0;
-    private long lastEntryId = Long.MIN_VALUE;
-    private long lastTransmitNanos = Long.MIN_VALUE;
-    private final int periodicKeepAliveMs;
-
-    // Indicates whether there are writes that have been successfully transmitted that would need
-    // a control record to be transmitted to make them visible to the readers by updating the last
-    // add confirmed
-    volatile private boolean controlFlushNeeded = false;
-    private boolean immediateFlushEnabled = false;
-    private int minDelayBetweenImmediateFlushMs = 0;
-    private Stopwatch lastTransmit;
-    private boolean streamEnded = false;
-    private final ScheduledFuture<?> periodicFlushSchedule;
-    private final ScheduledFuture<?> periodicKeepAliveSchedule;
-    final private AtomicReference<ScheduledFuture<?>> transmitSchedFutureRef = new AtomicReference<ScheduledFuture<?>>(null);
-    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 final boolean enableRecordCounts;
-    private int positionWithinLogSegment = 0;
-    private final long logSegmentSequenceNumber;
-    // Used only for values that *could* change (e.g. buffer size etc.)
-    private final DistributedLogConfiguration conf;
-    private final OrderedScheduler scheduler;
-
-    // stats
-    private final StatsLogger envelopeStatsLogger;
-    private final StatsLogger transmitOutstandingLogger;
-    private final Counter transmitDataSuccesses;
-    private final Counter transmitDataMisses;
-    private final Gauge<Number> transmitOutstandingGauge;
-    private final OpStatsLogger transmitDataPacketSize;
-    private final Counter transmitControlSuccesses;
-    private final Counter pFlushSuccesses;
-    private final Counter pFlushMisses;
-    private final OpStatsLogger writeTime;
-    private final OpStatsLogger addCompleteTime;
-    private final OpStatsLogger addCompleteQueuedTime;
-    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 AlertStatsLogger alertStatsLogger;
-    private final WriteLimiter writeLimiter;
-    private final FailureInjector writeDelayInjector;
-
-    /**
-     * Construct an edit log output stream which writes to a ledger.
-     */
-    protected BKLogSegmentWriter(String streamName,
-                                 String logSegmentName,
-                                 DistributedLogConfiguration conf,
-                                 int logSegmentMetadataVersion,
-                                 LogSegmentEntryWriter entryWriter,
-                                 DistributedLock lock, /** the lock needs to be acquired **/
-                                 long startTxId,
-                                 long logSegmentSequenceNumber,
-                                 OrderedScheduler scheduler,
-                                 StatsLogger statsLogger,
-                                 StatsLogger perLogStatsLogger,
-                                 AlertStatsLogger alertStatsLogger,
-                                 PermitLimiter globalWriteLimiter,
-                                 FeatureProvider featureProvider,
-                                 DynamicDistributedLogConfiguration dynConf)
-        throws IOException {
-        super();
-
-        // set up a write limiter
-        PermitLimiter streamWriteLimiter = null;
-        if (conf.getPerWriterOutstandingWriteLimit() < 0) {
-            streamWriteLimiter = PermitLimiter.NULL_PERMIT_LIMITER;
-        } else {
-            Feature disableWriteLimitFeature = featureProvider.getFeature(
-                CoreFeatureKeys.DISABLE_WRITE_LIMIT.name().toLowerCase());
-            streamWriteLimiter = new SimplePermitLimiter(
-                conf.getOutstandingWriteLimitDarkmode(),
-                conf.getPerWriterOutstandingWriteLimit(),
-                statsLogger.scope("streamWriteLimiter"),
-                false,
-                disableWriteLimitFeature);
-        }
-        this.writeLimiter = new WriteLimiter(streamName, streamWriteLimiter, globalWriteLimiter);
-        this.alertStatsLogger = alertStatsLogger;
-        this.envelopeStatsLogger = BroadCastStatsLogger.masterslave(statsLogger, perLogStatsLogger);
-
-        StatsLogger flushStatsLogger = statsLogger.scope("flush");
-        StatsLogger pFlushStatsLogger = flushStatsLogger.scope("periodic");
-        pFlushSuccesses = pFlushStatsLogger.getCounter("success");
-        pFlushMisses = pFlushStatsLogger.getCounter("miss");
-
-        // transmit
-        StatsLogger transmitDataStatsLogger = statsLogger.scope("data");
-        transmitDataSuccesses = transmitDataStatsLogger.getCounter("success");
-        transmitDataMisses = transmitDataStatsLogger.getCounter("miss");
-        StatsLogger transmitStatsLogger = statsLogger.scope("transmit");
-        transmitDataPacketSize =  transmitStatsLogger.getOpStatsLogger("packetsize");
-        StatsLogger transmitControlStatsLogger = statsLogger.scope("control");
-        transmitControlSuccesses = transmitControlStatsLogger.getCounter("success");
-        StatsLogger segWriterStatsLogger = statsLogger.scope("seg_writer");
-        writeTime = segWriterStatsLogger.getOpStatsLogger("write");
-        addCompleteTime = segWriterStatsLogger.scope("add_complete").getOpStatsLogger("callback");
-        addCompleteQueuedTime = segWriterStatsLogger.scope("add_complete").getOpStatsLogger("queued");
-        addCompleteDeferredTime = segWriterStatsLogger.scope("add_complete").getOpStatsLogger("deferred");
-        pendingWrites = segWriterStatsLogger.getCounter("pending");
-
-        // outstanding transmit requests
-        transmitOutstandingLogger = perLogStatsLogger.scope("transmit").scope("outstanding");
-        transmitOutstandingGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-            @Override
-            public Number getSample() {
-                return outstandingTransmits.get();
-            }
-        };
-        transmitOutstandingLogger.registerGauge("requests", transmitOutstandingGauge);
-
-        outstandingTransmits = new AtomicInteger(0);
-        this.fullyQualifiedLogSegment = streamName + ":" + logSegmentName;
-        this.streamName = streamName;
-        this.logSegmentMetadataVersion = logSegmentMetadataVersion;
-        this.entryWriter = entryWriter;
-        this.lock = lock;
-        this.lock.checkOwnershipAndReacquire();
-
-        final int configuredTransmissionThreshold = dynConf.getOutputBufferSize();
-        if (configuredTransmissionThreshold > MAX_LOGRECORDSET_SIZE) {
-            LOG.warn("Setting output buffer size {} greater than max transmission size {} for log segment {}",
-                new Object[] {configuredTransmissionThreshold, MAX_LOGRECORDSET_SIZE, fullyQualifiedLogSegment});
-            this.transmissionThreshold = MAX_LOGRECORDSET_SIZE;
-        } else {
-            this.transmissionThreshold = configuredTransmissionThreshold;
-        }
-        this.compressionType = CompressionUtils.stringToType(conf.getCompressionType());
-
-        this.logSegmentSequenceNumber = logSegmentSequenceNumber;
-        this.recordSetWriter = Entry.newEntry(
-                streamName,
-                Math.max(transmissionThreshold, 1024),
-                envelopeBeforeTransmit(),
-                compressionType,
-                envelopeStatsLogger);
-        this.packetPrevious = null;
-        this.startTxId = startTxId;
-        this.lastTxId = startTxId;
-        this.lastTxIdAcknowledged = startTxId;
-        this.enableRecordCounts = conf.getEnableRecordCounts();
-        this.immediateFlushEnabled = conf.getImmediateFlushEnabled();
-        this.isDurableWriteEnabled = dynConf.isDurableWriteEnabled();
-        this.scheduler = scheduler;
-
-        // Failure injection
-        if (conf.getEIInjectWriteDelay()) {
-            this.writeDelayInjector = new RandomDelayFailureInjector(dynConf);
-        } else {
-            this.writeDelayInjector = FailureInjector.NULL;
-        }
-
-        // If we are transmitting immediately (threshold == 0) and if immediate
-        // flush is enabled, we don't need the periodic flush task
-        final int configuredPeriodicFlushFrequency = dynConf.getPeriodicFlushFrequencyMilliSeconds();
-        if (!immediateFlushEnabled || (0 != this.transmissionThreshold)) {
-            int periodicFlushFrequency = configuredPeriodicFlushFrequency;
-            if (periodicFlushFrequency > 0 && scheduler != null) {
-                periodicFlushSchedule = scheduler.scheduleAtFixedRate(this,
-                        periodicFlushFrequency/2, periodicFlushFrequency/2, TimeUnit.MILLISECONDS);
-            } else {
-                periodicFlushSchedule = null;
-            }
-        } else {
-            // Min delay heuristic applies only when immediate flush is enabled
-            // and transmission threshold is zero
-            minDelayBetweenImmediateFlushMs = conf.getMinDelayBetweenImmediateFlushMs();
-            periodicFlushSchedule = null;
-        }
-        this.periodicKeepAliveMs = conf.getPeriodicKeepAliveMilliSeconds();
-        if (periodicKeepAliveMs > 0 && scheduler != null) {
-            periodicKeepAliveSchedule = scheduler.scheduleAtFixedRate(new Runnable() {
-                @Override
-                public void run() {
-                    keepAlive();
-                }
-            }, periodicKeepAliveMs, periodicKeepAliveMs, TimeUnit.MILLISECONDS);
-        } else {
-            periodicKeepAliveSchedule = null;
-        }
-
-        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();
-    }
-
-    String getFullyQualifiedLogSegment() {
-        return fullyQualifiedLogSegment;
-    }
-
-    @VisibleForTesting
-    DistributedLock getLock() {
-        return this.lock;
-    }
-
-    @VisibleForTesting
-    FuturePool getFuturePool() {
-        if (null == scheduler) {
-            return null;
-        }
-        return scheduler.getFuturePool(streamName);
-    }
-
-    @VisibleForTesting
-    void setTransmitResult(int rc) {
-        transmitResult.set(rc);
-    }
-
-    @VisibleForTesting
-    protected final LogSegmentEntryWriter getEntryWriter() {
-        return this.entryWriter;
-    }
-
-    @Override
-    public long getLogSegmentId() {
-        return this.entryWriter.getLogSegmentId();
-    }
-
-    protected final long getLogSegmentSequenceNumber() {
-        return logSegmentSequenceNumber;
-    }
-
-    /**
-     * Get the start tx id of the log segment.
-     *
-     * @return start tx id of the log segment.
-     */
-    protected final long getStartTxId() {
-        return startTxId;
-    }
-
-    /**
-     * Get the last tx id that has been written to the log segment buffer but not committed yet.
-     *
-     * @return last tx id that has been written to the log segment buffer but not committed yet.
-     * @see #getLastTxIdAcknowledged()
-     */
-    synchronized long getLastTxId() {
-        return lastTxId;
-    }
-
-    /**
-     * Get the last tx id that has been acknowledged.
-     *
-     * @return last tx id that has been acknowledged.
-     * @see #getLastTxId()
-     */
-    synchronized long getLastTxIdAcknowledged() {
-        return lastTxIdAcknowledged;
-    }
-
-    /**
-     * Get the position-within-logsemgnet of the last written log record.
-     *
-     * @return position-within-logsegment of the last written log record.
-     */
-    synchronized int getPositionWithinLogSegment() {
-        return positionWithinLogSegment;
-    }
-
-    @VisibleForTesting
-    long getLastEntryId() {
-        return lastEntryId;
-    }
-
-    /**
-     * Get the last dlsn of the last acknowledged record.
-     *
-     * @return last dlsn of the last acknowledged record.
-     */
-    synchronized DLSN getLastDLSN() {
-        return lastDLSN;
-    }
-
-    @Override
-    public long size() {
-        return entryWriter.size();
-    }
-
-    private synchronized int getAverageTransmitSize() {
-        if (numFlushesSinceRestart > 0) {
-            long ret = numBytes/numFlushesSinceRestart;
-
-            if (ret < Integer.MIN_VALUE || ret > Integer.MAX_VALUE) {
-                throw new IllegalArgumentException
-                    (ret + " transmit size should never exceed max transmit size");
-            }
-            return (int) ret;
-        }
-
-        return 0;
-    }
-
-    private Entry.Writer newRecordSetWriter() {
-        return Entry.newEntry(
-                streamName,
-                Math.max(transmissionThreshold, getAverageTransmitSize()),
-                envelopeBeforeTransmit(),
-                compressionType,
-                envelopeStatsLogger);
-    }
-
-    private boolean envelopeBeforeTransmit() {
-        return LogSegmentMetadata.supportsEnvelopedEntries(logSegmentMetadataVersion);
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        return closeInternal(false);
-    }
-
-    @Override
-    public Future<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) {
-            EntryBuffer recordSet = packet.getRecordSet();
-            numRecords = recordSet.getNumRecords();
-            int rc = transmitResult.get();
-            if (BKException.Code.OK == rc) {
-                rc = BKException.Code.InterruptedException;
-            }
-            Throwable reason = new WriteCancelledException(streamName, FutureUtils.transmitException(rc));
-            recordSet.abortTransmit(reason);
-        }
-        LOG.info("Stream {} aborted {} writes", fullyQualifiedLogSegment, numRecords);
-    }
-
-    private synchronized long getWritesPendingTransmit() {
-        if (null != recordSetWriter) {
-            return recordSetWriter.getNumRecords();
-        } else {
-            return 0;
-        }
-    }
-
-    private synchronized long getPendingAddCompleteCount() {
-        if (null != addCompleteFuturePool) {
-            return addCompleteFuturePool.size();
-        } else {
-            return 0;
-        }
-    }
-
-    private Future<Void> closeInternal(boolean abort) {
-        Promise<Void> closePromise;
-        synchronized (this) {
-            if (null != closeFuture) {
-                return closeFuture;
-            }
-            closePromise = closeFuture = new Promise<Void>();
-        }
-
-        AtomicReference<Throwable> throwExc = new AtomicReference<Throwable>(null);
-        closeInternal(abort, throwExc, closePromise);
-        return closePromise;
-    }
-
-    private void closeInternal(final boolean abort,
-                               final AtomicReference<Throwable> throwExc,
-                               final Promise<Void> closePromise) {
-        // clean stats resources
-        this.transmitOutstandingLogger.unregisterGauge("requests", transmitOutstandingGauge);
-        this.writeLimiter.close();
-
-        // Cancel the periodic keep alive schedule first
-        if (null != periodicKeepAliveSchedule) {
-            if (!periodicKeepAliveSchedule.cancel(false)) {
-                LOG.info("Periodic keepalive for log segment {} isn't cancelled.", getFullyQualifiedLogSegment());
-            }
-        }
-
-        // Cancel the periodic flush schedule first
-        // The task is allowed to exit gracefully
-        if (null != periodicFlushSchedule) {
-            // we don't need to care about the cancel result here. if the periodicl flush task couldn't
-            // be cancelled, it means that it is doing flushing. So following flushes would be synchronized
-            // to wait until background flush completed.
-            if (!periodicFlushSchedule.cancel(false)) {
-                LOG.info("Periodic flush for log segment {} isn't cancelled.", getFullyQualifiedLogSegment());
-            }
-        }
-
-        // If it is a normal close and the stream isn't in an error state, we attempt to flush any buffered data
-        if (!abort && !isLogSegmentInError()) {
-            this.enforceLock = false;
-            LOG.info("Flushing before closing log segment {}", getFullyQualifiedLogSegment());
-            flushAndCommit().addEventListener(new FutureEventListener<Long>() {
-                @Override
-                public void onSuccess(Long value) {
-                    abortTransmitPacketOnClose(abort, throwExc, closePromise);
-                }
-
-                @Override
-                public void onFailure(Throwable cause) {
-                    throwExc.set(cause);
-                    abortTransmitPacketOnClose(abort, throwExc, closePromise);
-                }
-            });
-        } else {
-            abortTransmitPacketOnClose(abort, throwExc, closePromise);
-        }
-
-    }
-
-    private void abortTransmitPacketOnClose(final boolean abort,
-                                            final AtomicReference<Throwable> throwExc,
-                                            final Promise<Void> closePromise) {
-        LOG.info("Closing BKPerStreamLogWriter (abort={}) for {} :" +
-                        " lastDLSN = {} outstandingTransmits = {} writesPendingTransmit = {} addCompletesPending = {}",
-                new Object[]{abort, fullyQualifiedLogSegment, getLastDLSN(),
-                        outstandingTransmits.get(), getWritesPendingTransmit(), getPendingAddCompleteCount()});
-
-        // Save the current packet to reset, leave a new empty packet to avoid a race with
-        // addCompleteDeferredProcessing.
-        final BKTransmitPacket packetPreviousSaved;
-        final BKTransmitPacket packetCurrentSaved;
-        synchronized (this) {
-            packetPreviousSaved = packetPrevious;
-            packetCurrentSaved = new BKTransmitPacket(recordSetWriter);
-            recordSetWriter = newRecordSetWriter();
-        }
-
-        // Once the last packet been transmitted, apply any remaining promises asynchronously
-        // to avoid blocking close if bk client is slow for some reason.
-        if (null != packetPreviousSaved) {
-            packetPreviousSaved.addTransmitCompleteListener(new FutureEventListener<Integer>() {
-                @Override
-                public void onSuccess(Integer transmitResult) {
-                    flushAddCompletes();
-                    abortPacket(packetCurrentSaved);
-                }
-                @Override
-                public void onFailure(Throwable cause) {
-                    LOG.error("Unexpected error on transmit completion ", cause);
-                }
-            });
-        } else {
-            // In this case there are no pending add completes, but we still need to abort the
-            // current packet.
-            abortPacket(packetCurrentSaved);
-        }
-        closeLedgerOnClose(abort, throwExc, closePromise);
-    }
-
-    private void closeLedgerOnClose(final boolean abort,
-                                    final AtomicReference<Throwable> throwExc,
-                                    final Promise<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.
-            // we should throw the exception to #closeToFinalize, so it would fail completing a log segment.
-            entryWriter.asyncClose(new CloseCallback() {
-                @Override
-                public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
-                    if (BKException.Code.OK != rc && BKException.Code.LedgerClosedException != rc) {
-                        if (!abort) {
-                            throwExc.set(new IOException("Failed to close ledger for " + fullyQualifiedLogSegment + " : " +
-                                    BKException.getMessage(rc)));
-                        }
-                    }
-                    completeClosePromise(abort, throwExc, closePromise);
-                }
-            }, null);
-        } else {
-            completeClosePromise(abort, throwExc, closePromise);
-        }
-    }
-
-    private void completeClosePromise(final boolean abort,
-                                      final AtomicReference<Throwable> throwExc,
-                                      final Promise<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);
-        } else {
-            FutureUtils.setException(closePromise, throwExc.get());
-        }
-    }
-
-    @Override
-    synchronized public void write(LogRecord record) throws IOException {
-        writeUserRecord(record);
-        flushIfNeeded();
-    }
-
-    @Override
-    synchronized public Future<DLSN> asyncWrite(LogRecord record) {
-        return asyncWrite(record, true);
-    }
-
-    synchronized public Future<DLSN> asyncWrite(LogRecord record, boolean flush) {
-        Future<DLSN> result = null;
-        try {
-            if (record.isControl()) {
-                // we don't pack control records with user records together
-                // so transmit current output buffer if possible
-                try {
-                    transmit();
-                } catch (IOException ioe) {
-                    return Future.exception(new WriteCancelledException(fullyQualifiedLogSegment, ioe));
-                }
-                result = writeControlLogRecord(record);
-                transmit();
-            } else {
-                result = writeUserRecord(record);
-                if (!isDurableWriteEnabled) {
-                    // we have no idea about the DLSN if durability is turned off.
-                    result = Future.value(DLSN.InvalidDLSN);
-                }
-                if (flush) {
-                    flushIfNeeded();
-                }
-            }
-        } catch (IOException ioe) {
-            // We may incorrectly report transmit failure here, but only if we happened to hit
-            // packet/xmit size limit conditions AND fail flush above, which should happen rarely
-            if (null != result) {
-                LOG.error("Overriding first result with flush failure {}", result);
-            }
-            result = Future.exception(ioe);
-
-            // Flush to ensure any prev. writes with flush=false are flushed despite failure.
-            flushIfNeededNoThrow();
-        }
-        return result;
-    }
-
-    synchronized private Future<DLSN> writeUserRecord(LogRecord record) throws IOException {
-        if (null != closeFuture) {
-            throw new WriteException(fullyQualifiedLogSegment, BKException.getMessage(BKException.Code.LedgerClosedException));
-        }
-
-        if (BKException.Code.OK != transmitResult.get()) {
-            // Failfast if the stream already encountered error with safe retry on the client
-            throw new WriteException(fullyQualifiedLogSegment, BKException.getMessage(transmitResult.get()));
-        }
-
-        if (streamEnded) {
-            throw new EndOfStreamException("Writing to a stream after it has been marked as completed");
-        }
-
-        if ((record.getTransactionId() < 0) ||
-            (record.getTransactionId() == DistributedLogConstants.MAX_TXID)) {
-            throw new TransactionIdOutOfOrderException(record.getTransactionId());
-        }
-
-        // Inject write delay if configured to do so
-        writeDelayInjector.inject();
-
-        // Will check write rate limits and throw if exceeded.
-        writeLimiter.acquire();
-        pendingWrites.inc();
-
-        // The count represents the number of user records up to the
-        // current record
-        // Increment the record count only when writing a user log record
-        // 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;
-        try {
-            // increment the position for the record to write
-            // if the record is failed to write, it would be decremented.
-            positionWithinLogSegment++;
-            int numRecords = 1;
-            if (record.isRecordSet()) {
-                numRecords = LogRecordSet.numRecords(record);
-            }
-            future = writeInternal(record);
-            // after the record (record set) is written, the position should be
-            // moved for {numRecords}, but since we already moved the record by 1
-            // so advance the position for other {numRecords - 1}.
-            positionWithinLogSegment += (numRecords - 1);
-        } catch (IOException ex) {
-            writeLimiter.release();
-            pendingWrites.dec();
-            positionWithinLogSegment--;
-            throw ex;
-        }
-
-        // Track outstanding requests and return the future.
-        return future.ensure(new Function0<BoxedUnit>() {
-            public BoxedUnit apply() {
-                pendingWrites.dec();
-                writeLimiter.release();
-                return null;
-            }
-        });
-    }
-
-    boolean isLogSegmentInError() {
-        return (transmitResult.get() != BKException.Code.OK);
-    }
-
-    boolean shouldFailCompleteLogSegment() {
-        return (transmitResult.get() != BKException.Code.OK) &&
-                (transmitResult.get() != BKException.Code.LedgerClosedException);
-    }
-
-    synchronized public Future<DLSN> writeInternal(LogRecord record)
-            throws LogRecordTooLongException, LockingException, BKTransmitException,
-                   WriteException, InvalidEnvelopedEntryException {
-        int logRecordSize = record.getPersistentSize();
-
-        if (logRecordSize > MAX_LOGRECORD_SIZE) {
-            throw new LogRecordTooLongException(String.format(
-                    "Log Record of size %d written when only %d is allowed",
-                    logRecordSize, MAX_LOGRECORD_SIZE));
-        }
-
-        // If we will exceed the max number of bytes allowed per entry
-        // initiate a transmit before accepting the new log record
-        if ((recordSetWriter.getNumBytes() + logRecordSize) > MAX_LOGRECORDSET_SIZE) {
-            checkStateAndTransmit();
-        }
-
-        checkWriteLock();
-
-        if (enableRecordCounts) {
-            // Set the count here. The caller would appropriately increment it
-            // if this log record is to be counted
-            record.setPositionWithinLogSegment(positionWithinLogSegment);
-        }
-
-        Promise<DLSN> writePromise = new Promise<DLSN>();
-        writePromise.addEventListener(new OpStatsListener<DLSN>(writeTime));
-        recordSetWriter.writeRecord(record, writePromise);
-
-        if (record.getTransactionId() < lastTxId) {
-            LOG.info("Log Segment {} TxId decreased Last: {} Record: {}",
-                    new Object[] {fullyQualifiedLogSegment, lastTxId, record.getTransactionId()});
-        }
-        if (!record.isControl()) {
-            // only update last tx id for user records
-            lastTxId = record.getTransactionId();
-            outstandingBytes += (20 + record.getPayload().length);
-        }
-        return writePromise;
-    }
-
-    synchronized private Future<DLSN> writeControlLogRecord()
-            throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
-                   LockingException, LogRecordTooLongException {
-        LogRecord controlRec = new LogRecord(lastTxId, DistributedLogConstants.CONTROL_RECORD_CONTENT);
-        controlRec.setControl();
-        return writeControlLogRecord(controlRec);
-    }
-
-    synchronized private Future<DLSN> writeControlLogRecord(LogRecord record)
-            throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
-                   LockingException, LogRecordTooLongException {
-        return writeInternal(record);
-    }
-
-    /**
-     * We write a special log record that marks the end of the stream. Since this is the last
-     * log record in the stream, it is marked with MAX_TXID. MAX_TXID also has the useful
-     * side-effect of disallowing future startLogSegment calls through the MaxTxID check
-     *
-     * @throws IOException
-     */
-    synchronized private void writeEndOfStreamMarker() throws IOException {
-        LogRecord endOfStreamRec = new LogRecord(DistributedLogConstants.MAX_TXID, "endOfStream".getBytes(UTF_8));
-        endOfStreamRec.setEndOfStream();
-        writeInternal(endOfStreamRec);
-    }
-
-    /**
-     * Flushes all the data up to this point,
-     * adds the end of stream marker and marks the stream
-     * as read-only in the metadata. No appends to the
-     * stream will be allowed after this point
-     */
-    public Future<Long> markEndOfStream() {
-        synchronized (this) {
-            try {
-                writeEndOfStreamMarker();
-            } catch (IOException e) {
-                return Future.exception(e);
-            }
-            streamEnded = true;
-        }
-        return flushAndCommit();
-    }
-
-    /**
-     * Write bulk of records.
-     *
-     * (TODO: moved this method to log writer level)
-     *
-     * @param records list of records to write
-     * @return number of records that has been written
-     * @throws IOException when there is I/O errors during writing records.
-     */
-    synchronized public int writeBulk(List<LogRecord> records) throws IOException {
-        int numRecords = 0;
-        for (LogRecord r : records) {
-            write(r);
-            numRecords++;
-        }
-        return numRecords;
-    }
-
-    private void checkStateBeforeTransmit() throws WriteException {
-        try {
-            FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_TransmitBeforeAddEntry);
-        } catch (IOException e) {
-            throw new WriteException(streamName, "Fail transmit before adding entries");
-        }
-    }
-
-    /**
-     * Transmit the output buffer data to the backend.
-     *
-     * @return last txn id that already acknowledged
-     * @throws BKTransmitException if the segment writer is already in error state
-     * @throws LockingException if the segment writer lost lock before transmit
-     * @throws WriteException if failed to create the envelope for the data to transmit
-     * @throws InvalidEnvelopedEntryException when built an invalid enveloped entry
-     */
-    synchronized void checkStateAndTransmit()
-            throws BKTransmitException, WriteException, InvalidEnvelopedEntryException, LockingException {
-        checkStateBeforeTransmit();
-        transmit();
-    }
-
-    @Override
-    public synchronized Future<Long> flush() {
-        try {
-            checkStateBeforeTransmit();
-        } catch (WriteException e) {
-            return Future.exception(e);
-        }
-
-        Future<Integer> transmitFuture;
-        try {
-            transmitFuture = transmit();
-        } catch (BKTransmitException e) {
-            return Future.exception(e);
-        } catch (LockingException e) {
-            return Future.exception(e);
-        } catch (WriteException e) {
-            return Future.exception(e);
-        } catch (InvalidEnvelopedEntryException e) {
-            return Future.exception(e);
-        }
-
-        if (null == transmitFuture) {
-            if (null != packetPrevious) {
-                transmitFuture = packetPrevious.getTransmitFuture();
-            }  else {
-                return Future.value(getLastTxIdAcknowledged());
-            }
-        }
-
-        return transmitFuture.flatMap(GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC);
-    }
-
-    @Override
-    public synchronized Future<Long> commit() {
-        // we don't pack control records with user records together
-        // so transmit current output buffer if possible
-        Future<Integer> transmitFuture;
-        try {
-            try {
-                transmitFuture = transmit();
-            } catch (IOException ioe) {
-                return Future.exception(ioe);
-            }
-            if (null == transmitFuture) {
-                writeControlLogRecord();
-                return flush();
-            }
-        } catch (IOException ioe) {
-            return Future.exception(ioe);
-        }
-        return transmitFuture.flatMap(GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC);
-    }
-
-    Future<Long> flushAndCommit() {
-        return flush().flatMap(COMMIT_AFTER_FLUSH_FUNC);
-    }
-
-    void flushIfNeededNoThrow() {
-        try {
-            flushIfNeeded();
-        } catch (IOException ioe) {
-            LOG.error("Encountered exception while flushing log records to stream {}",
-                fullyQualifiedLogSegment, ioe);
-        }
-    }
-
-    void scheduleFlushWithDelayIfNeeded(final Callable<?> callable,
-                                        final AtomicReference<ScheduledFuture<?>> scheduledFutureRef) {
-        final long delayMs = Math.max(0, minDelayBetweenImmediateFlushMs - lastTransmit.elapsed(TimeUnit.MILLISECONDS));
-        final ScheduledFuture<?> scheduledFuture = scheduledFutureRef.get();
-        if ((null == scheduledFuture) || scheduledFuture.isDone()) {
-            scheduledFutureRef.set(scheduler.schedule(new Runnable() {
-                @Override
-                public void run() {
-                    synchronized(this) {
-                        scheduledFutureRef.set(null);
-                        try {
-                            callable.call();
-
-                            // Flush was successful or wasn't needed, the exception should be unset.
-                            scheduledFlushException.set(null);
-                        } catch (Exception exc) {
-                            scheduledFlushException.set(exc);
-                            LOG.error("Delayed flush failed", exc);
-                        }
-                    }
-                }
-            }, delayMs, TimeUnit.MILLISECONDS));
-        }
-    }
-
-    // Based on transmit buffer size, immediate flush, etc., should we flush the current
-    // packet now.
-    void flushIfNeeded() throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
-            LockingException, FlushException {
-        if (outstandingBytes > transmissionThreshold) {
-            // If flush delay is disabled, flush immediately, else schedule appropriately.
-            if (0 == minDelayBetweenImmediateFlushMs) {
-                checkStateAndTransmit();
-            } else {
-                scheduleFlushWithDelayIfNeeded(new Callable<Void>() {
-                    @Override
-                    public Void call() throws Exception {
-                        checkStateAndTransmit();
-                        return null;
-                    }
-                }, transmitSchedFutureRef);
-
-                // Timing here is not very important--the last flush failed and we should
-                // indicate this to the caller. The next flush may succeed and unset the
-                // scheduledFlushException in which case the next write will succeed (if the caller
-                // hasn't already closed the writer).
-                if (scheduledFlushException.get() != null) {
-                    throw new FlushException("Last flush encountered an error while writing data to the backend",
-                        getLastTxId(), getLastTxIdAcknowledged(), scheduledFlushException.get());
-                }
-            }
-        }
-    }
-
-    private void checkWriteLock() throws LockingException {
-        try {
-            if (FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_WriteInternalLostLock)) {
-                throw new LockingException("/failpoint/lockpath", "failpoint is simulating a lost lock"
-                        + getFullyQualifiedLogSegment());
-            }
-        } catch (IOException e) {
-            throw new LockingException("/failpoint/lockpath", "failpoint is simulating a lost lock for "
-                    + getFullyQualifiedLogSegment());
-        }
-        if (enforceLock) {
-            lock.checkOwnershipAndReacquire();
-        }
-    }
-
-    /**
-     * Transmit the current buffer to bookkeeper.
-     * Synchronised at the class. #write() and #setReadyToFlush()
-     * are never called at the same time.
-     *
-     * NOTE: This method should only throw known exceptions so that we don't accidentally
-     *       add new code that throws in an inappropriate place.
-     *
-     * @return a transmit future for caller to wait for transmit result if we transmit successfully,
-     *         null if no data to transmit
-     * @throws BKTransmitException if the segment writer is already in error state
-     * @throws LockingException if the segment writer lost lock before transmit
-     * @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()
-        throws BKTransmitException, LockingException, WriteException, InvalidEnvelopedEntryException {
-        EntryBuffer recordSetToTransmit;
-        transmitLock.lock();
-        try {
-            synchronized (this) {
-                checkWriteLock();
-                // If transmitResult is anything other than BKException.Code.OK, it means that the
-                // stream has encountered an error and cannot be written to.
-                if (!transmitResult.compareAndSet(BKException.Code.OK,
-                                                  BKException.Code.OK)) {
-                    LOG.error("Log Segment {} Trying to write to an errored stream; Error is {}",
-                              fullyQualifiedLogSegment,
-                              BKException.getMessage(transmitResult.get()));
-                    throw new BKTransmitException("Trying to write to an errored stream;"
-                                                          + " Error code : (" + transmitResult.get()
-                                                          + ") " + BKException.getMessage(transmitResult.get()), transmitResult.get());
-                }
-
-                if (recordSetWriter.getNumRecords() == 0) {
-                    // Control flushes always have at least the control record to flush
-                    transmitDataMisses.inc();
-                    return null;
-                }
-
-                recordSetToTransmit = recordSetWriter;
-                recordSetWriter = newRecordSetWriter();
-                outstandingBytes = 0;
-
-                if (recordSetToTransmit.hasUserRecords()) {
-                    numBytes += recordSetToTransmit.getNumBytes();
-                    numFlushesSinceRestart++;
-                }
-            }
-
-            Buffer toSend;
-            try {
-                toSend = recordSetToTransmit.getBuffer();
-                FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_TransmitFailGetBuffer);
-            } catch (IOException e) {
-                if (e instanceof InvalidEnvelopedEntryException) {
-                    alertStatsLogger.raise("Invalid enveloped entry for segment {} : ", fullyQualifiedLogSegment, e);
-                }
-                LOG.error("Exception while enveloping entries for segment: {}",
-                          new Object[] {fullyQualifiedLogSegment}, e);
-                // If a write fails here, we need to set the transmit result to an error so that
-                // no future writes go through and violate ordering guarantees.
-                transmitResult.set(BKException.Code.WriteException);
-                if (e instanceof InvalidEnvelopedEntryException) {
-                    alertStatsLogger.raise("Invalid enveloped entry for segment {} : ", fullyQualifiedLogSegment, e);
-                    throw (InvalidEnvelopedEntryException) e;
-                } else {
-                    throw new WriteException(streamName, "Envelope Error");
-                }
-            }
-
-            synchronized (this) {
-                // update the transmit timestamp
-                lastTransmitNanos = MathUtils.nowInNano();
-
-                BKTransmitPacket packet = new BKTransmitPacket(recordSetToTransmit);
-                packetPrevious = packet;
-                entryWriter.asyncAddEntry(toSend.getData(), 0, toSend.size(),
-                                          this, packet);
-
-                if (recordSetToTransmit.hasUserRecords()) {
-                    transmitDataSuccesses.inc();
-                } else {
-                    transmitControlSuccesses.inc();
-                }
-
-                lastTransmit.reset().start();
-                outstandingTransmits.incrementAndGet();
-                controlFlushNeeded = false;
-                return packet.getTransmitFuture();
-            }
-        } finally {
-            transmitLock.unlock();
-        }
-    }
-
-    /**
-     *  Checks if there is any data to transmit so that the periodic flush
-     *  task can determine if there is anything it needs to do
-     */
-    synchronized private boolean haveDataToTransmit() {
-        if (!transmitResult.compareAndSet(BKException.Code.OK, BKException.Code.OK)) {
-            // Even if there is data it cannot be transmitted, so effectively nothing to send
-            return false;
-        }
-
-        return (recordSetWriter.getNumRecords() > 0);
-    }
-
-    @Override
-    public void addComplete(final int rc, LedgerHandle handle,
-                            final long entryId, final Object ctx) {
-        final AtomicReference<Integer> effectiveRC = new AtomicReference<Integer>(rc);
-        try {
-            if (FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_TransmitComplete)) {
-                effectiveRC.set(BKException.Code.UnexpectedConditionException);
-            }
-        } catch (Exception exc) {
-            effectiveRC.set(BKException.Code.UnexpectedConditionException);
-        }
-
-        // Sanity check to make sure we're receiving these callbacks in order.
-        if (entryId > -1 && lastEntryId >= entryId) {
-            LOG.error("Log segment {} saw out of order entry {} lastEntryId {}",
-                new Object[] {fullyQualifiedLogSegment, entryId, lastEntryId});
-        }
-        lastEntryId = entryId;
-
-        assert (ctx instanceof BKTransmitPacket);
-        final BKTransmitPacket transmitPacket = (BKTransmitPacket) ctx;
-
-        // Time from transmit until receipt of addComplete callback
-        addCompleteTime.registerSuccessfulEvent(TimeUnit.MICROSECONDS.convert(
-            System.nanoTime() - transmitPacket.getTransmitTime(), TimeUnit.NANOSECONDS));
-
-        if (BKException.Code.OK == rc) {
-            EntryBuffer recordSet = transmitPacket.getRecordSet();
-            if (recordSet.hasUserRecords()) {
-                synchronized (this) {
-                    lastTxIdAcknowledged = Math.max(lastTxIdAcknowledged, recordSet.getMaxTxId());
-                }
-            }
-        }
-
-        if (null != addCompleteFuturePool) {
-            final Stopwatch queuedTime = Stopwatch.createStarted();
-            addCompleteFuturePool.apply(new Function0<Void>() {
-                public Void apply() {
-                    final Stopwatch deferredTime = Stopwatch.createStarted();
-                    addCompleteQueuedTime.registerSuccessfulEvent(queuedTime.elapsed(TimeUnit.MICROSECONDS));
-                    addCompleteDeferredProcessing(transmitPacket, entryId, effectiveRC.get());
-                    addCompleteDeferredTime.registerSuccessfulEvent(deferredTime.elapsed(TimeUnit.MICROSECONDS));
-                    return null;
-                }
-                @Override
-                public String toString() {
-                    return String.format("AddComplete(Stream=%s, entryId=%d, rc=%d)",
-                            fullyQualifiedLogSegment, entryId, rc);
-                }
-            }).addEventListener(new FutureEventListener<Void>() {
-                @Override
-                public void onSuccess(Void done) {
-                }
-                @Override
-                public void onFailure(Throwable cause) {
-                    LOG.error("addComplete processing failed for {} entry {} lastTxId {} rc {} with error",
-                        new Object[] {fullyQualifiedLogSegment, entryId, transmitPacket.getRecordSet().getMaxTxId(), rc, cause});
-                }
-            });
-            // Race condition if we notify before the addComplete is enqueued.
-            transmitPacket.notifyTransmitComplete(effectiveRC.get());
-            outstandingTransmits.getAndDecrement();
-        } else {
-            // Notify transmit complete must be called before deferred processing in the
-            // sync case since otherwise callbacks in deferred processing may deadlock.
-            transmitPacket.notifyTransmitComplete(effectiveRC.get());
-            outstandingTransmits.getAndDecrement();
-            addCompleteDeferredProcessing(transmitPacket, entryId, effectiveRC.get());
-        }
-    }
-
-    private void addCompleteDeferredProcessing(final BKTransmitPacket transmitPacket,
-                                               final long entryId,
-                                               final int rc) {
-        boolean cancelPendingPromises = false;
-        EntryBuffer recordSet = transmitPacket.getRecordSet();
-        synchronized (this) {
-            if (transmitResult.compareAndSet(BKException.Code.OK, rc)) {
-                // If this is the first time we are setting an error code in the transmitResult then
-                // we must cancel pending promises; once this error has been set, more records will not
-                // be enqueued; they will be failed with WriteException
-                cancelPendingPromises = (BKException.Code.OK != rc);
-            } else {
-                LOG.warn("Log segment {} entryId {}: Tried to set transmit result to ({}) but is already ({})",
-                    new Object[] {fullyQualifiedLogSegment, entryId, rc, transmitResult.get()});
-            }
-
-            if (transmitResult.get() != BKException.Code.OK) {
-                if (recordSet.hasUserRecords()) {
-                    transmitDataPacketSize.registerFailedEvent(recordSet.getNumBytes());
-                }
-            } else {
-                // If we had data that we flushed then we need it to make sure that
-                // background flush in the next pass will make the previous writes
-                // visible by advancing the lastAck
-                if (recordSet.hasUserRecords()) {
-                    transmitDataPacketSize.registerSuccessfulEvent(recordSet.getNumBytes());
-                    controlFlushNeeded = true;
-                    if (immediateFlushEnabled) {
-                        if (0 == minDelayBetweenImmediateFlushMs) {
-                            backgroundFlush(true);
-                        } else {
-                            scheduleFlushWithDelayIfNeeded(new Callable<Void>() {
-                                @Override
-                                public Void call() throws Exception {
-                                    backgroundFlush(true);
-                                    return null;
-                                }
-                            }, immFlushSchedFutureRef);
-                        }
-                    }
-                }
-            }
-
-            // update last dlsn before satisifying future
-            if (BKException.Code.OK == transmitResult.get()) {
-                DLSN lastDLSNInPacket = recordSet.finalizeTransmit(
-                        logSegmentSequenceNumber, entryId);
-                if (recordSet.hasUserRecords()) {
-                    if (null != lastDLSNInPacket && lastDLSN.compareTo(lastDLSNInPacket) < 0) {
-                        lastDLSN = lastDLSNInPacket;
-                    }
-                }
-            }
-        }
-
-        if (BKException.Code.OK == transmitResult.get()) {
-            recordSet.completeTransmit(logSegmentSequenceNumber, entryId);
-        } else {
-            recordSet.abortTransmit(FutureUtils.transmitException(transmitResult.get()));
-        }
-
-        if (cancelPendingPromises) {
-            // Since the writer is in a bad state no more packets will be tramsitted, and its safe to
-            // assign a new empty packet. This is to avoid a race with closeInternal which may also
-            // try to cancel the current packet;
-            final BKTransmitPacket packetCurrentSaved;
-            synchronized (this) {
-                packetCurrentSaved = new BKTransmitPacket(recordSetWriter);
-                recordSetWriter = newRecordSetWriter();
-            }
-            packetCurrentSaved.getRecordSet().abortTransmit(
-                    new WriteCancelledException(streamName,
-                            FutureUtils.transmitException(transmitResult.get())));
-        }
-    }
-
-    @Override
-    synchronized public void run()  {
-        backgroundFlush(false);
-    }
-
-    synchronized private void backgroundFlush(boolean controlFlushOnly)  {
-        if (null != closeFuture) {
-            // if the log segment is closing, skip any background flushing
-            LOG.debug("Skip background flushing since log segment {} is closing.", getFullyQualifiedLogSegment());
-            return;
-        }
-        try {
-            boolean newData = haveDataToTransmit();
-
-            if (controlFlushNeeded || (!controlFlushOnly && newData)) {
-                // If we need this periodic transmit to persist previously written data but
-                // there is no new data (which would cause the transmit to be skipped) generate
-                // a control record
-                if (!newData) {
-                    writeControlLogRecord();
-                }
-
-                transmit();
-                pFlushSuccesses.inc();
-            } else {
-                pFlushMisses.inc();
-            }
-        } catch (IOException exc) {
-            LOG.error("Log Segment {}: Error encountered by the periodic flush", fullyQualifiedLogSegment, exc);
-        }
-    }
-
-    synchronized private void keepAlive() {
-        if (null != closeFuture) {
-            // if the log segment is closing, skip sending any keep alive records.
-            LOG.debug("Skip sending keepAlive control record since log segment {} is closing.",
-                    getFullyQualifiedLogSegment());
-            return;
-        }
-
-        if (MathUtils.elapsedMSec(lastTransmitNanos) < periodicKeepAliveMs) {
-            return;
-        }
-
-        LogRecord controlRec = new LogRecord(lastTxId, DistributedLogConstants.KEEPALIVE_RECORD_CONTENT);
-        controlRec.setControl();
-        asyncWrite(controlRec);
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java
deleted file mode 100644
index 2486297..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java
+++ /dev/null
@@ -1,1325 +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 com.twitter.distributedlog;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.LogSegmentNotFoundException;
-import com.twitter.distributedlog.exceptions.TransactionIdOutOfOrderException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.function.GetLastTxIdFunction;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.lock.DistributedLock;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.logsegment.RollingPolicy;
-import com.twitter.distributedlog.logsegment.SizeBasedRollingPolicy;
-import com.twitter.distributedlog.logsegment.TimeBasedRollingPolicy;
-import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
-import com.twitter.distributedlog.metadata.MetadataUpdater;
-import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import com.twitter.distributedlog.util.Allocator;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.FutureUtils.FutureEventListenerRunnable;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.distributedlog.util.PermitLimiter;
-import com.twitter.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 com.twitter.distributedlog.impl.ZKLogSegmentFilters.WRITE_HANDLE_FILTER;
-
-/**
- * Log Handler for Writers.
- *
- * <h3>Metrics</h3>
- * All the metrics about log write handler are exposed under scope `segments`.
- * <ul>
- * <li> `segments`/open : opstats. latency characteristics on starting a new log segment.
- * <li> `segments`/close : opstats. latency characteristics on completing an inprogress log segment.
- * <li> `segments`/recover : opstats. latency characteristics on recovering a log segment.
- * <li> `segments`/delete : opstats. latency characteristics on deleting a log segment.
- * </ul>
- */
-class BKLogWriteHandler extends BKLogHandler {
-    static final Logger LOG = LoggerFactory.getLogger(BKLogReadHandler.class);
-
-    private static Transaction.OpListener<LogSegmentEntryWriter> NULL_OP_LISTENER =
-            new Transaction.OpListener<LogSegmentEntryWriter>() {
-        @Override
-        public void onCommit(LogSegmentEntryWriter r) {
-            // no-op
-        }
-
-        @Override
-        public void onAbort(Throwable t) {
-            // no-op
-        }
-    };
-
-    protected final LogMetadataForWriter logMetadataForWriter;
-    protected final Allocator<LogSegmentEntryWriter, Object> logSegmentAllocator;
-    protected final DistributedLock lock;
-    protected final MaxTxId maxTxId;
-    protected final MaxLogSegmentSequenceNo maxLogSegmentSequenceNo;
-    protected final boolean validateLogSegmentSequenceNumber;
-    protected final int regionId;
-    protected final RollingPolicy rollingPolicy;
-    protected Future<? extends DistributedLock> lockFuture = null;
-    protected final PermitLimiter writeLimiter;
-    protected final FeatureProvider featureProvider;
-    protected final DynamicDistributedLogConfiguration dynConf;
-    protected final MetadataUpdater metadataUpdater;
-    // tracking the inprogress log segments
-    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;
-
-    // Recover Functions
-    private final RecoverLogSegmentFunction recoverLogSegmentFunction =
-            new RecoverLogSegmentFunction();
-    private final AbstractFunction1<List<LogSegmentMetadata>, Future<Long>> recoverLogSegmentsFunction =
-            new AbstractFunction1<List<LogSegmentMetadata>, Future<Long>>() {
-                @Override
-                public Future<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) {
-                        if (lastLedgerRollingTimeMillis < 0) {
-                            lastLedgerRollingTimeMillis = Utils.nowInMillis();
-                        }
-                    }
-
-                    if (validateLogSegmentSequenceNumber) {
-                        synchronized (inprogressLSSNs) {
-                            for (LogSegmentMetadata segment : segmentList) {
-                                if (segment.isInProgress()) {
-                                    inprogressLSSNs.addLast(segment.getLogSegmentSequenceNumber());
-                                }
-                            }
-                        }
-                    }
-
-                    return FutureUtils.processList(segmentList, recoverLogSegmentFunction, scheduler).map(
-                            GetLastTxIdFunction.INSTANCE);
-                }
-            };
-
-    // Stats
-    private final StatsLogger perLogStatsLogger;
-    private final OpStatsLogger closeOpStats;
-    private final OpStatsLogger openOpStats;
-    private final OpStatsLogger recoverOpStats;
-    private final OpStatsLogger deleteOpStats;
-
-    /**
-     * Construct a Bookkeeper journal manager.
-     */
-    BKLogWriteHandler(LogMetadataForWriter logMetadata,
-                      DistributedLogConfiguration conf,
-                      LogStreamMetadataStore streamMetadataStore,
-                      LogSegmentMetadataCache metadataCache,
-                      LogSegmentEntryStore entryStore,
-                      OrderedScheduler scheduler,
-                      Allocator<LogSegmentEntryWriter, Object> segmentAllocator,
-                      StatsLogger statsLogger,
-                      StatsLogger perLogStatsLogger,
-                      AlertStatsLogger alertStatsLogger,
-                      String clientId,
-                      int regionId,
-                      PermitLimiter writeLimiter,
-                      FeatureProvider featureProvider,
-                      DynamicDistributedLogConfiguration dynConf,
-                      DistributedLock lock /** owned by handler **/) {
-        super(logMetadata,
-                conf,
-                streamMetadataStore,
-                metadataCache,
-                entryStore,
-                scheduler,
-                statsLogger,
-                alertStatsLogger,
-                clientId);
-        this.logMetadataForWriter = logMetadata;
-        this.logSegmentAllocator = segmentAllocator;
-        this.perLogStatsLogger = perLogStatsLogger;
-        this.writeLimiter = writeLimiter;
-        this.featureProvider = featureProvider;
-        this.dynConf = dynConf;
-        this.lock = lock;
-        this.metadataUpdater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
-
-        if (conf.getEncodeRegionIDInLogSegmentMetadata()) {
-            this.regionId = regionId;
-        } else {
-            this.regionId = DistributedLogConstants.LOCAL_REGION_ID;
-        }
-        this.validateLogSegmentSequenceNumber = conf.isLogSegmentSequenceNumberValidationEnabled();
-
-        // Construct the max sequence no
-        maxLogSegmentSequenceNo = new MaxLogSegmentSequenceNo(logMetadata.getMaxLSSNData());
-        inprogressLSSNs = new LinkedList<Long>();
-        // Construct the max txn id.
-        maxTxId = new MaxTxId(logMetadata.getMaxTxIdData());
-
-        // Schedule fetching log segment list in background before we access it.
-        // We don't need to watch the log segment list changes for writer, as it manages log segment list.
-        fetchForWrite = readLogSegmentsFromStore(
-                LogSegmentMetadata.COMPARATOR,
-                WRITE_HANDLE_FILTER,
-                null);
-
-        // Initialize other parameters.
-        setLastLedgerRollingTimeMillis(Utils.nowInMillis());
-
-        // Rolling Policy
-        if (conf.getLogSegmentRollingIntervalMinutes() > 0) {
-            rollingPolicy = new TimeBasedRollingPolicy(conf.getLogSegmentRollingIntervalMinutes() * 60 * 1000L);
-        } else {
-            rollingPolicy = new SizeBasedRollingPolicy(conf.getMaxLogSegmentBytes());
-        }
-
-        // Stats
-        StatsLogger segmentsStatsLogger = statsLogger.scope("segments");
-        openOpStats = segmentsStatsLogger.getOpStatsLogger("open");
-        closeOpStats = segmentsStatsLogger.getOpStatsLogger("close");
-        recoverOpStats = segmentsStatsLogger.getOpStatsLogger("recover");
-        deleteOpStats = segmentsStatsLogger.getOpStatsLogger("delete");
-    }
-
-    private Future<List<LogSegmentMetadata>> getCachedLogSegmentsAfterFirstFetch(
-            final Comparator<LogSegmentMetadata> comparator) {
-        final Promise<List<LogSegmentMetadata>> promise = new Promise<List<LogSegmentMetadata>>();
-        fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
-            }
-
-            @Override
-            public void onSuccess(Versioned<List<LogSegmentMetadata>> result) {
-                try {
-                    FutureUtils.setValue(promise, getCachedLogSegments(comparator));
-                } catch (UnexpectedException e) {
-                    FutureUtils.setException(promise, e);
-                }
-            }
-        });
-        return promise;
-    }
-
-    private Future<List<LogSegmentMetadata>> getCachedLogSegmentsAfterFirstFullFetch(
-            final Comparator<LogSegmentMetadata> comparator) {
-        Future<Versioned<List<LogSegmentMetadata>>> result;
-        synchronized (this) {
-            if (null == fetchForTruncation) {
-                fetchForTruncation = readLogSegmentsFromStore(
-                        LogSegmentMetadata.COMPARATOR,
-                        LogSegmentFilter.DEFAULT_FILTER,
-                        null);
-            }
-            result = fetchForTruncation;
-        }
-
-        final Promise<List<LogSegmentMetadata>> promise = new Promise<List<LogSegmentMetadata>>();
-        result.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
-            }
-
-            @Override
-            public void onSuccess(Versioned<List<LogSegmentMetadata>> result) {
-                try {
-                    FutureUtils.setValue(promise, getCachedLogSegments(comparator));
-                } catch (UnexpectedException e) {
-                    FutureUtils.setException(promise, e);
-                }
-            }
-        });
-        return promise;
-    }
-
-    // Transactional operations for MaxLogSegmentSequenceNo
-    void storeMaxSequenceNumber(final Transaction<Object> txn,
-                                final MaxLogSegmentSequenceNo maxSeqNo,
-                                final long seqNo,
-                                final boolean isInprogress) {
-        metadataStore.storeMaxLogSegmentSequenceNumber(txn, logMetadata, maxSeqNo.getVersionedData(seqNo),
-                new Transaction.OpListener<Version>() {
-            @Override
-            public void onCommit(Version version) {
-                if (validateLogSegmentSequenceNumber) {
-                    synchronized (inprogressLSSNs) {
-                        if (isInprogress) {
-                            inprogressLSSNs.add(seqNo);
-                        } else {
-                            inprogressLSSNs.removeFirst();
-                        }
-                    }
-                }
-                maxSeqNo.update(version, seqNo);
-            }
-
-            @Override
-            public void onAbort(Throwable t) {
-                // no-op
-            }
-        });
-    }
-
-    // Transactional operations for MaxTxId
-    void storeMaxTxId(final Transaction<Object> txn,
-                      final MaxTxId maxTxId,
-                      final long txId) {
-        metadataStore.storeMaxTxnId(txn, logMetadataForWriter, maxTxId.getVersionedData(txId),
-                new Transaction.OpListener<Version>() {
-                    @Override
-                    public void onCommit(Version version) {
-                                                        maxTxId.update(version, txId);
-                                                                                      }
-
-                    @Override
-                    public void onAbort(Throwable t) {
-                        // no-op
-                    }
-                });
-    }
-
-    // Transactional operations for logsegment
-    void writeLogSegment(final Transaction<Object> txn,
-                         final LogSegmentMetadata metadata) {
-        metadataStore.createLogSegment(txn, metadata, new Transaction.OpListener<Void>() {
-            @Override
-            public void onCommit(Void r) {
-                addLogSegmentToCache(metadata.getSegmentName(), metadata);
-            }
-
-            @Override
-            public void onAbort(Throwable t) {
-                // no-op
-            }
-        });
-    }
-
-    void deleteLogSegment(final Transaction<Object> txn,
-                          final LogSegmentMetadata metadata) {
-        metadataStore.deleteLogSegment(txn, metadata, new Transaction.OpListener<Void>() {
-            @Override
-            public void onCommit(Void r) {
-                removeLogSegmentFromCache(metadata.getSegmentName());
-            }
-
-            @Override
-            public void onAbort(Throwable t) {
-                // no-op
-            }
-        });
-    }
-
-    /**
-     * The caller could call this before any actions, which to hold the lock for
-     * the write handler of its whole lifecycle. The lock will only be released
-     * when closing the write handler.
-     *
-     * This method is useful to prevent releasing underlying zookeeper lock during
-     * recovering/completing log segments. Releasing underlying zookeeper lock means
-     * 1) increase latency when re-lock on starting new log segment. 2) increase the
-     * possibility of a stream being re-acquired by other instances.
-     *
-     * @return future represents the lock result
-     */
-    Future<? extends DistributedLock> lockHandler() {
-        if (null != lockFuture) {
-            return lockFuture;
-        }
-        lockFuture = lock.asyncAcquire();
-        return lockFuture;
-    }
-
-    Future<Void> unlockHandler() {
-        if (null != lockFuture) {
-            return lock.asyncClose();
-        } else {
-            return Future.Void();
-        }
-    }
-
-    /**
-     * Start a new log segment in a BookKeeper ledger.
-     * First ensure that we have the write lock for this journal.
-     * Then create a ledger and stream based on that ledger.
-     * The ledger id is written to the inprogress znode, so that in the
-     * case of a crash, a recovery process can find the ledger we were writing
-     * to when we crashed.
-     *
-     * @param txId First transaction id to be written to the stream
-     * @return
-     * @throws IOException
-     */
-    public BKLogSegmentWriter startLogSegment(long txId) throws IOException {
-        return startLogSegment(txId, false, false);
-    }
-
-    /**
-     * Start a new log segment in a BookKeeper ledger.
-     * First ensure that we have the write lock for this journal.
-     * Then create a ledger and stream based on that ledger.
-     * The ledger id is written to the inprogress znode, so that in the
-     * case of a crash, a recovery process can find the ledger we were writing
-     * to when we crashed.
-     *
-     * @param txId First transaction id to be written to the stream
-     * @param bestEffort
-     * @param allowMaxTxID
-     *          allow using max tx id to start log segment
-     * @return
-     * @throws IOException
-     */
-    public BKLogSegmentWriter startLogSegment(long txId, boolean bestEffort, boolean allowMaxTxID)
-            throws IOException {
-        Stopwatch stopwatch = Stopwatch.createStarted();
-        boolean success = false;
-        try {
-            BKLogSegmentWriter writer = doStartLogSegment(txId, bestEffort, allowMaxTxID);
-            success = true;
-            return writer;
-        } finally {
-            if (success) {
-                openOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            } else {
-                openOpStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            }
-        }
-    }
-
-    protected long assignLogSegmentSequenceNumber() throws IOException {
-        // For any active stream we will always make sure that there is at least one
-        // active ledger (except when the stream first starts out). Therefore when we
-        // see no ledger metadata for a stream, we assume that this is the first ledger
-        // in the stream
-        long logSegmentSeqNo = DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
-        boolean logSegmentsFound = false;
-
-        if (LogSegmentMetadata.supportsLogSegmentSequenceNo(conf.getDLLedgerMetadataLayoutVersion())) {
-            List<LogSegmentMetadata> ledgerListDesc = getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR);
-            Long nextLogSegmentSeqNo = DLUtils.nextLogSegmentSequenceNumber(ledgerListDesc);
-
-            if (null == nextLogSegmentSeqNo) {
-                logSegmentsFound = false;
-                // we don't find last assigned log segment sequence number
-                // then we start the log segment with configured FirstLogSegmentSequenceNumber.
-                logSegmentSeqNo = conf.getFirstLogSegmentSequenceNumber();
-            } else {
-                logSegmentsFound = true;
-                // latest log segment is assigned with a sequence number, start with next sequence number
-                logSegmentSeqNo = nextLogSegmentSeqNo;
-            }
-        }
-
-        // We only skip log segment sequence number validation only when no log segments found &
-        // the maximum log segment sequence number is "UNASSIGNED".
-        if (!logSegmentsFound &&
-            (DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO == maxLogSegmentSequenceNo.getSequenceNumber())) {
-            // no ledger seqno stored in /ledgers before
-            LOG.info("No max ledger sequence number found while creating log segment {} for {}.",
-                logSegmentSeqNo, getFullyQualifiedName());
-        } else if (maxLogSegmentSequenceNo.getSequenceNumber() + 1 != logSegmentSeqNo) {
-            LOG.warn("Unexpected max log segment sequence number {} for {} : list of cached segments = {}",
-                new Object[]{maxLogSegmentSequenceNo.getSequenceNumber(), getFullyQualifiedName(),
-                    getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR)});
-            // there is max log segment number recorded there and it isn't match. throw exception.
-            throw new DLIllegalStateException("Unexpected max log segment sequence number "
-                + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName()
-                + ", expected " + (logSegmentSeqNo - 1));
-        }
-
-        return logSegmentSeqNo;
-    }
-
-    protected BKLogSegmentWriter doStartLogSegment(long txId, boolean bestEffort, boolean allowMaxTxID) throws IOException {
-        return FutureUtils.result(asyncStartLogSegment(txId, bestEffort, allowMaxTxID));
-    }
-
-    protected Future<BKLogSegmentWriter> asyncStartLogSegment(final long txId,
-                                                              final boolean bestEffort,
-                                                              final boolean allowMaxTxID) {
-        final Promise<BKLogSegmentWriter> promise = new Promise<BKLogSegmentWriter>();
-        try {
-            lock.checkOwnershipAndReacquire();
-        } catch (LockingException e) {
-            FutureUtils.setException(promise, e);
-            return promise;
-        }
-        fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
-            }
-
-            @Override
-            public void onSuccess(Versioned<List<LogSegmentMetadata>> list) {
-                doStartLogSegment(txId, bestEffort, allowMaxTxID, promise);
-            }
-        });
-        return promise;
-    }
-
-    protected void doStartLogSegment(final long txId,
-                                     final boolean bestEffort,
-                                     final boolean allowMaxTxID,
-                                     final Promise<BKLogSegmentWriter> promise) {
-        // validate the tx id
-        if ((txId < 0) ||
-                (!allowMaxTxID && (txId == DistributedLogConstants.MAX_TXID))) {
-            FutureUtils.setException(promise, new IOException("Invalid Transaction Id " + txId));
-            return;
-        }
-
-        long highestTxIdWritten = maxTxId.get();
-        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"));
-                return;
-            } else {
-                LOG.error("We've already seen TxId {} the max TXId is {}", txId, highestTxIdWritten);
-                FutureUtils.setException(promise, new TransactionIdOutOfOrderException(txId, highestTxIdWritten));
-                return;
-            }
-        }
-
-        try {
-            logSegmentAllocator.allocate();
-        } catch (IOException e) {
-            // failed to issue an allocation request
-            failStartLogSegment(promise, bestEffort, e);
-            return;
-        }
-
-        // start the transaction from zookeeper
-        final Transaction<Object> txn = streamMetadataStore.newTransaction();
-
-        // failpoint injected before creating ledger
-        try {
-            FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_StartLogSegmentBeforeLedgerCreate);
-        } catch (IOException ioe) {
-            failStartLogSegment(promise, bestEffort, ioe);
-            return;
-        }
-
-        logSegmentAllocator.tryObtain(txn, NULL_OP_LISTENER)
-                .addEventListener(new FutureEventListener<LogSegmentEntryWriter>() {
-
-            @Override
-            public void onSuccess(LogSegmentEntryWriter entryWriter) {
-                // try-obtain succeed
-                createInprogressLogSegment(
-                        txn,
-                        txId,
-                        entryWriter,
-                        bestEffort,
-                        promise);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                failStartLogSegment(promise, bestEffort, cause);
-            }
-        });
-    }
-
-    private void failStartLogSegment(Promise<BKLogSegmentWriter> promise,
-                                     boolean bestEffort,
-                                     Throwable cause) {
-        if (bestEffort) {
-            FutureUtils.setValue(promise, null);
-        } else {
-            FutureUtils.setException(promise, cause);
-        }
-    }
-
-    // once the ledger handle is obtained from allocator, this function should guarantee
-    // either the transaction is executed or aborted. Otherwise, the ledger handle will
-    // just leak from the allocation pool - hence cause "No Ledger Allocator"
-    private void createInprogressLogSegment(Transaction<Object> txn,
-                                            final long txId,
-                                            final LogSegmentEntryWriter entryWriter,
-                                            boolean bestEffort,
-                                            final Promise<BKLogSegmentWriter> promise) {
-        final long logSegmentSeqNo;
-        try {
-            FailpointUtils.checkFailPoint(
-                    FailpointUtils.FailPointName.FP_StartLogSegmentOnAssignLogSegmentSequenceNumber);
-            logSegmentSeqNo = assignLogSegmentSequenceNumber();
-        } catch (IOException e) {
-            // abort the current prepared transaction
-            txn.abort(e);
-            failStartLogSegment(promise, bestEffort, e);
-            return;
-        }
-
-        final String inprogressZnodePath = inprogressZNode(
-                entryWriter.getLogSegmentId(), txId, logSegmentSeqNo);
-        final LogSegmentMetadata l =
-            new LogSegmentMetadata.LogSegmentMetadataBuilder(inprogressZnodePath,
-                conf.getDLLedgerMetadataLayoutVersion(), entryWriter.getLogSegmentId(), txId)
-                    .setLogSegmentSequenceNo(logSegmentSeqNo)
-                    .setRegionId(regionId)
-                    .setEnvelopeEntries(
-                            LogSegmentMetadata.supportsEnvelopedEntries(conf.getDLLedgerMetadataLayoutVersion()))
-                    .build();
-
-        // Create an inprogress segment
-        writeLogSegment(txn, l);
-
-        // Try storing max sequence number.
-        LOG.debug("Try storing max sequence number in startLogSegment {} : {}", inprogressZnodePath, logSegmentSeqNo);
-        storeMaxSequenceNumber(txn, maxLogSegmentSequenceNo, logSegmentSeqNo, true);
-
-        // Try storing max tx id.
-        LOG.debug("Try storing MaxTxId in startLogSegment  {} {}", inprogressZnodePath, txId);
-        storeMaxTxId(txn, maxTxId, txId);
-
-        txn.execute().addEventListener(FutureEventListenerRunnable.of(new FutureEventListener<Void>() {
-
-            @Override
-            public void onSuccess(Void value) {
-                try {
-                    FutureUtils.setValue(promise, new BKLogSegmentWriter(
-                            getFullyQualifiedName(),
-                            l.getSegmentName(),
-                            conf,
-                            conf.getDLLedgerMetadataLayoutVersion(),
-                            entryWriter,
-                            lock,
-                            txId,
-                            logSegmentSeqNo,
-                            scheduler,
-                            statsLogger,
-                            perLogStatsLogger,
-                            alertStatsLogger,
-                            writeLimiter,
-                            featureProvider,
-                            dynConf));
-                } catch (IOException ioe) {
-                    failStartLogSegment(promise, false, ioe);
-                }
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                failStartLogSegment(promise, false, cause);
-            }
-        }, scheduler));
-    }
-
-    boolean shouldStartNewSegment(BKLogSegmentWriter writer) {
-        return rollingPolicy.shouldRollover(writer, lastLedgerRollingTimeMillis);
-    }
-
-    /**
-     * Finalize a log segment. If the journal manager is currently
-     * writing to a ledger, ensure that this is the ledger of the log segment
-     * being finalized.
-     * <p/>
-     * Otherwise this is the recovery case. In the recovery case, ensure that
-     * 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>();
-        completeAndCloseLogSegment(writer, promise);
-        return promise;
-    }
-
-    private void completeAndCloseLogSegment(final BKLogSegmentWriter writer,
-                                            final Promise<LogSegmentMetadata> promise) {
-        writer.asyncClose().addEventListener(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,
-                            new IOException("LogSegmentWriter for " + writer.getFullyQualifiedLogSegment() + " is already in error."));
-                    return;
-                }
-                doCompleteAndCloseLogSegment(
-                        inprogressZNodeName(writer.getLogSegmentId(), writer.getStartTxId(), writer.getLogSegmentSequenceNumber()),
-                        writer.getLogSegmentSequenceNumber(),
-                        writer.getLogSegmentId(),
-                        writer.getStartTxId(),
-                        writer.getLastTxId(),
-                        writer.getPositionWithinLogSegment(),
-                        writer.getLastDLSN().getEntryId(),
-                        writer.getLastDLSN().getSlotId(),
-                        promise);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
-            }
-        });
-    }
-
-    @VisibleForTesting
-    LogSegmentMetadata completeAndCloseLogSegment(long logSegmentSeqNo,
-                                                  long logSegmentId,
-                                                  long firstTxId,
-                                                  long lastTxId,
-                                                  int recordCount)
-        throws IOException {
-        return completeAndCloseLogSegment(inprogressZNodeName(logSegmentId, firstTxId, logSegmentSeqNo), logSegmentSeqNo,
-            logSegmentId, firstTxId, lastTxId, recordCount, -1, -1);
-    }
-
-    /**
-     * Finalize a log segment. If the journal manager is currently
-     * writing to a ledger, ensure that this is the ledger of the log segment
-     * being finalized.
-     * <p/>
-     * Otherwise this is the recovery case. In the recovery case, ensure that
-     * the firstTxId of the ledger matches firstTxId for the segment we are
-     * trying to finalize.
-     */
-    LogSegmentMetadata completeAndCloseLogSegment(String inprogressZnodeName, long logSegmentSeqNo,
-                                                  long logSegmentId, long firstTxId, long lastTxId,
-                                                  int recordCount, long lastEntryId, long lastSlotId)
-            throws IOException {
-        Stopwatch stopwatch = Stopwatch.createStarted();
-        boolean success = false;
-        try {
-            LogSegmentMetadata completedLogSegment =
-                    doCompleteAndCloseLogSegment(inprogressZnodeName, logSegmentSeqNo,
-                            logSegmentId, firstTxId, lastTxId, recordCount,
-                            lastEntryId, lastSlotId);
-            success = true;
-            return completedLogSegment;
-        } finally {
-            if (success) {
-                closeOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            } else {
-                closeOpStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            }
-        }
-    }
-
-    protected long computeStartSequenceId(LogSegmentMetadata segment) throws IOException {
-        if (!segment.isInProgress()) {
-            return segment.getStartSequenceId();
-        }
-
-        long startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
-
-        // we only record sequence id when both write version and logsegment's version support sequence id
-        if (LogSegmentMetadata.supportsSequenceId(conf.getDLLedgerMetadataLayoutVersion())
-                && segment.supportsSequenceId()) {
-            List<LogSegmentMetadata> logSegmentDescList =
-                    getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR);
-            startSequenceId = DLUtils.computeStartSequenceId(logSegmentDescList, segment);
-        }
-
-        return startSequenceId;
-    }
-
-    /**
-     * Close log segment
-     *
-     * @param inprogressZnodeName
-     * @param logSegmentSeqNo
-     * @param logSegmentId
-     * @param firstTxId
-     * @param lastTxId
-     * @param recordCount
-     * @param lastEntryId
-     * @param lastSlotId
-     * @throws IOException
-     */
-    protected LogSegmentMetadata doCompleteAndCloseLogSegment(
-            String inprogressZnodeName,
-            long logSegmentSeqNo,
-            long logSegmentId,
-            long firstTxId,
-            long lastTxId,
-            int recordCount,
-            long lastEntryId,
-            long lastSlotId) throws IOException {
-        Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
-        doCompleteAndCloseLogSegment(
-                inprogressZnodeName,
-                logSegmentSeqNo,
-                logSegmentId,
-                firstTxId,
-                lastTxId,
-                recordCount,
-                lastEntryId,
-                lastSlotId,
-                promise);
-        return FutureUtils.result(promise);
-    }
-
-    protected void doCompleteAndCloseLogSegment(final String inprogressZnodeName,
-                                                final long logSegmentSeqNo,
-                                                final long logSegmentId,
-                                                final long firstTxId,
-                                                final long lastTxId,
-                                                final int recordCount,
-                                                final long lastEntryId,
-                                                final long lastSlotId,
-                                                final Promise<LogSegmentMetadata> promise) {
-        fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
-            }
-
-            @Override
-            public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
-                doCompleteAndCloseLogSegmentAfterLogSegmentListFetched(
-                        inprogressZnodeName,
-                        logSegmentSeqNo,
-                        logSegmentId,
-                        firstTxId,
-                        lastTxId,
-                        recordCount,
-                        lastEntryId,
-                        lastSlotId,
-                        promise);
-            }
-        });
-    }
-
-    private void doCompleteAndCloseLogSegmentAfterLogSegmentListFetched(
-            final String inprogressZnodeName,
-            long logSegmentSeqNo,
-            long logSegmentId,
-            long firstTxId,
-            long lastTxId,
-            int recordCount,
-            long lastEntryId,
-            long lastSlotId,
-            final Promise<LogSegmentMetadata> promise) {
-        try {
-            lock.checkOwnershipAndReacquire();
-        } catch (IOException ioe) {
-            FutureUtils.setException(promise, ioe);
-            return;
-        }
-
-        LOG.debug("Completing and Closing Log Segment {} {}", firstTxId, lastTxId);
-        LogSegmentMetadata inprogressLogSegment = readLogSegmentFromCache(inprogressZnodeName);
-
-        // validate log segment
-        if (inprogressLogSegment.getLogSegmentId() != logSegmentId) {
-            FutureUtils.setException(promise, new IOException(
-                "Active ledger has different ID to inprogress. "
-                    + inprogressLogSegment.getLogSegmentId() + " found, "
-                    + logSegmentId + " expected"));
-            return;
-        }
-        // validate the transaction id
-        if (inprogressLogSegment.getFirstTxId() != firstTxId) {
-            FutureUtils.setException(promise, new IOException("Transaction id not as expected, "
-                + inprogressLogSegment.getFirstTxId() + " found, " + firstTxId + " expected"));
-            return;
-        }
-        // validate the log sequence number
-        if (validateLogSegmentSequenceNumber) {
-            synchronized (inprogressLSSNs) {
-                if (inprogressLSSNs.isEmpty()) {
-                    FutureUtils.setException(promise, new UnexpectedException(
-                            "Didn't find matched inprogress log segments when completing inprogress "
-                                    + inprogressLogSegment));
-                    return;
-                }
-                long leastInprogressLSSN = inprogressLSSNs.getFirst();
-                // the log segment sequence number in metadata {@link inprogressLogSegment.getLogSegmentSequenceNumber()}
-                // should be same as the sequence number we are completing (logSegmentSeqNo)
-                // and
-                // 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(
-                            "Didn't find matched inprogress log segments when completing inprogress "
-                                    + inprogressLogSegment));
-                    return;
-                }
-            }
-        }
-
-        // store max sequence number.
-        long maxSeqNo= Math.max(logSegmentSeqNo, maxLogSegmentSequenceNo.getSequenceNumber());
-        if (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo ||
-                (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo + 1)) {
-            // ignore the case that a new inprogress log segment is pre-allocated
-            // before completing current inprogress one
-            LOG.info("Try storing max sequence number {} in completing {}.",
-                    new Object[] { logSegmentSeqNo, inprogressLogSegment.getZkPath() });
-        } else {
-            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 "
-                        + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName()
-                        + ", expected " + (logSegmentSeqNo - 1)));
-                return;
-            }
-        }
-
-        // Prepare the completion
-        final String pathForCompletedLedger = completedLedgerZNode(firstTxId, lastTxId, logSegmentSeqNo);
-        long startSequenceId;
-        try {
-            startSequenceId = computeStartSequenceId(inprogressLogSegment);
-        } catch (IOException ioe) {
-            FutureUtils.setException(promise, ioe);
-            return;
-        }
-        // write completed ledger znode
-        final LogSegmentMetadata completedLogSegment =
-                inprogressLogSegment.completeLogSegment(
-                        pathForCompletedLedger,
-                        lastTxId,
-                        recordCount,
-                        lastEntryId,
-                        lastSlotId,
-                        startSequenceId);
-        setLastLedgerRollingTimeMillis(completedLogSegment.getCompletionTime());
-
-        // prepare the transaction
-        Transaction<Object> txn = streamMetadataStore.newTransaction();
-
-        // create completed log segment
-        writeLogSegment(txn, completedLogSegment);
-        // delete inprogress log segment
-        deleteLogSegment(txn, inprogressLogSegment);
-        // store max sequence number
-        storeMaxSequenceNumber(txn, maxLogSegmentSequenceNo, maxSeqNo, false);
-        // update max txn id.
-        LOG.debug("Trying storing LastTxId in Finalize Path {} LastTxId {}", pathForCompletedLedger, lastTxId);
-        storeMaxTxId(txn, maxTxId, lastTxId);
-
-        txn.execute().addEventListener(FutureEventListenerRunnable.of(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);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
-            }
-        }, scheduler));
-    }
-
-    public Future<Long> recoverIncompleteLogSegments() {
-        try {
-            FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_RecoverIncompleteLogSegments);
-        } catch (IOException ioe) {
-            return Future.exception(ioe);
-        }
-        return getCachedLogSegmentsAfterFirstFetch(LogSegmentMetadata.COMPARATOR).flatMap(recoverLogSegmentsFunction);
-    }
-
-    class RecoverLogSegmentFunction extends Function<LogSegmentMetadata, Future<LogSegmentMetadata>> {
-
-        @Override
-        public Future<LogSegmentMetadata> apply(final LogSegmentMetadata l) {
-            if (!l.isInProgress()) {
-                return Future.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);
-                        }
-                    });
-        }
-
-        private Future<LogSegmentMetadata> completeLogSegment(LogSegmentMetadata l,
-                                                              LogRecordWithDLSN lastRecord) {
-            LOG.info("Recovered last record in log segment {} for {}.", l, getFullyQualifiedName());
-
-            long endTxId = DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID;
-            int recordCount = 0;
-            long lastEntryId = -1;
-            long lastSlotId = -1;
-
-            if (null != lastRecord) {
-                endTxId = lastRecord.getTransactionId();
-                recordCount = lastRecord.getLastPositionWithinLogSegment();
-                lastEntryId = lastRecord.getDlsn().getEntryId();
-                lastSlotId = lastRecord.getDlsn().getSlotId();
-            }
-
-            if (endTxId == DistributedLogConstants.INVALID_TXID) {
-                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,"
-                    + " 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>();
-            doCompleteAndCloseLogSegment(
-                    l.getZNodeName(),
-                    l.getLogSegmentSequenceNumber(),
-                    l.getLogSegmentId(),
-                    l.getFirstTxId(),
-                    endTxId,
-                    recordCount,
-                    lastEntryId,
-                    lastSlotId,
-                    promise);
-            return promise;
-        }
-
-    }
-
-    Future<List<LogSegmentMetadata>> setLogSegmentsOlderThanDLSNTruncated(final DLSN dlsn) {
-        if (DLSN.InvalidDLSN == dlsn) {
-            List<LogSegmentMetadata> emptyList = new ArrayList<LogSegmentMetadata>(0);
-            return Future.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);
-                    }
-                });
-    }
-
-    private Future<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()});
-        List<LogSegmentMetadata> truncateList = new ArrayList<LogSegmentMetadata>(logSegments.size());
-        LogSegmentMetadata partialTruncate = null;
-        LOG.info("{}: Truncating log segments older than {}", getFullyQualifiedName(), dlsn);
-        for (int i = 0; i < logSegments.size(); i++) {
-            LogSegmentMetadata l = logSegments.get(i);
-            if (!l.isInProgress()) {
-                if (l.getLastDLSN().compareTo(dlsn) < 0) {
-                    LOG.debug("{}: Truncating log segment {} ", getFullyQualifiedName(), l);
-                    truncateList.add(l);
-                } else if (l.getFirstDLSN().compareTo(dlsn) < 0) {
-                    // Can be satisfied by at most one segment
-                    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));
-                    }
-                    LOG.info("{}: Partially truncating log segment {} older than {}.", new Object[] {getFullyQualifiedName(), l, dlsn});
-                    partialTruncate = l;
-                } else {
-                    break;
-                }
-            } else {
-                break;
-            }
-        }
-        return setLogSegmentTruncationStatus(truncateList, partialTruncate, dlsn);
-    }
-
-    private int getNumCandidateLogSegmentsToPurge(List<LogSegmentMetadata> logSegments) {
-        if (logSegments.isEmpty()) {
-            return 0;
-        } else {
-            // we have to keep at least one completed log segment for sequence id
-            int numCandidateLogSegments = 0;
-            for (LogSegmentMetadata segment : logSegments) {
-                if (segment.isInProgress()) {
-                    break;
-                } else {
-                    ++numCandidateLogSegments;
-                }
-            }
-
-            return numCandidateLogSegments - 1;
-        }
-    }
-
-    Future<List<LogSegmentMetadata>> purgeLogSegmentsOlderThanTimestamp(final long minTimestampToKeep) {
-        if (minTimestampToKeep >= Utils.nowInMillis()) {
-            return Future.exception(new IllegalArgumentException(
-                    "Invalid timestamp " + minTimestampToKeep + " to purge logs for " + getFullyQualifiedName()));
-        }
-        return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).flatMap(
-                new Function<List<LogSegmentMetadata>, Future<List<LogSegmentMetadata>>>() {
-            @Override
-            public Future<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) {
-                List<LogSegmentMetadata> purgeList = new ArrayList<LogSegmentMetadata>(logSegments.size());
-
-                int numCandidates = getNumCandidateLogSegmentsToPurge(logSegments);
-
-                for (int iterator = 0; iterator < numCandidates; iterator++) {
-                    LogSegmentMetadata l = logSegments.get(iterator);
-                    // When application explicitly truncates segments; timestamp based purge is
-                    // only used to cleanup log segments that have been marked for truncation
-                    if ((l.isTruncated() || !conf.getExplicitTruncationByApplication()) &&
-                        !l.isInProgress() && (l.getCompletionTime() < minTimestampToKeep)) {
-                        purgeList.add(l);
-                    } else {
-                        // stop truncating log segments if we find either an inprogress or a partially
-                        // truncated log segment
-                        break;
-                    }
-                }
-                LOG.info("Deleting log segments older than {} for {} : {}",
-                        new Object[] { minTimestampToKeep, getFullyQualifiedName(), purgeList });
-                return deleteLogSegments(purgeList);
-            }
-        });
-    }
-
-    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;
-
-                    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(
-            final List<LogSegmentMetadata> truncateList,
-            LogSegmentMetadata partialTruncate,
-            DLSN minActiveDLSN) {
-        final List<LogSegmentMetadata> listToTruncate = Lists.newArrayListWithCapacity(truncateList.size() + 1);
-        final List<LogSegmentMetadata> listAfterTruncated = Lists.newArrayListWithCapacity(truncateList.size() + 1);
-        Transaction<Object> updateTxn = metadataUpdater.transaction();
-        for(LogSegmentMetadata l : truncateList) {
-            if (!l.isTruncated()) {
-                LogSegmentMetadata newSegment = metadataUpdater.setLogSegmentTruncated(updateTxn, l);
-                listToTruncate.add(l);
-                listAfterTruncated.add(newSegment);
-            }
-        }
-
-        if (null != partialTruncate && (partialTruncate.isNonTruncated() ||
-                (partialTruncate.isPartiallyTruncated() && (partialTruncate.getMinActiveDLSN().compareTo(minActiveDLSN) < 0)))) {
-            LogSegmentMetadata newSegment = metadataUpdater.setLogSegmentPartiallyTruncated(
-                    updateTxn, partialTruncate, minActiveDLSN);
-            listToTruncate.add(partialTruncate);
-            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;
-            }
-        });
-    }
-
-    private Future<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);
-    }
-
-    private Future<LogSegmentMetadata> deleteLogSegment(
-            final LogSegmentMetadata ledgerMetadata) {
-        LOG.info("Deleting ledger {} for {}", ledgerMetadata, getFullyQualifiedName());
-        final Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
-        final Stopwatch stopwatch = Stopwatch.createStarted();
-        promise.addEventListener(new FutureEventListener<LogSegmentMetadata>() {
-            @Override
-            public void onSuccess(LogSegmentMetadata segment) {
-                deleteOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                deleteOpStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            }
-        });
-        entryStore.deleteLogSegment(ledgerMetadata)
-                .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                FutureUtils.setException(promise, cause);
-            }
-
-            @Override
-            public void onSuccess(LogSegmentMetadata segment) {
-                deleteLogSegmentMetadata(segment, promise);
-            }
-        });
-        return promise;
-    }
-
-    private void deleteLogSegmentMetadata(final LogSegmentMetadata segmentMetadata,
-                                          final Promise<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);
-            }
-
-            @Override
-            public void onAbort(Throwable t) {
-                if (t instanceof LogSegmentNotFoundException) {
-                    // purge log segment
-                    removeLogSegmentFromCache(segmentMetadata.getZNodeName());
-                    promise.setValue(segmentMetadata);
-                    return;
-                } else {
-                    LOG.error("Couldn't purge {} for {}: with error {}",
-                            new Object[]{ segmentMetadata, getFullyQualifiedName(), t });
-                    promise.setException(t);
-                }
-            }
-        });
-        deleteTxn.execute();
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        return Utils.closeSequence(scheduler,
-                lock,
-                logSegmentAllocator);
-    }
-
-    @Override
-    public Future<Void> asyncAbort() {
-        return asyncClose();
-    }
-
-    String completedLedgerZNodeName(long firstTxId, long lastTxId, long logSegmentSeqNo) {
-        if (DistributedLogConstants.LOGSEGMENT_NAME_VERSION == conf.getLogSegmentNameVersion()) {
-            return String.format("%s_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX, logSegmentSeqNo);
-        } else {
-            return String.format("%s_%018d_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX,
-                    firstTxId, lastTxId);
-        }
-    }
-
-    /**
-     * Get the znode path for a finalize ledger
-     */
-    String completedLedgerZNode(long firstTxId, long lastTxId, long logSegmentSeqNo) {
-        return String.format("%s/%s", logMetadata.getLogSegmentsPath(),
-                completedLedgerZNodeName(firstTxId, lastTxId, logSegmentSeqNo));
-    }
-
-    /**
-     * Get the name of the inprogress znode.
-     *
-     * @return name of the inprogress znode.
-     */
-    String inprogressZNodeName(long logSegmentId, long firstTxId, long logSegmentSeqNo) {
-        if (DistributedLogConstants.LOGSEGMENT_NAME_VERSION == conf.getLogSegmentNameVersion()) {
-            // Lots of the problems are introduced due to different inprogress names with same ledger sequence number.
-            return String.format("%s_%018d", DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX, logSegmentSeqNo);
-        } else {
-            return DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX + "_" + Long.toString(firstTxId, 16);
-        }
-    }
-
-    /**
-     * Get the znode path for the inprogressZNode
-     */
-    String inprogressZNode(long logSegmentId, long firstTxId, long logSegmentSeqNo) {
-        return logMetadata.getLogSegmentsPath() + "/" + inprogressZNodeName(logSegmentId, firstTxId, logSegmentSeqNo);
-    }
-
-    String inprogressZNode(String inprogressZNodeName) {
-        return logMetadata.getLogSegmentsPath() + "/" + inprogressZNodeName;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogReader.java
deleted file mode 100644
index 308f42a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogReader.java
+++ /dev/null
@@ -1,276 +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 com.twitter.distributedlog;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Ticker;
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.distributedlog.exceptions.IdleReaderException;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.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;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * Synchronous Log Reader based on {@link AsyncLogReader}
- */
-class BKSyncLogReader implements LogReader, AsyncNotification {
-
-    private final BKDistributedLogManager bkdlm;
-    private final BKLogReadHandler readHandler;
-    private final AtomicReference<IOException> readerException =
-            new AtomicReference<IOException>(null);
-    private final int maxReadAheadWaitTime;
-    private Promise<Void> closeFuture;
-    private final Optional<Long> startTransactionId;
-    private boolean positioned = false;
-    private Entry.Reader currentEntry = null;
-
-    // readahead reader
-    ReadAheadEntryReader readAheadReader = null;
-
-    // idle reader settings
-    private final boolean shouldCheckIdleReader;
-    private final int idleErrorThresholdMillis;
-
-    // Stats
-    private final Counter idleReaderError;
-
-    BKSyncLogReader(DistributedLogConfiguration conf,
-                    BKDistributedLogManager bkdlm,
-                    DLSN startDLSN,
-                    Optional<Long> startTransactionId,
-                    StatsLogger statsLogger) throws IOException {
-        this.bkdlm = bkdlm;
-        this.readHandler = bkdlm.createReadHandler(
-                Optional.<String>absent(),
-                this,
-                true);
-        this.maxReadAheadWaitTime = conf.getReadAheadWaitTime();
-        this.idleErrorThresholdMillis = conf.getReaderIdleErrorThresholdMillis();
-        this.shouldCheckIdleReader = idleErrorThresholdMillis > 0 && idleErrorThresholdMillis < Integer.MAX_VALUE;
-        this.startTransactionId = startTransactionId;
-
-        // start readahead
-        startReadAhead(startDLSN);
-        if (!startTransactionId.isPresent()) {
-            positioned = true;
-        }
-
-        // Stats
-        StatsLogger syncReaderStatsLogger = statsLogger.scope("sync_reader");
-        idleReaderError = syncReaderStatsLogger.getCounter("idle_reader_error");
-    }
-
-    private void startReadAhead(DLSN startDLSN) throws IOException {
-        readAheadReader = new ReadAheadEntryReader(
-                    bkdlm.getStreamName(),
-                    startDLSN,
-                    bkdlm.getConf(),
-                    readHandler,
-                    bkdlm.getReaderEntryStore(),
-                    bkdlm.getScheduler(),
-                    Ticker.systemTicker(),
-                    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;
-                    }
-                });
-    }
-
-    @VisibleForTesting
-    ReadAheadEntryReader getReadAheadReader() {
-        return readAheadReader;
-    }
-
-    @VisibleForTesting
-    BKLogReadHandler getReadHandler() {
-        return readHandler;
-    }
-
-    private Entry.Reader readNextEntry(boolean nonBlocking) throws IOException {
-        Entry.Reader entry = null;
-        if (nonBlocking) {
-            return readAheadReader.getNextReadAheadEntry(0L, TimeUnit.MILLISECONDS);
-        } else {
-            while (!readAheadReader.isReadAheadCaughtUp()
-                    && null == readerException.get()
-                    && null == entry) {
-                entry = readAheadReader.getNextReadAheadEntry(maxReadAheadWaitTime, TimeUnit.MILLISECONDS);
-            }
-            if (null != entry) {
-                return entry;
-            }
-            // reader is caught up
-            if (readAheadReader.isReadAheadCaughtUp()
-                    && null == readerException.get()) {
-                entry = readAheadReader.getNextReadAheadEntry(maxReadAheadWaitTime, TimeUnit.MILLISECONDS);
-            }
-            return entry;
-        }
-    }
-
-    private void markReaderAsIdle() throws IdleReaderException {
-        idleReaderError.inc();
-        IdleReaderException ire = new IdleReaderException("Sync reader on stream "
-                + readHandler.getFullyQualifiedName()
-                + " is idle for more than " + idleErrorThresholdMillis + " ms");
-        readerException.compareAndSet(null, ire);
-        throw ire;
-    }
-
-    @Override
-    public synchronized LogRecordWithDLSN readNext(boolean nonBlocking)
-            throws IOException {
-        if (null != readerException.get()) {
-            throw readerException.get();
-        }
-        LogRecordWithDLSN record = doReadNext(nonBlocking);
-        // no record is returned, check if the reader becomes idle
-        if (null == record && shouldCheckIdleReader) {
-            if (readAheadReader.getNumCachedEntries() <= 0 &&
-                    readAheadReader.isReaderIdle(idleErrorThresholdMillis, TimeUnit.MILLISECONDS)) {
-                markReaderAsIdle();
-            }
-        }
-        return record;
-    }
-
-    private LogRecordWithDLSN doReadNext(boolean nonBlocking) throws IOException {
-        LogRecordWithDLSN record = null;
-
-        do {
-            // fetch one record until we don't find any entry available in the readahead cache
-            while (null == record) {
-                if (null == currentEntry) {
-                    currentEntry = readNextEntry(nonBlocking);
-                    if (null == currentEntry) {
-                        return null;
-                    }
-                }
-                record = currentEntry.nextRecord();
-                if (null == record) {
-                    currentEntry = null;
-                }
-            }
-
-            // check if we reached the end of stream
-            if (record.isEndOfStream()) {
-                EndOfStreamException eos = new EndOfStreamException("End of Stream Reached for "
-                        + readHandler.getFullyQualifiedName());
-                readerException.compareAndSet(null, eos);
-                throw eos;
-            }
-            // skip control records
-            if (record.isControl()) {
-                record = null;
-                continue;
-            }
-            if (!positioned) {
-                if (record.getTransactionId() < startTransactionId.get()) {
-                    record = null;
-                    continue;
-                } else {
-                    positioned = true;
-                    break;
-                }
-            } else {
-                break;
-            }
-        } while (true);
-        return record;
-    }
-
-    @Override
-    public synchronized List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords)
-            throws IOException {
-        LinkedList<LogRecordWithDLSN> retList =
-                new LinkedList<LogRecordWithDLSN>();
-
-        int numRead = 0;
-        LogRecordWithDLSN record = readNext(nonBlocking);
-        while ((null != record)) {
-            retList.add(record);
-            numRead++;
-            if (numRead >= numLogRecords) {
-                break;
-            }
-            record = readNext(nonBlocking);
-        }
-        return retList;
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        Promise<Void> closePromise;
-        synchronized (this) {
-            if (null != closeFuture) {
-                return closeFuture;
-            }
-            closeFuture = closePromise = new Promise<Void>();
-        }
-        readHandler.unregisterListener(readAheadReader);
-        readAheadReader.removeStateChangeNotification(this);
-        Utils.closeSequence(bkdlm.getScheduler(), true,
-                readAheadReader,
-                readHandler
-        ).proxyTo(closePromise);
-        return closePromise;
-    }
-
-    @Override
-    public void close() throws IOException {
-        FutureUtils.result(asyncClose());
-    }
-
-    //
-    // Notification From ReadHandler
-    //
-
-    @Override
-    public void notifyOnError(Throwable cause) {
-        if (cause instanceof IOException) {
-            readerException.compareAndSet(null, (IOException) cause);
-        } else {
-            readerException.compareAndSet(null, new IOException(cause));
-        }
-    }
-
-    @Override
-    public void notifyOnOperationComplete() {
-        // no-op
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogWriter.java
deleted file mode 100644
index b638020..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogWriter.java
+++ /dev/null
@@ -1,113 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.util.FutureUtils;
-
-import java.io.IOException;
-import java.util.List;
-
-class BKSyncLogWriter extends BKAbstractLogWriter implements LogWriter {
-
-    public BKSyncLogWriter(DistributedLogConfiguration conf,
-                           DynamicDistributedLogConfiguration dynConf,
-                           BKDistributedLogManager bkdlm) {
-        super(conf, dynConf, bkdlm);
-    }
-    /**
-     * Write log records to the stream.
-     *
-     * @param record operation
-     */
-    @Override
-    public void write(LogRecord record) throws IOException {
-        getLedgerWriter(record.getTransactionId(), false).write(record);
-    }
-
-    /**
-     * Write edits logs operation to the stream.
-     *
-     * @param records list of records
-     */
-    @Override
-    @Deprecated
-    public int writeBulk(List<LogRecord> records) throws IOException {
-        return getLedgerWriter(records.get(0).getTransactionId(), false).writeBulk(records);
-    }
-
-    /**
-     * Flushes all the data up to this point,
-     * adds the end of stream marker and marks the stream
-     * as read-only in the metadata. No appends to the
-     * stream will be allowed after this point
-     */
-    @Override
-    public void markEndOfStream() throws IOException {
-        FutureUtils.result(getLedgerWriter(DistributedLogConstants.MAX_TXID, true).markEndOfStream());
-        closeAndComplete();
-    }
-
-    /**
-     * All data that has been written to the stream so far will be flushed.
-     * New data can be still written to the stream while flush is ongoing.
-     */
-    @Override
-    public long setReadyToFlush() throws IOException {
-        checkClosedOrInError("setReadyToFlush");
-        long highestTransactionId = 0;
-        BKLogSegmentWriter writer = getCachedLogWriter();
-        if (null != writer) {
-            highestTransactionId = Math.max(highestTransactionId, FutureUtils.result(writer.flush()));
-        }
-        return highestTransactionId;
-    }
-
-    /**
-     * Commit data that is already flushed.
-     * <p/>
-     * This API is optional as the writer implements a policy for automatically syncing
-     * the log records in the buffer. The buffered edits can be flushed when the buffer
-     * becomes full or a certain period of time is elapsed.
-     */
-    @Override
-    public long flushAndSync() throws IOException {
-        checkClosedOrInError("flushAndSync");
-
-        LOG.debug("FlushAndSync Started");
-        long highestTransactionId = 0;
-        BKLogSegmentWriter writer = getCachedLogWriter();
-        if (null != writer) {
-            highestTransactionId = Math.max(highestTransactionId, FutureUtils.result(writer.commit()));
-            LOG.debug("FlushAndSync Completed");
-        } else {
-            LOG.debug("FlushAndSync Completed - Nothing to Flush");
-        }
-        return highestTransactionId;
-    }
-
-    /**
-     * Close the stream without necessarily flushing immediately.
-     * This may be called if the stream is in error such as after a
-     * previous write or close threw an exception.
-     */
-    @Override
-    public void abort() throws IOException {
-        super.abort();
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKTransmitPacket.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BKTransmitPacket.java
deleted file mode 100644
index 4586602..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BKTransmitPacket.java
+++ /dev/null
@@ -1,90 +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 com.twitter.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.TimeUnit;
-
-class BKTransmitPacket {
-
-    private final EntryBuffer recordSet;
-    private final long transmitTime;
-    private final Promise<Integer> transmitComplete;
-
-    BKTransmitPacket(EntryBuffer recordSet) {
-        this.recordSet = recordSet;
-        this.transmitTime = System.nanoTime();
-        this.transmitComplete = new Promise<Integer>();
-    }
-
-    EntryBuffer getRecordSet() {
-        return recordSet;
-    }
-
-    Promise<Integer> getTransmitFuture() {
-        return transmitComplete;
-    }
-
-    /**
-     * Complete the transmit with result code <code>transmitRc</code>.
-     * <p>It would notify all the waiters that are waiting via {@link #awaitTransmitComplete(long, TimeUnit)}
-     * or {@link #addTransmitCompleteListener(FutureEventListener)}.
-     *
-     * @param transmitResult
-     *          transmit result code.
-     */
-    public void notifyTransmitComplete(int transmitResult) {
-        transmitComplete.setValue(transmitResult);
-    }
-
-    /**
-     * Register a transmit complete listener.
-     * <p>The listener will be triggered with transmit result when transmit completes.
-     * The method should be non-blocking.
-     *
-     * @param transmitCompleteListener
-     *          listener on transmit completion
-     * @see #awaitTransmitComplete(long, TimeUnit)
-     */
-    void addTransmitCompleteListener(FutureEventListener<Integer> transmitCompleteListener) {
-        transmitComplete.addEventListener(transmitCompleteListener);
-    }
-
-    /**
-     * Await for the transmit to be complete
-     *
-     * @param timeout
-     *          wait timeout
-     * @param unit
-     *          wait timeout unit
-     */
-    int awaitTransmitComplete(long timeout, TimeUnit unit)
-        throws Exception {
-        return Await.result(transmitComplete,
-                Duration.fromTimeUnit(timeout, unit));
-    }
-
-    public long getTransmitTime() {
-        return transmitTime;
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java
deleted file mode 100644
index 8d3c418..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java
+++ /dev/null
@@ -1,289 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.ZooKeeperClient.Credentials;
-import com.twitter.distributedlog.ZooKeeperClient.DigestCredentials;
-import com.twitter.distributedlog.exceptions.AlreadyClosedException;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.net.NetUtils;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Throw;
-import org.apache.bookkeeper.client.AsyncCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy;
-import org.apache.bookkeeper.conf.ClientConfiguration;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.net.DNSToSwitchMapping;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.zookeeper.KeeperException;
-import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * BookKeeper Client wrapper over {@link BookKeeper}.
- *
- * <h3>Metrics</h3>
- * <ul>
- * <li> bookkeeper operation stats are exposed under current scope by {@link BookKeeper}
- * </ul>
- */
-public class BookKeeperClient {
-    static final Logger LOG = LoggerFactory.getLogger(BookKeeperClient.class);
-
-    // Parameters to build bookkeeper client
-    private final DistributedLogConfiguration conf;
-    private final String name;
-    private final String zkServers;
-    private final String ledgersPath;
-    private final byte[] passwd;
-    private final ClientSocketChannelFactory channelFactory;
-    private final HashedWheelTimer requestTimer;
-    private final StatsLogger statsLogger;
-
-    // bookkeeper client state
-    private boolean closed = false;
-    private BookKeeper bkc = null;
-    private ZooKeeperClient zkc;
-    private final boolean ownZK;
-    // feature provider
-    private final Optional<FeatureProvider> featureProvider;
-
-    @SuppressWarnings("deprecation")
-    private synchronized void commonInitialization(
-            DistributedLogConfiguration conf, String ledgersPath,
-            ClientSocketChannelFactory channelFactory, StatsLogger statsLogger, HashedWheelTimer requestTimer)
-        throws IOException, InterruptedException, KeeperException {
-        ClientConfiguration bkConfig = new ClientConfiguration();
-        bkConfig.setAddEntryTimeout(conf.getBKClientWriteTimeout());
-        bkConfig.setReadTimeout(conf.getBKClientReadTimeout());
-        bkConfig.setZkLedgersRootPath(ledgersPath);
-        bkConfig.setZkTimeout(conf.getBKClientZKSessionTimeoutMilliSeconds());
-        bkConfig.setNumWorkerThreads(conf.getBKClientNumberWorkerThreads());
-        bkConfig.setEnsemblePlacementPolicy(RegionAwareEnsemblePlacementPolicy.class);
-        bkConfig.setZkRequestRateLimit(conf.getBKClientZKRequestRateLimit());
-        bkConfig.setProperty(RegionAwareEnsemblePlacementPolicy.REPP_DISALLOW_BOOKIE_PLACEMENT_IN_REGION_FEATURE_NAME,
-                DistributedLogConstants.DISALLOW_PLACEMENT_IN_REGION_FEATURE_NAME);
-        // reload configuration from dl configuration with settings prefixed with 'bkc.'
-        ConfUtils.loadConfiguration(bkConfig, conf, "bkc.");
-
-        Class<? extends DNSToSwitchMapping> dnsResolverCls;
-        try {
-            dnsResolverCls = conf.getEnsemblePlacementDnsResolverClass();
-        } catch (ConfigurationException e) {
-            LOG.error("Failed to load bk dns resolver : ", e);
-            throw new IOException("Failed to load bk dns resolver : ", e);
-        }
-        final DNSToSwitchMapping dnsResolver =
-                NetUtils.getDNSResolver(dnsResolverCls, conf.getBkDNSResolverOverrides());
-
-        this.bkc = BookKeeper.newBuilder()
-            .config(bkConfig)
-            .zk(zkc.get())
-            .channelFactory(channelFactory)
-            .statsLogger(statsLogger)
-            .dnsResolver(dnsResolver)
-            .requestTimer(requestTimer)
-            .featureProvider(featureProvider.orNull())
-            .build();
-    }
-
-    BookKeeperClient(DistributedLogConfiguration conf,
-                     String name,
-                     String zkServers,
-                     ZooKeeperClient zkc,
-                     String ledgersPath,
-                     ClientSocketChannelFactory channelFactory,
-                     HashedWheelTimer requestTimer,
-                     StatsLogger statsLogger,
-                     Optional<FeatureProvider> featureProvider) {
-        this.conf = conf;
-        this.name = name;
-        this.zkServers = zkServers;
-        this.ledgersPath = ledgersPath;
-        this.passwd = conf.getBKDigestPW().getBytes(UTF_8);
-        this.channelFactory = channelFactory;
-        this.requestTimer = requestTimer;
-        this.statsLogger = statsLogger;
-        this.featureProvider = featureProvider;
-        this.ownZK = null == zkc;
-        if (null != zkc) {
-            // reference the passing zookeeper client
-            this.zkc = zkc;
-        }
-    }
-
-    private synchronized void initialize() throws IOException {
-        if (null != this.bkc) {
-            return;
-        }
-        if (null == this.zkc) {
-            int zkSessionTimeout = conf.getBKClientZKSessionTimeoutMilliSeconds();
-            RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
-                        conf.getBKClientZKRetryBackoffStartMillis(),
-                        conf.getBKClientZKRetryBackoffMaxMillis(), conf.getBKClientZKNumRetries());
-            Credentials credentials = Credentials.NONE;
-            if (conf.getZkAclId() != null) {
-                credentials = new DigestCredentials(conf.getZkAclId(), conf.getZkAclId());
-            }
-
-            this.zkc = new ZooKeeperClient(name + ":zk", zkSessionTimeout, 2 * zkSessionTimeout, zkServers,
-                                           retryPolicy, statsLogger.scope("bkc_zkc"), conf.getZKClientNumberRetryThreads(),
-                                           conf.getBKClientZKRequestRateLimit(), credentials);
-        }
-
-        try {
-            commonInitialization(conf, ledgersPath, channelFactory, statsLogger, requestTimer);
-        } catch (InterruptedException e) {
-            throw new DLInterruptedException("Interrupted on creating bookkeeper client " + name + " : ", e);
-        } catch (KeeperException e) {
-            throw new ZKException("Error on creating bookkeeper client " + name + " : ", e);
-        }
-
-        if (ownZK) {
-            LOG.info("BookKeeper Client created {} with its own ZK Client : ledgersPath = {}, numRetries = {}, " +
-                    "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}",
-                    new Object[] { name, ledgersPath,
-                    conf.getBKClientZKNumRetries(), conf.getBKClientZKSessionTimeoutMilliSeconds(),
-                    conf.getBKClientZKRetryBackoffStartMillis(), conf.getBKClientZKRetryBackoffMaxMillis(),
-                    conf.getBkDNSResolverOverrides() });
-        } else {
-            LOG.info("BookKeeper Client created {} with shared zookeeper client : ledgersPath = {}, numRetries = {}, " +
-                    "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}",
-                    new Object[] { name, ledgersPath,
-                    conf.getZKNumRetries(), conf.getZKSessionTimeoutMilliseconds(),
-                    conf.getZKRetryBackoffStartMillis(), conf.getZKRetryBackoffMaxMillis(),
-                    conf.getBkDNSResolverOverrides() });
-        }
-    }
-
-
-    public synchronized BookKeeper get() throws IOException {
-        checkClosedOrInError();
-        if (null == bkc) {
-            initialize();
-        }
-        return bkc;
-    }
-
-    // Util functions
-    public Future<LedgerHandle> createLedger(int ensembleSize,
-                                             int writeQuorumSize,
-                                             int ackQuorumSize) {
-        BookKeeper bk;
-        try {
-            bk = get();
-        } catch (IOException ioe) {
-            return Future.exception(ioe);
-        }
-        final Promise<LedgerHandle> promise = new Promise<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));
-                        } else {
-                            promise.updateIfEmpty(new Throw<LedgerHandle>(BKException.create(rc)));
-                        }
-                    }
-                }, null);
-        return promise;
-    }
-
-    public Future<Void> deleteLedger(long lid,
-                                     final boolean ignoreNonExistentLedger) {
-        BookKeeper bk;
-        try {
-            bk = get();
-        } catch (IOException ioe) {
-            return Future.exception(ioe);
-        }
-        final Promise<Void> promise = new Promise<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));
-                } else if (BKException.Code.NoSuchLedgerExistsException == rc) {
-                    if (ignoreNonExistentLedger) {
-                        promise.updateIfEmpty(new Return<Void>(null));
-                    } else {
-                        promise.updateIfEmpty(new Throw<Void>(BKException.create(rc)));
-                    }
-                } else {
-                    promise.updateIfEmpty(new Throw<Void>(BKException.create(rc)));
-                }
-            }
-        }, null);
-        return promise;
-    }
-
-    public void close() {
-        BookKeeper bkcToClose;
-        ZooKeeperClient zkcToClose;
-        synchronized (this) {
-            if (closed) {
-                return;
-            }
-            closed = true;
-            bkcToClose = bkc;
-            zkcToClose = zkc;
-        }
-
-        LOG.info("BookKeeper Client closed {}", name);
-        if (null != bkcToClose) {
-            try {
-                bkcToClose.close();
-            } catch (InterruptedException e) {
-                LOG.warn("Interrupted on closing bookkeeper client {} : ", name, e);
-                Thread.currentThread().interrupt();
-            } catch (BKException e) {
-                LOG.warn("Error on closing bookkeeper client {} : ", name, e);
-            }
-        }
-        if (null != zkcToClose) {
-            if (ownZK) {
-                zkcToClose.close();
-            }
-        }
-    }
-
-    public synchronized void checkClosedOrInError() throws AlreadyClosedException {
-        if (closed) {
-            LOG.error("BookKeeper Client {} is already closed", name);
-            throw new AlreadyClosedException("BookKeeper Client " + name + " is already closed");
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClientBuilder.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClientBuilder.java
deleted file mode 100644
index cad1096..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClientBuilder.java
+++ /dev/null
@@ -1,209 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.apache.bookkeeper.feature.FeatureProvider;
-
-import org.apache.bookkeeper.feature.Feature;
-
-/**
- * Builder to build bookkeeper client.
- */
-public class BookKeeperClientBuilder {
-
-    /**
-     * Create a bookkeeper client builder to build bookkeeper clients.
-     *
-     * @return bookkeeper client builder.
-     */
-    public static BookKeeperClientBuilder newBuilder() {
-        return new BookKeeperClientBuilder();
-    }
-
-    // client name
-    private String name = null;
-    // dl config
-    private DistributedLogConfiguration dlConfig = null;
-    // bookkeeper settings
-    // zookeeper client
-    private ZooKeeperClient zkc = null;
-    // or zookeeper servers
-    private String zkServers = null;
-    // ledgers path
-    private String ledgersPath = null;
-    // statsLogger
-    private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
-    // client channel factory
-    private ClientSocketChannelFactory channelFactory = null;
-    // request timer
-    private HashedWheelTimer requestTimer = null;
-    // feature provider
-    private Optional<FeatureProvider> featureProvider = Optional.absent();
-
-    // Cached BookKeeper Client
-    private BookKeeperClient cachedClient = null;
-
-    /**
-     * Private bookkeeper builder.
-     */
-    private BookKeeperClientBuilder() {}
-
-    /**
-     * Set client name.
-     *
-     * @param name
-     *          client name.
-     * @return builder
-     */
-    public synchronized BookKeeperClientBuilder name(String name) {
-        this.name = name;
-        return this;
-    }
-
-    /**
-     * <i>dlConfig</i> used to configure bookkeeper client.
-     *
-     * @param dlConfig
-     *          distributedlog config.
-     * @return builder.
-     */
-    public synchronized BookKeeperClientBuilder dlConfig(DistributedLogConfiguration dlConfig) {
-        this.dlConfig = dlConfig;
-        return this;
-    }
-
-    /**
-     * Set the zkc used to build bookkeeper client. If a zookeeper client is provided in this
-     * method, bookkeeper client will use it rather than creating a brand new one.
-     *
-     * @param zkc
-     *          zookeeper client.
-     * @return builder
-     * @see #zkServers(String)
-     */
-    public synchronized BookKeeperClientBuilder zkc(ZooKeeperClient zkc) {
-        this.zkc = zkc;
-        return this;
-    }
-
-    /**
-     * Set the zookeeper servers that bookkeeper client would connect to. If no zookeeper client
-     * is provided by {@link #zkc(ZooKeeperClient)}, bookkeeper client will use the given string
-     * to create a brand new zookeeper client.
-     *
-     * @param zkServers
-     *          zookeeper servers that bookkeeper client would connect to.
-     * @return builder
-     * @see #zkc(ZooKeeperClient)
-     */
-    public synchronized BookKeeperClientBuilder zkServers(String zkServers) {
-        this.zkServers = zkServers;
-        return this;
-    }
-
-    /**
-     * Set the ledgers path that bookkeeper client is going to access.
-     *
-     * @param ledgersPath
-     *          ledgers path
-     * @return builder
-     * @see org.apache.bookkeeper.conf.ClientConfiguration#getZkLedgersRootPath()
-     */
-    public synchronized BookKeeperClientBuilder ledgersPath(String ledgersPath) {
-        this.ledgersPath = ledgersPath;
-        return this;
-    }
-
-    /**
-     * Build BookKeeper client using existing <i>bkc</i> client.
-     *
-     * @param bkc
-     *          bookkeeper client.
-     * @return builder
-     */
-    public synchronized BookKeeperClientBuilder bkc(BookKeeperClient bkc) {
-        this.cachedClient = bkc;
-        return this;
-    }
-
-    /**
-     * Build BookKeeper client using existing <i>channelFactory</i>.
-     *
-     * @param channelFactory
-     *          Channel Factory used to build bookkeeper client.
-     * @return bookkeeper client builder.
-     */
-    public synchronized BookKeeperClientBuilder channelFactory(ClientSocketChannelFactory channelFactory) {
-        this.channelFactory = channelFactory;
-        return this;
-    }
-
-    /**
-     * Build BookKeeper client using existing <i>request timer</i>.
-     *
-     * @param requestTimer
-     *          HashedWheelTimer used to build bookkeeper client.
-     * @return bookkeeper client builder.
-     */
-    public synchronized BookKeeperClientBuilder requestTimer(HashedWheelTimer requestTimer) {
-        this.requestTimer = requestTimer;
-        return this;
-    }
-
-    /**
-     * Build BookKeeper Client using given stats logger <i>statsLogger</i>.
-     *
-     * @param statsLogger
-     *          stats logger to report stats
-     * @return builder.
-     */
-    public synchronized BookKeeperClientBuilder statsLogger(StatsLogger statsLogger) {
-        this.statsLogger = statsLogger;
-        return this;
-    }
-
-    public synchronized BookKeeperClientBuilder featureProvider(Optional<FeatureProvider> featureProvider) {
-        this.featureProvider = featureProvider;
-        return this;
-    }
-
-    private void validateParameters() {
-        Preconditions.checkNotNull(name, "Missing client name.");
-        Preconditions.checkNotNull(dlConfig, "Missing DistributedLog Configuration.");
-        Preconditions.checkArgument(null == zkc || null == zkServers, "Missing zookeeper setting.");
-        Preconditions.checkNotNull(ledgersPath, "Missing Ledgers Root Path.");
-    }
-
-    public synchronized BookKeeperClient build() {
-        if (null == cachedClient) {
-            cachedClient = buildClient();
-        }
-        return cachedClient;
-    }
-
-    private BookKeeperClient buildClient() {
-        validateParameters();
-        return new BookKeeperClient(dlConfig, name, zkServers, zkc, ledgersPath, channelFactory, requestTimer, statsLogger, featureProvider);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java
deleted file mode 100644
index 6da4b8d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java
+++ /dev/null
@@ -1,3528 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.bk.QuorumConfig;
-import com.twitter.distributedlog.feature.DefaultFeatureProvider;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.net.DNSResolverForRacks;
-import com.twitter.distributedlog.net.DNSResolverForRows;
-import org.apache.bookkeeper.conf.ClientConfiguration;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.net.DNSToSwitchMapping;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.configuration.CompositeConfiguration;
-import org.apache.commons.configuration.Configuration;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.PropertiesConfiguration;
-import org.apache.commons.configuration.SystemConfiguration;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URL;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.Set;
-
-/**
- * DistributedLog Configuration.
- * <p>
- * DistributedLog configuration is basically a properties based configuration, which extends from
- * Apache commons {@link CompositeConfiguration}. All the DL settings are in camel case and prefixed
- * with a meaningful component name. for example, `zkSessionTimeoutSeconds` means <i>SessionTimeoutSeconds</i>
- * for component `zk`.
- *
- * <h3>BookKeeper Configuration</h3>
- *
- * BookKeeper client configuration settings could be loaded via DistributedLog configuration. All those
- * settings are prefixed with <i>`bkc.`</i>. For example, <i>bkc.zkTimeout</i> in distributedlog configuration
- * will be applied as <i>`zkTimeout`</i> in bookkeeper client configuration.
- *
- * <h3>How to load configuration</h3>
- *
- * The default distributedlog configuration is constructed by instantiated a new instance. This
- * distributedlog configuration will automatically load the settings that specified via
- * {@link SystemConfiguration}.
- *
- * <pre>
- *      DistributedLogConfiguration conf = new DistributedLogConfiguration();
- * </pre>
- *
- * The recommended way is to load configuration from URL that points to a configuration file
- * ({@link #loadConf(URL)}).
- *
- * <pre>
- *      String configFile = "/path/to/distributedlog/conf/file";
- *      DistributedLogConfiguration conf = new DistributedLogConfiguration();
- *      conf.loadConf(new File(configFile).toURI().toURL());
- * </pre>
- *
- * @see org.apache.bookkeeper.conf.ClientConfiguration
- */
-public class DistributedLogConfiguration extends CompositeConfiguration {
-    static final Logger LOG = LoggerFactory.getLogger(DistributedLogConfiguration.class);
-
-    private static ClassLoader defaultLoader;
-
-    static {
-        defaultLoader = Thread.currentThread().getContextClassLoader();
-        if (null == defaultLoader) {
-            defaultLoader = DistributedLogConfiguration.class.getClassLoader();
-        }
-    }
-
-    //
-    // ZooKeeper Related Settings
-    //
-
-    public static final String BKDL_ZK_ACL_ID = "zkAclId";
-    public static final String BKDL_ZK_ACL_ID_DEFAULT = null;
-    public static final String BKDL_ZK_SESSION_TIMEOUT_SECONDS = "zkSessionTimeoutSeconds";
-    public static final int BKDL_ZK_SESSION_TIMEOUT_SECONDS_DEFAULT = 30;
-    public static final String BKDL_ZK_REQUEST_RATE_LIMIT = "zkRequestRateLimit";
-    public static final double BKDL_ZK_REQUEST_RATE_LIMIT_DEFAULT = 0;
-    public static final String BKDL_ZK_NUM_RETRIES = "zkNumRetries";
-    public static final int BKDL_ZK_NUM_RETRIES_DEFAULT = 3;
-    public static final String BKDL_ZK_RETRY_BACKOFF_START_MILLIS = "zkRetryStartBackoffMillis";
-    public static final int BKDL_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT = 5000;
-    public static final String BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS = "zkRetryMaxBackoffMillis";
-    public static final int BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT = 30000;
-    public static final String BKDL_ZKCLIENT_NUM_RETRY_THREADS = "zkcNumRetryThreads";
-    public static final int BKDL_ZKCLIENT_NUM_RETRY_THREADS_DEFAULT = 1;
-
-    //
-    // BookKeeper Related Settings
-    //
-
-    // BookKeeper zookeeper settings
-    public static final String BKDL_BKCLIENT_ZK_SESSION_TIMEOUT = "bkcZKSessionTimeoutSeconds";
-    public static final int BKDL_BKCLIENT_ZK_SESSION_TIMEOUT_DEFAULT = 30;
-    public static final String BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT = "bkcZKRequestRateLimit";
-    public static final double BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT_DEFAULT = 0;
-    public static final String BKDL_BKCLIENT_ZK_NUM_RETRIES = "bkcZKNumRetries";
-    public static final int BKDL_BKCLIENT_ZK_NUM_RETRIES_DEFAULT = 3;
-    public static final String BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS = "bkcZKRetryStartBackoffMillis";
-    public static final int BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT = 5000;
-    public static final String BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS = "bkcZKRetryMaxBackoffMillis";
-    public static final int BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT = 30000;
-
-    // Bookkeeper ensemble placement settings
-    // Bookkeeper ensemble size
-    public static final String BKDL_BOOKKEEPER_ENSEMBLE_SIZE = "bkcEnsembleSize";
-    // @Deprecated
-    public static final String BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD = "ensemble-size";
-    public static final int BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT = 3;
-    // Bookkeeper write quorum size
-    public static final String BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE = "bkcWriteQuorumSize";
-    // @Deprecated
-    public static final String BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD = "write-quorum-size";
-    public static final int BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT = 3;
-    // Bookkeeper ack quorum size
-    public static final String BKDL_BOOKKEEPER_ACK_QUORUM_SIZE = "bkcAckQuorumSize";
-    // @Deprecated
-    public static final String BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD = "ack-quorum-size";
-    public static final int BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT = 2;
-    public static final String BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT = "bkRowAwareEnsemblePlacement";
-    public static final String BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_OLD = "row-aware-ensemble-placement";
-    public static final boolean BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_DEFAULT = false;
-    public static final String BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS = "bkEnsemblePlacementDnsResolverClass";
-    public static final String BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS_DEFAULT =
-            DNSResolverForRacks.class.getName();
-    public static final String BKDL_BK_DNS_RESOLVER_OVERRIDES = "dnsResolverOverrides";
-    public static final String BKDL_BK_DNS_RESOLVER_OVERRIDES_DEFAULT = "";
-
-    // General Settings
-    // @Deprecated
-    public static final String BKDL_BOOKKEEPER_DIGEST_PW = "digestPw";
-    public static final String BKDL_BOOKKEEPER_DIGEST_PW_DEFAULT = "";
-    public static final String BKDL_BKCLIENT_NUM_IO_THREADS = "bkcNumIOThreads";
-    public static final String BKDL_TIMEOUT_TIMER_TICK_DURATION_MS = "timerTickDuration";
-    public static final long BKDL_TIMEOUT_TIMER_TICK_DURATION_MS_DEFAULT = 100;
-    public static final String BKDL_TIMEOUT_TIMER_NUM_TICKS = "timerNumTicks";
-    public static final int BKDL_TIMEOUT_TIMER_NUM_TICKS_DEFAULT = 1024;
-
-    //
-    // Deprecated BookKeeper Settings (in favor of "bkc." style bookkeeper settings)
-    //
-
-    public static final String BKDL_BKCLIENT_READ_TIMEOUT = "bkcReadTimeoutSeconds";
-    public static final int BKDL_BKCLIENT_READ_TIMEOUT_DEFAULT = 10;
-    public static final String BKDL_BKCLIENT_WRITE_TIMEOUT = "bkcWriteTimeoutSeconds";
-    public static final int BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT = 10;
-    public static final String BKDL_BKCLIENT_NUM_WORKER_THREADS = "bkcNumWorkerThreads";
-    public static final int BKDL_BKCLEINT_NUM_WORKER_THREADS_DEFAULT = 1;
-
-    //
-    // DL General Settings
-    //
-
-    // Executor Parameters
-    public static final String BKDL_NUM_WORKER_THREADS = "numWorkerThreads";
-    public static final String BKDL_NUM_READAHEAD_WORKER_THREADS = "numReadAheadWorkerThreads";
-    public static final String BKDL_NUM_LOCKSTATE_THREADS = "numLockStateThreads";
-    public static final String BKDL_NUM_RESOURCE_RELEASE_THREADS = "numResourceReleaseThreads";
-    public static final String BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS = "schedulerShutdownTimeoutMs";
-    public static final int BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS_DEFAULT = 5000;
-    public static final String BKDL_USE_DAEMON_THREAD = "useDaemonThread";
-    public static final boolean BKDL_USE_DAEMON_THREAD_DEFAULT = false;
-
-    // Metadata Parameters
-    public static final String BKDL_LEDGER_METADATA_LAYOUT_VERSION = "ledgerMetadataLayoutVersion";
-    public static final String BKDL_LEDGER_METADATA_LAYOUT_VERSION_OLD = "ledger-metadata-layout";
-    public static final int BKDL_LEDGER_METADATA_LAYOUT_VERSION_DEFAULT =
-            LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value;
-    public static final String BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK = "ledgerMetadataSkipMinVersionCheck";
-    public static final boolean BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK_DEFAULT = false;
-    public static final String BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER = "firstLogsegmentSequenceNumber";
-    public static final String BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_OLD = "first-logsegment-sequence-number";
-    public static final long BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_DEFAULT =
-            DistributedLogConstants.FIRST_LOGSEGMENT_SEQNO;
-    public static final String BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED = "logSegmentSequenceNumberValidationEnabled";
-    public static final boolean BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED_DEFAULT = true;
-    public static final String BKDL_ENABLE_RECORD_COUNTS = "enableRecordCounts";
-    public static final boolean BKDL_ENABLE_RECORD_COUNTS_DEFAULT = true;
-    public static final String BKDL_MAXID_SANITYCHECK = "maxIdSanityCheck";
-    public static final boolean BKDL_MAXID_SANITYCHECK_DEFAULT = true;
-    public static final String BKDL_ENCODE_REGION_ID_IN_VERSION = "encodeRegionIDInVersion";
-    public static final boolean BKDL_ENCODE_REGION_ID_IN_VERSION_DEFAULT = false;
-    // (@Deprecated)
-    public static final String BKDL_LOGSEGMENT_NAME_VERSION = "logSegmentNameVersion";
-    public static final int BKDL_LOGSEGMENT_NAME_VERSION_DEFAULT = DistributedLogConstants.LOGSEGMENT_NAME_VERSION;
-    // (@Derepcated) Name for the default (non-partitioned) stream
-    public static final String BKDL_UNPARTITIONED_STREAM_NAME = "unpartitionedStreamName";
-    public static final String BKDL_UNPARTITIONED_STREAM_NAME_DEFAULT = "<default>";
-
-    // Log Segment Cache Parameters
-    public static final String BKDL_LOGSEGMENT_CACHE_TTL_MS = "logSegmentCacheTTLMs";
-    public static final long BKDL_LOGSEGMENT_CACHE_TTL_MS_DEFAULT = 600000; // 10 mins
-    public static final String BKDL_LOGSEGMENT_CACHE_MAX_SIZE = "logSegmentCacheMaxSize";
-    public static final long BKDL_LOGSEGMENT_CACHE_MAX_SIZE_DEFAULT = 10000;
-    public static final String BKDL_LOGSEGMENT_CACHE_ENABLED = "logSegmentCacheEnabled";
-    public static final boolean BKDL_LOGSEGMENT_CACHE_ENABLED_DEFAULT = true;
-
-    //
-    // DL Writer Settings
-    //
-
-    // General Settings
-    public static final String BKDL_CREATE_STREAM_IF_NOT_EXISTS = "createStreamIfNotExists";
-    public static final boolean BKDL_CREATE_STREAM_IF_NOT_EXISTS_DEFAULT = true;
-    public static final String BKDL_LOG_FLUSH_TIMEOUT = "logFlushTimeoutSeconds";
-    public static final int BKDL_LOG_FLUSH_TIMEOUT_DEFAULT = 30;
-    /**
-     *  CompressionCodec.Type     String to use (See CompressionUtils)
-     *  ---------------------     ------------------------------------
-     *          NONE               none
-     *          LZ4                lz4
-     *          UNKNOWN            any other instance of String.class
-     */
-    public static final String BKDL_COMPRESSION_TYPE = "compressionType";
-    public static final String BKDL_COMPRESSION_TYPE_DEFAULT = "none";
-    public static final String BKDL_FAILFAST_ON_STREAM_NOT_READY = "failFastOnStreamNotReady";
-    public static final boolean BKDL_FAILFAST_ON_STREAM_NOT_READY_DEFAULT = false;
-    public static final String BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR = "disableRollingOnLogSegmentError";
-    public static final boolean BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR_DEFAULT = false;
-
-    // Durability Settings
-    public static final String BKDL_IS_DURABLE_WRITE_ENABLED = "isDurableWriteEnabled";
-    public static final boolean BKDL_IS_DURABLE_WRITE_ENABLED_DEFAULT = true;
-
-    // Transmit Settings
-    public static final String BKDL_OUTPUT_BUFFER_SIZE = "writerOutputBufferSize";
-    public static final String BKDL_OUTPUT_BUFFER_SIZE_OLD = "output-buffer-size";
-    public static final int BKDL_OUTPUT_BUFFER_SIZE_DEFAULT = 1024;
-    public static final String BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS = "periodicFlushFrequencyMilliSeconds";
-    public static final int BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT = 0;
-    public static final String BKDL_ENABLE_IMMEDIATE_FLUSH = "enableImmediateFlush";
-    public static final boolean BKDL_ENABLE_IMMEDIATE_FLUSH_DEFAULT = false;
-    public static final String BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS = "minimumDelayBetweenImmediateFlushMilliSeconds";
-    public static final int BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS_DEFAULT = 0;
-    public static final String BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS = "periodicKeepAliveMilliSeconds";
-    public static final int BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS_DEFAULT = 0;
-
-    // Retention/Truncation Settings
-    public static final String BKDL_RETENTION_PERIOD_IN_HOURS = "logSegmentRetentionHours";
-    public static final String BKDL_RETENTION_PERIOD_IN_HOURS_OLD = "retention-size";
-    public static final int BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT = 72;
-    public static final String BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION = "explicitTruncationByApp";
-    public static final boolean BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION_DEFAULT = false;
-
-    // Log Segment Rolling Settings
-    public static final String BKDL_ROLLING_INTERVAL_IN_MINUTES = "logSegmentRollingMinutes";
-    public static final String BKDL_ROLLING_INTERVAL_IN_MINUTES_OLD = "rolling-interval";
-    public static final int BKDL_ROLLING_INTERVAL_IN_MINUTES_DEFAULT = 120;
-    public static final String BKDL_MAX_LOGSEGMENT_BYTES = "maxLogSegmentBytes";
-    public static final int BKDL_MAX_LOGSEGMENT_BYTES_DEFAULT = 256 * 1024 * 1024; // default 256MB
-    public static final String BKDL_LOGSEGMENT_ROLLING_CONCURRENCY = "logSegmentRollingConcurrency";
-    public static final int BKDL_LOGSEGMENT_ROLLING_CONCURRENCY_DEFAULT = 1;
-
-    // Lock Settings
-    public static final String BKDL_WRITE_LOCK_ENABLED = "writeLockEnabled";
-    public static final boolean BKDL_WRITE_LOCK_ENABLED_DEFAULT = true;
-    public static final String BKDL_LOCK_TIMEOUT = "lockTimeoutSeconds";
-    public static final long BKDL_LOCK_TIMEOUT_DEFAULT = 30;
-    public static final String BKDL_LOCK_REACQUIRE_TIMEOUT = "lockReacquireTimeoutSeconds";
-    public static final long BKDL_LOCK_REACQUIRE_TIMEOUT_DEFAULT = DistributedLogConstants.LOCK_REACQUIRE_TIMEOUT_DEFAULT;
-    public static final String BKDL_LOCK_OP_TIMEOUT = "lockOpTimeoutSeconds";
-    public static final long BKDL_LOCK_OP_TIMEOUT_DEFAULT = DistributedLogConstants.LOCK_OP_TIMEOUT_DEFAULT;
-
-    // Ledger Allocator Settings
-    public static final String BKDL_ENABLE_LEDGER_ALLOCATOR_POOL = "enableLedgerAllocatorPool";
-    public static final boolean BKDL_ENABLE_LEDGER_ALLOCATOR_POOL_DEFAULT = false;
-    public static final String BKDL_LEDGER_ALLOCATOR_POOL_PATH = "ledgerAllocatorPoolPath";
-    public static final String BKDL_LEDGER_ALLOCATOR_POOL_PATH_DEFAULT = DistributedLogConstants.ALLOCATION_POOL_NODE;
-    public static final String BKDL_LEDGER_ALLOCATOR_POOL_NAME = "ledgerAllocatorPoolName";
-    public static final String BKDL_LEDGER_ALLOCATOR_POOL_NAME_DEFAULT = null;
-    public static final String BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE = "ledgerAllocatorPoolCoreSize";
-    public static final int BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE_DEFAULT = 20;
-
-    // Write Limit Settings
-    public static final String BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT = "perWriterOutstandingWriteLimit";
-    public static final int BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT_DEFAULT = -1;
-    public static final String BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT = "globalOutstandingWriteLimit";
-    public static final int BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT_DEFAULT = -1;
-    public static final String BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE = "outstandingWriteLimitDarkmode";
-    public static final boolean BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE_DEFAULT = true;
-
-    //
-    // DL Reader Settings
-    //
-
-    // General Settings
-    public static final String BKDL_READLAC_OPTION = "readLACLongPoll";
-    public static final int BKDL_READLAC_OPTION_DEFAULT = 3; //BKLogPartitionReadHandler.ReadLACOption.READENTRYPIGGYBACK_SEQUENTIAL.value
-    public static final String BKDL_READLACLONGPOLL_TIMEOUT = "readLACLongPollTimeout";
-    public static final int BKDL_READLACLONGPOLL_TIMEOUT_DEFAULT = 1000;
-    public static final String BKDL_DESERIALIZE_RECORDSET_ON_READS = "deserializeRecordSetOnReads";
-    public static final boolean BKDL_DESERIALIZE_RECORDSET_ON_READS_DEFAULT = true;
-
-    // Idle reader settings
-    public static final String BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS = "readerIdleWarnThresholdMillis";
-    public static final int BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS_DEFAULT = 120000;
-    public static final String BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS = "readerIdleErrorThresholdMillis";
-    public static final int BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT = Integer.MAX_VALUE;
-
-    // Reader constraint settings
-    public static final String BKDL_READER_IGNORE_TRUNCATION_STATUS = "ignoreTruncationStatus";
-    public static final boolean BKDL_READER_IGNORE_TRUNCATION_STATUS_DEFAULT = false;
-    public static final String BKDL_READER_ALERT_POSITION_ON_TRUNCATED = "alertPositionOnTruncated";
-    public static final boolean BKDL_READER_ALERT_POSITION_ON_TRUNCATED_DEFAULT = true;
-    public static final String BKDL_READER_POSITION_GAP_DETECTION_ENABLED = "positionGapDetectionEnabled";
-    public static final boolean BKDL_READER_POSITION_GAP_DETECTION_ENABLED_DEFAULT = false;
-
-    // Read ahead related parameters
-    public static final String BKDL_ENABLE_READAHEAD = "enableReadAhead";
-    public static final boolean BKDL_ENABLE_READAHEAD_DEFAULT = true;
-    public static final String BKDL_ENABLE_FORCEREAD = "enableForceRead";
-    public static final boolean BKDL_ENABLE_FORCEREAD_DEFAULT = true;
-    public static final String BKDL_READAHEAD_MAX_RECORDS = "readAheadMaxRecords";
-    public static final String BKDL_READAHEAD_MAX_RECORDS_OLD = "ReadAheadMaxEntries";
-    public static final int BKDL_READAHEAD_MAX_RECORDS_DEFAULT = 10;
-    public static final String BKDL_READAHEAD_BATCHSIZE = "readAheadBatchSize";
-    public static final String BKDL_READAHEAD_BATCHSIZE_OLD = "ReadAheadBatchSize";
-    public static final int BKDL_READAHEAD_BATCHSIZE_DEFAULT = 2;
-    public static final String BKDL_READAHEAD_WAITTIME = "readAheadWaitTime";
-    public static final String BKDL_READAHEAD_WAITTIME_OLD = "ReadAheadWaitTime";
-    public static final int BKDL_READAHEAD_WAITTIME_DEFAULT = 200;
-    public static final String BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM = "readAheadWaitTimeOnEndOfStream";
-    public static final String BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM_OLD = "ReadAheadWaitTimeOnEndOfStream";
-    public static final int BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM_DEFAULT = 10000;
-    public static final String BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS =
-            "readAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis";
-    public static final int BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS_DEFAULT = 10000;
-    public static final String BKDL_READAHEAD_SKIP_BROKEN_ENTRIES = "readAheadSkipBrokenEntries";
-    public static final boolean BKDL_READAHEAD_SKIP_BROKEN_ENTRIES_DEFAULT = false;
-    public static final String BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT = "numPrefetchEntriesPerLogSegment";
-    public static final int BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT = 4;
-    public static final String BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT = "maxPrefetchEntriesPerLogSegment";
-    public static final int BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT = 32;
-
-    // Scan Settings
-    public static final String BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN = "firstNumEntriesEachPerLastRecordScan";
-    public static final int BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT = 2;
-    public static final String BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN = "maxNumEntriesPerReadLastRecordScan";
-    public static final int BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT = 16;
-
-    // Log Existence Settings
-    public static final String BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS = "checkLogExistenceBackoffStartMillis";
-    public static final int BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS_DEFAULT = 200;
-    public static final String BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS = "checkLogExistenceBackoffMaxMillis";
-    public static final int BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS_DEFAULT = 1000;
-
-    //
-    // Tracing/Stats Settings
-    //
-
-    public static final String BKDL_TRACE_READAHEAD_DELIVERY_LATENCY = "traceReadAheadDeliveryLatency";
-    public static final boolean BKDL_TRACE_READAHEAD_DELIVERY_LATENCY_DEFAULT = false;
-    public static final String BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS = "metadataLatencyWarnThresholdMs";
-    public static final long BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT = DistributedLogConstants.LATENCY_WARN_THRESHOLD_IN_MILLIS;
-    public static final String BKDL_DATA_LATENCY_WARN_THRESHOLD_MS = "dataLatencyWarnThresholdMs";
-    public static final long BKDL_DATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT = 2 * DistributedLogConstants.LATENCY_WARN_THRESHOLD_IN_MILLIS;
-    public static final String BKDL_TRACE_READAHEAD_METADATA_CHANGES = "traceReadAheadMetadataChanges";
-    public static final boolean BKDL_TRACE_READAHEAD_MEATDATA_CHANGES_DEFAULT = false;
-    public final static String BKDL_ENABLE_TASK_EXECUTION_STATS = "enableTaskExecutionStats";
-    public final static boolean BKDL_ENABLE_TASK_EXECUTION_STATS_DEFAULT = false;
-    public final static String BKDL_TASK_EXECUTION_WARN_TIME_MICROS = "taskExecutionWarnTimeMicros";
-    public final static long BKDL_TASK_EXECUTION_WARN_TIME_MICROS_DEFAULT = 100000;
-    public static final String BKDL_ENABLE_PERSTREAM_STAT = "enablePerStreamStat";
-    public static final boolean BKDL_ENABLE_PERSTREAM_STAT_DEFAULT = false;
-
-    //
-    // Settings for Feature Providers
-    //
-
-    public static final String BKDL_FEATURE_PROVIDER_CLASS = "featureProviderClass";
-
-    //
-    // Settings for Configuration Based Feature Provider
-    //
-
-    public static final String BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH = "fileFeatureProviderBaseConfigPath";
-    public static final String BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH_DEFAULT = "decider.conf";
-    public static final String BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH = "fileFeatureProviderOverlayConfigPath";
-    public static final String BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH_DEFAULT = null;
-
-    //
-    // Settings for Namespaces
-    //
-
-    public static final String BKDL_FEDERATED_NAMESPACE_ENABLED = "federatedNamespaceEnabled";
-    public static final boolean BKDL_FEDERATED_NAMESPACE_ENABLED_DEFAULT = false;
-    public static final String BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE = "federatedMaxLogsPerSubnamespace";
-    public static final int BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE_DEFAULT = 15000;
-    public static final String BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS = "federatedCheckExistenceWhenCacheMiss";
-    public static final boolean BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS_DEFAULT = true;
-
-    // Settings for Configurations
-
-    public static final String BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC = "dynamicConfigReloadIntervalSec";
-    public static final int BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC_DEFAULT = 60;
-    public static final String BKDL_STREAM_CONFIG_ROUTER_CLASS = "streamConfigRouterClass";
-    public static final String BKDL_STREAM_CONFIG_ROUTER_CLASS_DEFAULT = "com.twitter.distributedlog.service.config.IdentityConfigRouter";
-
-    // Settings for RateLimit (used by distributedlog-service)
-
-    public static final String BKDL_BPS_SOFT_WRITE_LIMIT = "bpsSoftWriteLimit";
-    public static final int BKDL_BPS_SOFT_WRITE_LIMIT_DEFAULT = -1;
-    public static final String BKDL_BPS_HARD_WRITE_LIMIT = "bpsHardWriteLimit";
-    public static final int BKDL_BPS_HARD_WRITE_LIMIT_DEFAULT = -1;
-    public static final String BKDL_RPS_SOFT_WRITE_LIMIT = "rpsSoftWriteLimit";
-    public static final int BKDL_RPS_SOFT_WRITE_LIMIT_DEFAULT = -1;
-    public static final String BKDL_RPS_HARD_WRITE_LIMIT = "rpsHardWriteLimit";
-    public static final int BKDL_RPS_HARD_WRITE_LIMIT_DEFAULT = -1;
-
-    // Rate and resource limits: per shard
-
-    public static final String BKDL_RPS_SOFT_SERVICE_LIMIT = "rpsSoftServiceLimit";
-    public static final int BKDL_RPS_SOFT_SERVICE_LIMIT_DEFAULT = -1;
-    public static final String BKDL_RPS_HARD_SERVICE_LIMIT = "rpsHardServiceLimit";
-    public static final int BKDL_RPS_HARD_SERVICE_LIMIT_DEFAULT = -1;
-    public static final String BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT = "rpsStreamAcquireServiceLimit";
-    public static final int BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT = -1;
-    public static final String BKDL_BPS_SOFT_SERVICE_LIMIT = "bpsSoftServiceLimit";
-    public static final int BKDL_BPS_SOFT_SERVICE_LIMIT_DEFAULT = -1;
-    public static final String BKDL_BPS_HARD_SERVICE_LIMIT = "bpsHardServiceLimit";
-    public static final int BKDL_BPS_HARD_SERVICE_LIMIT_DEFAULT = -1;
-    public static final String BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT = "bpsStreamAcquireServiceLimit";
-    public static final int BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT = -1;
-
-    // Settings for Partitioning
-
-    public static final String BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY = "maxAcquiredPartitionsPerProxy";
-    public static final int BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY_DEFAULT = -1;
-
-    public static final String BKDL_MAX_CACHED_PARTITIONS_PER_PROXY = "maxCachedPartitionsPerProxy";
-    public static final int BKDL_MAX_CACHED_PARTITIONS_PER_PROXY_DEFAULT = -1;
-
-    //
-    // Settings for Error Injection
-    //
-    public static final String BKDL_EI_INJECT_WRITE_DELAY = "eiInjectWriteDelay";
-    public static final boolean BKDL_EI_INJECT_WRITE_DELAY_DEFAULT = false;
-    public static final String BKDL_EI_INJECTED_WRITE_DELAY_PERCENT = "eiInjectedWriteDelayPercent";
-    public static final double BKDL_EI_INJECTED_WRITE_DELAY_PERCENT_DEFAULT = 0.0;
-    public static final String BKDL_EI_INJECTED_WRITE_DELAY_MS = "eiInjectedWriteDelayMs";
-    public static final int BKDL_EI_INJECTED_WRITE_DELAY_MS_DEFAULT = 0;
-    public static final String BKDL_EI_INJECT_READAHEAD_STALL = "eiInjectReadAheadStall";
-    public static final boolean BKDL_EI_INJECT_READAHEAD_STALL_DEFAULT = false;
-    public static final String BKDL_EI_INJECT_READAHEAD_DELAY = "eiInjectReadAheadDelay";
-    public static final boolean BKDL_EI_INJECT_READAHEAD_DELAY_DEFAULT = false;
-    public static final String BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS = "eiInjectMaxReadAheadDelayMs";
-    public static final int BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS_DEFAULT = 0;
-    public static final String BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT = "eiInjectReadAheadDelayPercent";
-    public static final int BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT_DEFAULT = 10;
-    public static final String BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES = "eiInjectReadAheadBrokenEntries";
-    public static final boolean BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES_DEFAULT = false;
-
-    // Whitelisted stream-level configuration settings.
-    private static final Set<String> streamSettings = Sets.newHashSet(
-        BKDL_READER_POSITION_GAP_DETECTION_ENABLED,
-        BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS,
-        BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS,
-        BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
-        BKDL_ENABLE_IMMEDIATE_FLUSH
-    );
-
-    /**
-     * Construct distributedlog configuration with default settings.
-     * It also loads the settings from system properties.
-     */
-    public DistributedLogConfiguration() {
-        super();
-        // add configuration for system properties
-        addConfiguration(new SystemConfiguration());
-    }
-
-    /**
-     * You can load configurations in precedence order. The first one takes
-     * precedence over any loaded later.
-     *
-     * @param confURL Configuration URL
-     */
-    public void loadConf(URL confURL) throws ConfigurationException {
-        Configuration loadedConf = new PropertiesConfiguration(confURL);
-        addConfiguration(loadedConf);
-    }
-
-    /**
-     * You can load configuration from other configuration
-     *
-     * @param baseConf Other Configuration
-     */
-    public void loadConf(DistributedLogConfiguration baseConf) {
-        addConfiguration(baseConf);
-    }
-
-    /**
-     * Load configuration from other configuration object
-     *
-     * @param otherConf Other configuration object
-     */
-    public void loadConf(Configuration otherConf) {
-        addConfiguration(otherConf);
-    }
-
-    /**
-     * Load whitelisted stream configuration from another configuration object
-     *
-     * @param streamConfiguration stream configuration overrides
-     */
-    public void loadStreamConf(Optional<DistributedLogConfiguration> streamConfiguration) {
-        if (!streamConfiguration.isPresent()) {
-            return;
-        }
-        ArrayList<Object> ignoredSettings = new ArrayList<Object>();
-        Iterator iterator = streamConfiguration.get().getKeys();
-        while (iterator.hasNext()) {
-            Object setting = iterator.next();
-            if (setting instanceof String && streamSettings.contains(setting)) {
-                String settingStr = (String) setting;
-                setProperty(settingStr, streamConfiguration.get().getProperty(settingStr));
-            } else {
-                ignoredSettings.add(setting);
-            }
-        }
-        if (LOG.isWarnEnabled() && !ignoredSettings.isEmpty()) {
-            LOG.warn("invalid stream configuration override(s): {}",
-                StringUtils.join(ignoredSettings, ";"));
-        }
-    }
-
-    //
-    // ZooKeeper Related Settings
-    //
-
-    /**
-     * Get all properties as a string.
-     */
-    public String getPropsAsString() {
-        Iterator iterator = getKeys();
-        StringBuilder builder = new StringBuilder();
-        boolean appendNewline = false;
-        while (iterator.hasNext()) {
-            Object key = iterator.next();
-            if (key instanceof String) {
-                if (appendNewline) {
-                    builder.append("\n");
-                }
-                Object value = getProperty((String)key);
-                builder.append(key).append("=").append(value);
-                appendNewline = true;
-            }
-        }
-        return builder.toString();
-    }
-
-    /**
-     * Get digest id used for ZK acl.
-     *
-     * @return zk acl id.
-     */
-    public String getZkAclId() {
-        return getString(BKDL_ZK_ACL_ID, BKDL_ZK_ACL_ID_DEFAULT);
-    }
-
-    /**
-     * Set digest id to use for ZK acl.
-     *
-     * @param zkAclId acl id.
-     * @return distributedlog configuration
-     * @see #getZkAclId()
-     */
-    public DistributedLogConfiguration setZkAclId(String zkAclId) {
-        setProperty(BKDL_ZK_ACL_ID, zkAclId);
-        return this;
-    }
-
-    /**
-     * Get ZK Session timeout in seconds.
-     * <p>
-     * This is the session timeout applied for zookeeper client used by distributedlog.
-     * Use {@link #getBKClientZKSessionTimeoutMilliSeconds()} for zookeeper client used
-     * by bookkeeper client.
-     *
-     * @return zookeeeper session timeout in seconds.
-     * @deprecated use {@link #getZKSessionTimeoutMilliseconds()}
-     */
-    public int getZKSessionTimeoutSeconds() {
-        return this.getInt(BKDL_ZK_SESSION_TIMEOUT_SECONDS, BKDL_ZK_SESSION_TIMEOUT_SECONDS_DEFAULT);
-    }
-
-    /**
-     * Get ZK Session timeout in milliseconds.
-     * <p>
-     * This is the session timeout applied for zookeeper client used by distributedlog.
-     * Use {@link #getBKClientZKSessionTimeoutMilliSeconds()} for zookeeper client used
-     * by bookkeeper client.
-     *
-     * @return zk session timeout in milliseconds.
-     */
-    public int getZKSessionTimeoutMilliseconds() {
-        return this.getInt(BKDL_ZK_SESSION_TIMEOUT_SECONDS, BKDL_ZK_SESSION_TIMEOUT_SECONDS_DEFAULT) * 1000;
-    }
-
-    /**
-     * Set ZK Session Timeout in seconds.
-     *
-     * @param zkSessionTimeoutSeconds session timeout in seconds.
-     * @return distributed log configuration
-     * @see #getZKSessionTimeoutMilliseconds()
-     */
-    public DistributedLogConfiguration setZKSessionTimeoutSeconds(int zkSessionTimeoutSeconds) {
-        setProperty(BKDL_ZK_SESSION_TIMEOUT_SECONDS, zkSessionTimeoutSeconds);
-        return this;
-    }
-
-    /**
-     * Get zookeeper access rate limit.
-     * <p>The rate limiter is basically a guava {@link com.google.common.util.concurrent.RateLimiter}.
-     * It is rate limiting the requests that sent by zookeeper client. If the value is non-positive,
-     * the rate limiting is disable. By default it is disable (value = 0).
-     *
-     * @return zookeeper access rate, by default it is 0.
-     */
-    public double getZKRequestRateLimit() {
-        return this.getDouble(BKDL_ZK_REQUEST_RATE_LIMIT, BKDL_ZK_REQUEST_RATE_LIMIT_DEFAULT);
-    }
-
-    /**
-     * Set zookeeper access rate limit (rps).
-     *
-     * @param requestRateLimit
-     *          zookeeper access rate limit
-     * @return distributedlog configuration
-     * @see #getZKRequestRateLimit()
-     */
-    public DistributedLogConfiguration setZKRequestRateLimit(double requestRateLimit) {
-        setProperty(BKDL_ZK_REQUEST_RATE_LIMIT, requestRateLimit);
-        return this;
-    }
-
-    /**
-     * Get num of retries per request for zookeeper client.
-     * <p>Retries only happen on retryable failures like session expired,
-     * session moved. for permanent failures, the request will fail immediately.
-     * The default value is 3.
-     *
-     * @return num of retries per request of zookeeper client.
-     */
-    public int getZKNumRetries() {
-        return this.getInt(BKDL_ZK_NUM_RETRIES, BKDL_ZK_NUM_RETRIES_DEFAULT);
-    }
-
-    /**
-     * Set num of retries per request for zookeeper client.
-     *
-     * @param zkNumRetries num of retries per request of zookeeper client.
-     * @return distributed log configuration
-     * @see #getZKNumRetries()
-     */
-    public DistributedLogConfiguration setZKNumRetries(int zkNumRetries) {
-        setProperty(BKDL_ZK_NUM_RETRIES, zkNumRetries);
-        return this;
-    }
-
-    /**
-     * Get the start backoff time of zookeeper operation retries, in milliseconds.
-     * <p>The retry time will increase in bound exponential way, and become flat
-     * after hit max backoff time ({@link #getZKRetryBackoffMaxMillis()}).
-     * The default start backoff time is 5000 milliseconds.
-     *
-     * @return start backoff time of zookeeper operation retries, in milliseconds.
-     * @see #getZKRetryBackoffMaxMillis()
-     */
-    public int getZKRetryBackoffStartMillis() {
-        return this.getInt(BKDL_ZK_RETRY_BACKOFF_START_MILLIS,
-                           BKDL_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT);
-    }
-
-    /**
-     * Set the start backoff time of zookeeper operation retries, in milliseconds.
-     *
-     * @param zkRetryBackoffStartMillis start backoff time of zookeeper operation retries,
-     *                                  in milliseconds.
-     * @return distributed log configuration
-     * @see #getZKRetryBackoffStartMillis()
-     */
-    public DistributedLogConfiguration setZKRetryBackoffStartMillis(int zkRetryBackoffStartMillis) {
-        setProperty(BKDL_ZK_RETRY_BACKOFF_START_MILLIS, zkRetryBackoffStartMillis);
-        return this;
-    }
-
-    /**
-     * Get the max backoff time of zookeeper operation retries, in milliseconds.
-     * <p>The retry time will increase in bound exponential way starting from
-     * {@link #getZKRetryBackoffStartMillis()}, and become flat after hit this max
-     * backoff time.
-     * The default max backoff time is 30000 milliseconds.
-     *
-     * @return max backoff time of zookeeper operation retries, in milliseconds.
-     * @see #getZKRetryBackoffStartMillis()
-     */
-    public int getZKRetryBackoffMaxMillis() {
-        return this.getInt(BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS,
-                           BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT);
-    }
-
-    /**
-     * Set the max backoff time of zookeeper operation retries, in milliseconds.
-     *
-     * @param zkRetryBackoffMaxMillis max backoff time of zookeeper operation retries,
-     *                                in milliseconds.
-     * @return distributed log configuration
-     * @see #getZKRetryBackoffMaxMillis()
-     */
-    public DistributedLogConfiguration setZKRetryBackoffMaxMillis(int zkRetryBackoffMaxMillis) {
-        setProperty(BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS, zkRetryBackoffMaxMillis);
-        return this;
-    }
-
-    /**
-     * Get ZK client number of retry executor threads.
-     * By default it is 1.
-     *
-     * @return number of bookkeeper client worker threads.
-     */
-    public int getZKClientNumberRetryThreads() {
-        return this.getInt(BKDL_ZKCLIENT_NUM_RETRY_THREADS, BKDL_ZKCLIENT_NUM_RETRY_THREADS_DEFAULT);
-    }
-
-    /**
-     * Set ZK client number of retry executor threads.
-     *
-     * @param numThreads
-     *          number of retry executor threads.
-     * @return distributedlog configuration.
-     * @see #getZKClientNumberRetryThreads()
-     */
-    public DistributedLogConfiguration setZKClientNumberRetryThreads(int numThreads) {
-        setProperty(BKDL_ZKCLIENT_NUM_RETRY_THREADS, numThreads);
-        return this;
-    }
-
-    //
-    // BookKeeper ZooKeeper Client Settings
-    //
-
-    /**
-     * Get BK's zookeeper session timout in milliseconds.
-     * <p>
-     * This is the session timeout applied for zookeeper client used by bookkeeper client.
-     * Use {@link #getZKSessionTimeoutMilliseconds()} for zookeeper client used
-     * by distributedlog.
-     *
-     * @return Bk's zookeeper session timeout in milliseconds
-     */
-    public int getBKClientZKSessionTimeoutMilliSeconds() {
-        return this.getInt(BKDL_BKCLIENT_ZK_SESSION_TIMEOUT, BKDL_BKCLIENT_ZK_SESSION_TIMEOUT_DEFAULT) * 1000;
-    }
-
-    /**
-     * Set BK's zookeeper session timeout in seconds.
-     *
-     * @param sessionTimeout session timeout for the ZK Client used by BK Client, in seconds.
-     * @return distributed log configuration
-     * @see #getBKClientZKSessionTimeoutMilliSeconds()
-     */
-    public DistributedLogConfiguration setBKClientZKSessionTimeout(int sessionTimeout) {
-        setProperty(BKDL_BKCLIENT_ZK_SESSION_TIMEOUT, sessionTimeout);
-        return this;
-    }
-
-    /**
-     * Get zookeeper access rate limit for zookeeper client used in bookkeeper client.
-     * <p>The rate limiter is basically a guava {@link com.google.common.util.concurrent.RateLimiter}.
-     * It is rate limiting the requests that sent by zookeeper client. If the value is non-positive,
-     * the rate limiting is disable. By default it is disable (value = 0).
-     *
-     * @return zookeeper access rate limit for zookeeper client used in bookkeeper client.
-     * By default it is 0.
-     */
-    public double getBKClientZKRequestRateLimit() {
-        return this.getDouble(BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT,
-                BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT_DEFAULT);
-    }
-
-    /**
-     * Set zookeeper access rate limit for zookeeper client used in bookkeeper client.
-     *
-     * @param rateLimit
-     *          zookeeper access rate limit
-     * @return distributedlog configuration.
-     * @see #getBKClientZKRequestRateLimit()
-     */
-    public DistributedLogConfiguration setBKClientZKRequestRateLimit(double rateLimit) {
-        setProperty(BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT, rateLimit);
-        return this;
-    }
-
-    /**
-     * Get num of retries for zookeeper client that used by bookkeeper client.
-     * <p>Retries only happen on retryable failures like session expired,
-     * session moved. for permanent failures, the request will fail immediately.
-     * The default value is 3. Setting it to zero or negative will retry infinitely.
-     *
-     * @return num of retries of zookeeper client used by bookkeeper client.
-     */
-    public int getBKClientZKNumRetries() {
-        int zkNumRetries = this.getInt(BKDL_BKCLIENT_ZK_NUM_RETRIES, BKDL_BKCLIENT_ZK_NUM_RETRIES_DEFAULT);
-        if (zkNumRetries <= 0) {
-            return Integer.MAX_VALUE;
-        }
-        return zkNumRetries;
-    }
-
-    /**
-     * Get the start backoff time of zookeeper operation retries, in milliseconds.
-     * <p>The retry time will increase in bound exponential way, and become flat
-     * after hit max backoff time ({@link #getBKClientZKRetryBackoffMaxMillis()}.
-     * The default start backoff time is 5000 milliseconds.
-     *
-     * @return start backoff time of zookeeper operation retries, in milliseconds.
-     * @see #getBKClientZKRetryBackoffMaxMillis()
-     */
-    public int getBKClientZKRetryBackoffStartMillis() {
-        return this.getInt(BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS,
-                           BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT);
-    }
-
-    /**
-     * Get the max backoff time of zookeeper operation retries, in milliseconds.
-     * <p>The retry time will increase in bound exponential way starting from
-     * {@link #getBKClientZKRetryBackoffStartMillis()}, and become flat after
-     * hit this max backoff time.
-     * The default max backoff time is 30000 milliseconds.
-     *
-     * @return max backoff time of zookeeper operation retries, in milliseconds.
-     * @see #getBKClientZKRetryBackoffStartMillis()
-     */
-    public int getBKClientZKRetryBackoffMaxMillis() {
-        return this.getInt(BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS,
-                BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT);
-    }
-
-    //
-    // BookKeeper Ensemble Placement Settings
-    //
-
-    /**
-     * Get ensemble size of each log segment (ledger) will use.
-     * By default it is 3.
-     * <p>
-     * A log segment's data is stored in an ensemble of bookies in
-     * a stripping way. Each entry will be added in a <code>write-quorum</code>
-     * size of bookies. The add operation will complete once it receives
-     * responses from a <code>ack-quorum</code> size of bookies. The stripping
-     * is done in a round-robin way in bookkeeper.
-     * <p>
-     * For example, we configure the ensemble-size to 5, write-quorum-size to 3,
-     * and ack-quorum-size to 2. The data will be stored in following stripping way.
-     * <pre>
-     * | entry id | bk1 | bk2 | bk3 | bk4 | bk5 |
-     * |     0    |  x  |  x  |  x  |     |     |
-     * |     1    |     |  x  |  x  |  x  |     |
-     * |     2    |     |     |  x  |  x  |  x  |
-     * |     3    |  x  |     |     |  x  |  x  |
-     * |     4    |  x  |  x  |     |     |  x  |
-     * |     5    |  x  |  x  |  x  |     |     |
-     * </pre>
-     * <p>
-     * We don't recommend stripping within a log segment to increase bandwidth.
-     * We'd recommend to strip by `partition` in higher level of distributedlog
-     * to increase performance. so typically the ensemble size will set to be
-     * the same value as write quorum size.
-     *
-     * @return ensemble size
-     * @see #getWriteQuorumSize()
-     * @see #getAckQuorumSize()
-     */
-    public int getEnsembleSize() {
-        return this.getInt(BKDL_BOOKKEEPER_ENSEMBLE_SIZE,
-                getInt(BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD,
-                        BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT));
-    }
-
-    /**
-     * Set ensemble size of each log segment (ledger) will use.
-     *
-     * @param ensembleSize ensemble size.
-     * @return distributed log configuration
-     * @see #getEnsembleSize()
-     */
-    public DistributedLogConfiguration setEnsembleSize(int ensembleSize) {
-        setProperty(BKDL_BOOKKEEPER_ENSEMBLE_SIZE, ensembleSize);
-        return this;
-    }
-
-    /**
-     * Get write quorum size of each log segment (ledger) will use.
-     * By default it is 3.
-     *
-     * @return write quorum size
-     * @see #getEnsembleSize()
-     */
-    public int getWriteQuorumSize() {
-        return this.getInt(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
-                getInt(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
-                        BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT));
-    }
-
-    /**
-     * Set write quorum size of each log segment (ledger) will use.
-     *
-     * @param quorumSize
-     *          quorum size.
-     * @return distributedlog configuration.
-     * @see #getWriteQuorumSize()
-     */
-    public DistributedLogConfiguration setWriteQuorumSize(int quorumSize) {
-        setProperty(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE, quorumSize);
-        return this;
-    }
-
-    /**
-     * Get ack quorum size of each log segment (ledger) will use.
-     * By default it is 2.
-     *
-     * @return ack quorum size
-     * @see #getEnsembleSize()
-     */
-    public int getAckQuorumSize() {
-        return this.getInt(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
-                getInt(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
-                        BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT));
-    }
-
-    /**
-     * Set ack quorum size of each log segment (ledger) will use.
-     *
-     * @param quorumSize
-     *          quorum size.
-     * @return distributedlog configuration.
-     * @see #getAckQuorumSize()
-     */
-    public DistributedLogConfiguration setAckQuorumSize(int quorumSize) {
-        setProperty(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE, quorumSize);
-        return this;
-    }
-
-    /**
-     * Get the quorum config for each log segment (ledger).
-     *
-     * @return quorum config that used by log segments
-     * @see #getEnsembleSize()
-     * @see #getWriteQuorumSize()
-     * @see #getAckQuorumSize()
-     */
-    public QuorumConfig getQuorumConfig() {
-        return new QuorumConfig(
-                getEnsembleSize(),
-                getWriteQuorumSize(),
-                getAckQuorumSize());
-    }
-
-    /**
-     * Get if row aware ensemble placement is enabled.
-     * <p>If enabled, {@link DNSResolverForRows} will be used for dns resolution
-     * rather than {@link DNSResolverForRacks}, if no other dns resolver set via
-     * {@link #setEnsemblePlacementDnsResolverClass(Class)}.
-     * By default it is disable.
-     *
-     * @return true if row aware ensemble placement is enabled, otherwise false.
-     * @see #getEnsemblePlacementDnsResolverClass()
-     */
-    public boolean getRowAwareEnsemblePlacementEnabled() {
-        return getBoolean(BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT,
-                getBoolean(BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_OLD,
-                        BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_DEFAULT));
-    }
-
-    /**
-     * Set if we should enable row aware ensemble placement.
-     *
-     * @param enableRowAwareEnsemblePlacement
-     *          enableRowAwareEnsemblePlacement
-     * @return distributedlog configuration.
-     * @see #getRowAwareEnsemblePlacementEnabled()
-     */
-    public DistributedLogConfiguration setRowAwareEnsemblePlacementEnabled(boolean enableRowAwareEnsemblePlacement) {
-        setProperty(BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT, enableRowAwareEnsemblePlacement);
-        return this;
-    }
-
-    /**
-     * Get the DNS resolver class for bookkeeper ensemble placement.
-     * <p>By default, {@link DNSResolverForRacks} will be used if
-     * {@link #getRowAwareEnsemblePlacementEnabled()} is disabled and
-     * {@link DNSResolverForRows} will be used if {@link #getRowAwareEnsemblePlacementEnabled()}
-     * is enabled.
-     *
-     * @return dns resolver class for bookkeeper ensemble placement.
-     * @throws ConfigurationException
-     * @see #getRowAwareEnsemblePlacementEnabled()
-     */
-    public Class<? extends DNSToSwitchMapping> getEnsemblePlacementDnsResolverClass()
-            throws ConfigurationException {
-        Class<? extends DNSToSwitchMapping> defaultResolverCls;
-        if (getRowAwareEnsemblePlacementEnabled()) {
-            defaultResolverCls = DNSResolverForRows.class;
-        } else {
-            defaultResolverCls = DNSResolverForRacks.class;
-        }
-        return ReflectionUtils.getClass(this, BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS,
-                defaultResolverCls, DNSToSwitchMapping.class, defaultLoader);
-    }
-
-    /**
-     * Set the DNS resolver class for bookkeeper ensemble placement.
-     *
-     * @param dnsResolverClass
-     *          dns resolver class for bookkeeper ensemble placement.
-     * @return distributedlog configuration
-     * @see #getEnsemblePlacementDnsResolverClass()
-     */
-    public DistributedLogConfiguration setEnsemblePlacementDnsResolverClass(
-            Class<? extends DNSToSwitchMapping> dnsResolverClass) {
-        setProperty(BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS, dnsResolverClass.getName());
-        return this;
-    }
-
-    /**
-     * Get mapping used to override the region mapping derived by the default resolver.
-     * <p>It is a string of pairs of host-region mappings (host:region) separated by semicolon.
-     * By default it is empty string.
-     *
-     * @return dns resolver overrides.
-     * @see #getEnsemblePlacementDnsResolverClass()
-     * @see DNSResolverForRacks
-     * @see DNSResolverForRows
-     */
-    public String getBkDNSResolverOverrides() {
-        return getString(BKDL_BK_DNS_RESOLVER_OVERRIDES, BKDL_BK_DNS_RESOLVER_OVERRIDES_DEFAULT);
-    }
-
-    /**
-     * Set mapping used to override the region mapping derived by the default resolver
-     * <p>It is a string of pairs of host-region mappings (host:region) separated by semicolon.
-     * By default it is empty string.
-     *
-     * @param overrides
-     *          dns resolver overrides
-     * @return dl configuration.
-     * @see #getBkDNSResolverOverrides()
-     */
-    public DistributedLogConfiguration setBkDNSResolverOverrides(String overrides) {
-        setProperty(BKDL_BK_DNS_RESOLVER_OVERRIDES, overrides);
-        return this;
-    }
-
-    //
-    // BookKeeper General Settings
-    //
-
-    /**
-     * Set password used by bookkeeper client for digestion.
-     * <p>
-     * NOTE: not recommend to change. will be derepcated in future.
-     *
-     * @param bkDigestPW BK password digest
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setBKDigestPW(String bkDigestPW) {
-        setProperty(BKDL_BOOKKEEPER_DIGEST_PW, bkDigestPW);
-        return this;
-    }
-
-    /**
-     * Get password used by bookkeeper client for digestion.
-     * <p>
-     * NOTE: not recommend to change. will be deprecated in future.
-     *
-     * @return password used by bookkeeper client for digestion
-     * @see #setBKDigestPW(String)
-     */
-    public String getBKDigestPW() {
-        return getString(BKDL_BOOKKEEPER_DIGEST_PW, BKDL_BOOKKEEPER_DIGEST_PW_DEFAULT);
-    }
-
-    /**
-     * Get BK client number of i/o threads used by Netty.
-     * The default value equals DL's number worker threads.
-     *
-     * @return number of bookkeeper netty i/o threads.
-     * @see #getNumWorkerThreads()
-     */
-    public int getBKClientNumberIOThreads() {
-        return this.getInt(BKDL_BKCLIENT_NUM_IO_THREADS, getNumWorkerThreads());
-    }
-
-    /**
-     * Set BK client number of i/o threads used by netty.
-     *
-     * @param numThreads
-     *          number io threads.
-     * @return distributedlog configuration.
-     * @see #getBKClientNumberIOThreads()
-     */
-    public DistributedLogConfiguration setBKClientNumberIOThreads(int numThreads) {
-        setProperty(BKDL_BKCLIENT_NUM_IO_THREADS, numThreads);
-        return this;
-    }
-
-    /**
-     * Get the tick duration in milliseconds that used for timeout timer in bookkeeper client.
-     * By default it is 100.
-     *
-     * @return tick duration in milliseconds
-     * @see org.jboss.netty.util.HashedWheelTimer
-     */
-    public long getTimeoutTimerTickDurationMs() {
-        return getLong(BKDL_TIMEOUT_TIMER_TICK_DURATION_MS, BKDL_TIMEOUT_TIMER_TICK_DURATION_MS_DEFAULT);
-    }
-
-    /**
-     * Set the tick duration in milliseconds that used for timeout timer in bookkeeper client.
-     *
-     * @param tickDuration
-     *          tick duration in milliseconds.
-     * @return distributed log configuration.
-     * @see #getTimeoutTimerTickDurationMs()
-     */
-    public DistributedLogConfiguration setTimeoutTimerTickDurationMs(long tickDuration) {
-        setProperty(BKDL_TIMEOUT_TIMER_TICK_DURATION_MS, tickDuration);
-        return this;
-    }
-
-    /**
-     * Get number of ticks that used for timeout timer in bookkeeper client.
-     * By default is 1024.
-     *
-     * @return number of ticks that used for timeout timer.
-     * @see org.jboss.netty.util.HashedWheelTimer
-     */
-    public int getTimeoutTimerNumTicks() {
-        return getInt(BKDL_TIMEOUT_TIMER_NUM_TICKS, BKDL_TIMEOUT_TIMER_NUM_TICKS_DEFAULT);
-    }
-
-    /**
-     * Set number of ticks that used for timeout timer in bookkeeper client.
-     *
-     * @param numTicks
-     *          number of ticks that used for timeout timer.
-     * @return distributed log configuration.
-     * @see #getTimeoutTimerNumTicks()
-     */
-    public DistributedLogConfiguration setTimeoutTimerNumTicks(int numTicks) {
-        setProperty(BKDL_TIMEOUT_TIMER_NUM_TICKS, numTicks);
-        return this;
-    }
-
-    //
-    // Deprecated BookKeeper Settings
-    //
-
-    /**
-     * Get BK client read timeout in seconds.
-     * <p>
-     * Please use {@link ClientConfiguration#getReadEntryTimeout()}
-     * instead of this setting.
-     *
-     * @return read timeout in seconds
-     * @deprecated
-     * @see ClientConfiguration#getReadEntryTimeout()
-     */
-    public int getBKClientReadTimeout() {
-        return this.getInt(BKDL_BKCLIENT_READ_TIMEOUT,
-                BKDL_BKCLIENT_READ_TIMEOUT_DEFAULT);
-    }
-
-    /**
-     * Set BK client read timeout in seconds.
-     *
-     * @param readTimeout read timeout in seconds.
-     * @return distributed log configuration
-     * @deprecated
-     * @see #getBKClientReadTimeout()
-     */
-    public DistributedLogConfiguration setBKClientReadTimeout(int readTimeout) {
-        setProperty(BKDL_BKCLIENT_READ_TIMEOUT, readTimeout);
-        return this;
-    }
-
-    /**
-     * Get BK client write timeout in seconds.
-     * <p>
-     * Please use {@link ClientConfiguration#getAddEntryTimeout()}
-     * instead of this setting.
-     *
-     * @return write timeout in seconds.
-     * @deprecated
-     * @see ClientConfiguration#getAddEntryTimeout()
-     */
-    public int getBKClientWriteTimeout() {
-        return this.getInt(BKDL_BKCLIENT_WRITE_TIMEOUT, BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT);
-    }
-
-    /**
-     * Set BK client write timeout in seconds
-     *
-     * @param writeTimeout write timeout in seconds.
-     * @return distributed log configuration
-     * @deprecated
-     * @see #getBKClientWriteTimeout()
-     */
-    public DistributedLogConfiguration setBKClientWriteTimeout(int writeTimeout) {
-        setProperty(BKDL_BKCLIENT_WRITE_TIMEOUT, writeTimeout);
-        return this;
-    }
-
-    /**
-     * Get BK client number of worker threads.
-     * <p>
-     * Please use {@link ClientConfiguration#getNumWorkerThreads()}
-     * instead of this setting.
-     *
-     * @return number of bookkeeper client worker threads.
-     * @deprecated
-     * @see ClientConfiguration#getNumWorkerThreads()
-     */
-    public int getBKClientNumberWorkerThreads() {
-        return this.getInt(BKDL_BKCLIENT_NUM_WORKER_THREADS, BKDL_BKCLEINT_NUM_WORKER_THREADS_DEFAULT);
-    }
-
-    /**
-     * Set BK client number of worker threads.
-     *
-     * @param numThreads
-     *          number worker threads.
-     * @return distributedlog configuration.
-     * @deprecated
-     * @see #getBKClientNumberWorkerThreads()
-     */
-    public DistributedLogConfiguration setBKClientNumberWorkerThreads(int numThreads) {
-        setProperty(BKDL_BKCLIENT_NUM_WORKER_THREADS, numThreads);
-        return this;
-    }
-
-    //
-    // DL Executor Settings
-    //
-
-    /**
-     * Get the number of worker threads used by distributedlog namespace.
-     * By default it is the number of available processors.
-     *
-     * @return number of worker threads used by distributedlog namespace.
-     */
-    public int getNumWorkerThreads() {
-        return getInt(BKDL_NUM_WORKER_THREADS, Runtime.getRuntime().availableProcessors());
-    }
-
-    /**
-     * Set the number of worker threads used by distributedlog namespace.
-     *
-     * @param numWorkerThreads
-     *          number of worker threads used by distributedlog namespace.
-     * @return configuration
-     * @see #getNumWorkerThreads()
-     */
-    public DistributedLogConfiguration setNumWorkerThreads(int numWorkerThreads) {
-        setProperty(BKDL_NUM_WORKER_THREADS, numWorkerThreads);
-        return this;
-    }
-
-    /**
-     * Get the number of dedicated readahead worker threads used by distributedlog namespace.
-     * <p>If this value is non-positive, it would share the normal executor (see {@link #getNumWorkerThreads()}
-     * for readahead. otherwise, it would use a dedicated executor for readhead. By default,
-     * it is 0.
-     *
-     * @return number of dedicated readahead worker threads.
-     * @see #getNumWorkerThreads()
-     */
-    @Deprecated
-    public int getNumReadAheadWorkerThreads() {
-        return getInt(BKDL_NUM_READAHEAD_WORKER_THREADS, 0);
-    }
-
-    /**
-     * Set the number of dedicated readahead worker threads used by distributedlog namespace.
-     *
-     * @param numWorkerThreads
-     *          number of dedicated readahead worker threads.
-     * @return configuration
-     * @see #getNumReadAheadWorkerThreads()
-     */
-    @Deprecated
-    public DistributedLogConfiguration setNumReadAheadWorkerThreads(int numWorkerThreads) {
-        setProperty(BKDL_NUM_READAHEAD_WORKER_THREADS, numWorkerThreads);
-        return this;
-    }
-
-    /**
-     * Get the number of lock state threads used by distributedlog namespace.
-     * By default it is 1.
-     *
-     * @return number of lock state threads used by distributedlog namespace.
-     */
-    public int getNumLockStateThreads() {
-        return getInt(BKDL_NUM_LOCKSTATE_THREADS, 1);
-    }
-
-    /**
-     * Set the number of lock state threads used by distributedlog manager factory.
-     *
-     * @param numLockStateThreads
-     *          number of lock state threads used by distributedlog manager factory.
-     * @return configuration
-     * @see #getNumLockStateThreads()
-     */
-    public DistributedLogConfiguration setNumLockStateThreads(int numLockStateThreads) {
-        setProperty(BKDL_NUM_LOCKSTATE_THREADS, numLockStateThreads);
-        return this;
-    }
-
-    /**
-     * Get the number of resource release threads used by distributedlog namespace.
-     * By default it is 0 - the thread will be created dynamically by a executor service.
-     * The executor service is an unbounded pool. Application can use `total_tasks - completed_tasks`
-     * on monitoring the number of threads that are used for releasing resources.
-     * <p>
-     * The setting is only applied for v2 implementation.
-     *
-     * @see com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor
-     * @return number of resource release threads used by distributedlog namespace.
-     */
-    public int getNumResourceReleaseThreads() {
-        return getInt(BKDL_NUM_RESOURCE_RELEASE_THREADS, 0);
-    }
-
-    /**
-     * Set the number of resource release threads used by distributedlog manager factory.
-     *
-     * @param numResourceReleaseThreads
-     *          number of resource release threads used by distributedlog manager factory.
-     * @return configuration
-     * @see #getNumResourceReleaseThreads()
-     */
-    public DistributedLogConfiguration setNumResourceReleaseThreads(int numResourceReleaseThreads) {
-        setProperty(BKDL_NUM_RESOURCE_RELEASE_THREADS, numResourceReleaseThreads);
-        return this;
-    }
-
-    /**
-     * Get timeout for shutting down schedulers in dl manager, in milliseconds.
-     * By default, it is 5 seconds.
-     *
-     * @return timeout for shutting down schedulers in dl manager, in miliseconds.
-     */
-    public int getSchedulerShutdownTimeoutMs() {
-        return getInt(BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS, BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS_DEFAULT);
-    }
-
-    /**
-     * Set timeout for shutting down schedulers in dl manager, in milliseconds.
-     *
-     * @param timeoutMs
-     *         timeout for shutting down schedulers in dl manager, in milliseconds.
-     * @return dl configuration.
-     * @see #getSchedulerShutdownTimeoutMs()
-     */
-    public DistributedLogConfiguration setSchedulerShutdownTimeoutMs(int timeoutMs) {
-        setProperty(BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS, timeoutMs);
-        return this;
-    }
-
-    /**
-     * Whether to use daemon thread for DL threads.
-     * By default it is false.
-     *
-     * @return true if use daemon threads, otherwise false.
-     */
-    public boolean getUseDaemonThread() {
-        return getBoolean(BKDL_USE_DAEMON_THREAD, BKDL_USE_DAEMON_THREAD_DEFAULT);
-    }
-
-    /**
-     * Set whether to use daemon thread for DL threads.
-     *
-     * @param daemon
-     *          whether to use daemon thread for DL threads.
-     * @return distributedlog configuration
-     * @see #getUseDaemonThread()
-     */
-    public DistributedLogConfiguration setUseDaemonThread(boolean daemon) {
-        setProperty(BKDL_USE_DAEMON_THREAD, daemon);
-        return this;
-    }
-
-    //
-    // Metadata Settings
-    //
-
-    /**
-     * Get DL ledger metadata output layout version.
-     *
-     * @return layout version
-     * @see com.twitter.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion
-     */
-    public int getDLLedgerMetadataLayoutVersion() {
-        return this.getInt(BKDL_LEDGER_METADATA_LAYOUT_VERSION,
-                getInt(BKDL_LEDGER_METADATA_LAYOUT_VERSION_OLD,
-                        BKDL_LEDGER_METADATA_LAYOUT_VERSION_DEFAULT));
-    }
-
-    /**
-     * Set DL ledger metadata output layout version.
-     *
-     * @param layoutVersion layout version
-     * @return distributed log configuration
-     * @throws IllegalArgumentException if setting an unknown layout version.
-     * @see #getDLLedgerMetadataLayoutVersion()
-     */
-    public DistributedLogConfiguration setDLLedgerMetadataLayoutVersion(int layoutVersion)
-            throws IllegalArgumentException {
-        if ((layoutVersion <= 0) ||
-            (layoutVersion > LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION)) {
-            // Incorrect version specified
-            throw new IllegalArgumentException("Incorrect value for ledger metadata layout version");
-        }
-        setProperty(BKDL_LEDGER_METADATA_LAYOUT_VERSION, layoutVersion);
-        return this;
-    }
-
-    /**
-     * Get the setting for whether we should enforce the min ledger metadata version check.
-     * By default it is false.
-     *
-     * @return whether we should enforce the min ledger metadata version check
-     * @see com.twitter.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion
-     */
-    public boolean getDLLedgerMetadataSkipMinVersionCheck() {
-        return this.getBoolean(BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK,
-                BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK_DEFAULT);
-    }
-
-    /**
-     * Set if we should skip the enforcement of min ledger metadata version.
-     * <p>NOTE: please be aware the side effects of skipping min ledger metadata
-     * version checking.
-     *
-     * @param skipMinVersionCheck whether we should enforce the min ledger metadata version check
-     * @return distributed log configuration
-     * @see #getDLLedgerMetadataSkipMinVersionCheck()
-     */
-    public DistributedLogConfiguration setDLLedgerMetadataSkipMinVersionCheck(boolean skipMinVersionCheck) throws IllegalArgumentException {
-        setProperty(BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK, skipMinVersionCheck);
-        return this;
-    }
-
-    /**
-     * Get the value at which ledger sequence number should start for streams that are being
-     * upgraded and did not have ledger sequence number to start with or for newly created
-     * streams. By default, it is 1.
-     * <p>In most of the cases this value should not be changed. It is useful for backfilling
-     * in the case of migrating log segments whose metadata don't have log segment sequence number.
-     *
-     * @return first ledger sequence number
-     */
-    public long getFirstLogSegmentSequenceNumber() {
-        return this.getLong(BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER,
-                getLong(BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_OLD,
-                        BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_DEFAULT));
-    }
-
-    /**
-     * Set the value at which ledger sequence number should start for streams that are being
-     * upgraded and did not have ledger sequence number to start with or for newly created
-     * streams
-     *
-     * @param firstLogSegmentSequenceNumber first ledger sequence number
-     * @return distributed log configuration
-     * @see #getFirstLogSegmentSequenceNumber()
-     */
-    public DistributedLogConfiguration setFirstLogSegmentSequenceNumber(long firstLogSegmentSequenceNumber)
-            throws IllegalArgumentException {
-        if (firstLogSegmentSequenceNumber <= 0) {
-            // Incorrect ledger sequence number specified
-            throw new IllegalArgumentException("Incorrect value for ledger sequence number");
-        }
-        setProperty(BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER, firstLogSegmentSequenceNumber);
-        return this;
-    }
-
-    /**
-     * Whether log segment sequence number validation is enabled?
-     *
-     * @return true if the log segment sequence number validation is enabled, otherwise false.
-     */
-    public boolean isLogSegmentSequenceNumberValidationEnabled() {
-        return this.getBoolean(BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED,
-                BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED_DEFAULT);
-    }
-
-    /**
-     * Whether log segment sequence number validation is enabled?
-     *
-     * @return true if the log segment sequence number validation is enabled, otherwise false.
-     */
-    public DistributedLogConfiguration setLogSegmentSequenceNumberValidationEnabled(boolean enabled) {
-        setProperty(BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED, enabled);
-        return this;
-    }
-
-    /**
-     * Whether we should publish record counts in the log records and metadata.
-     * <p>By default it is true. This is a legacy setting for log segment version 1. It
-     * should be considered removed.
-     *
-     * @return if record counts should be persisted
-     */
-    public boolean getEnableRecordCounts() {
-        return getBoolean(BKDL_ENABLE_RECORD_COUNTS, BKDL_ENABLE_RECORD_COUNTS_DEFAULT);
-    }
-
-    /**
-     * Set if we should publish record counts in the log records and metadata.
-     *
-     * @param enableRecordCounts enable record counts
-     * @return distributed log configuration
-     * @see #getEnableRecordCounts()
-     */
-    public DistributedLogConfiguration setEnableRecordCounts(boolean enableRecordCounts) {
-        setProperty(BKDL_ENABLE_RECORD_COUNTS, enableRecordCounts);
-        return this;
-    }
-
-    /**
-     * Whether sanity check txn id on starting log segments.
-     * <p>If it is enabled, DL writer would throw
-     * {@link com.twitter.distributedlog.exceptions.TransactionIdOutOfOrderException}
-     * when it received a smaller transaction id than current maximum transaction id.
-     *
-     * @return true if should check txn id with max txn id, otherwise false.
-     */
-    @Deprecated
-    public boolean getSanityCheckTxnID() {
-        return getBoolean(BKDL_MAXID_SANITYCHECK, BKDL_MAXID_SANITYCHECK_DEFAULT);
-    }
-
-    /**
-     * Enable/Disable sanity check txn id.
-     *
-     * @param enabled
-     *          enable/disable sanity check txn id.
-     * @return configuration.
-     * @see #getSanityCheckTxnID()
-     */
-    @Deprecated
-    public DistributedLogConfiguration setSanityCheckTxnID(boolean enabled) {
-        setProperty(BKDL_MAXID_SANITYCHECK, enabled);
-        return this;
-    }
-
-    /**
-     * Whether encode region id in log segment metadata.
-     * <p>In global DL use case, encoding region id in log segment medata would
-     * help understanding what region that a log segment is created. The region
-     * id field in log segment metadata would help for moniotring and troubleshooting.
-     *
-     * @return whether to encode region id in log segment metadata.
-     */
-    public boolean getEncodeRegionIDInLogSegmentMetadata() {
-        return getBoolean(BKDL_ENCODE_REGION_ID_IN_VERSION, BKDL_ENCODE_REGION_ID_IN_VERSION_DEFAULT);
-    }
-
-    /**
-     * Enable/Disable encoding region id in log segment metadata.
-     *
-     * @param enabled
-     *          flag to enable/disable encoding region id in log segment metadata.
-     * @return configuration instance.
-     * @see #getEncodeRegionIDInLogSegmentMetadata()
-     */
-    public DistributedLogConfiguration setEncodeRegionIDInLogSegmentMetadata(boolean enabled) {
-        setProperty(BKDL_ENCODE_REGION_ID_IN_VERSION, enabled);
-        return this;
-    }
-
-    /**
-     * Get log segment name version.
-     * <p>
-     * <ul>
-     * <li>version 0: inprogress_(start_txid) |
-     * logrecs_(start_txid)_(end_txid)</li>
-     * <li>version 1: inprogress_(logsegment_sequence_number) |
-     * logrecs_(logsegment_sequence_number)</li>
-     * </ul>
-     * By default it is 1.
-     *
-     * @return log segment name verison.
-     */
-    public int getLogSegmentNameVersion() {
-        return getInt(BKDL_LOGSEGMENT_NAME_VERSION, BKDL_LOGSEGMENT_NAME_VERSION_DEFAULT);
-    }
-
-    /**
-     * Set log segment name version.
-     *
-     * @param version
-     *          log segment name version.
-     * @return configuration object.
-     * @see #getLogSegmentNameVersion()
-     */
-    public DistributedLogConfiguration setLogSegmentNameVersion(int version) {
-        setProperty(BKDL_LOGSEGMENT_NAME_VERSION, version);
-        return this;
-    }
-
-    /**
-     * Get name of the unpartitioned stream.
-     * <p>It is a legacy setting. consider removing it in future.
-     *
-     * @return unpartitioned stream
-     */
-    public String getUnpartitionedStreamName() {
-        return getString(BKDL_UNPARTITIONED_STREAM_NAME, BKDL_UNPARTITIONED_STREAM_NAME_DEFAULT);
-    }
-
-    /**
-     * Set name of the unpartitioned stream
-     *
-     * @param streamName name of the unpartitioned stream
-     * @return distributedlog configuration
-     * @see #getUnpartitionedStreamName()
-     */
-    public DistributedLogConfiguration setUnpartitionedStreamName(String streamName) {
-        setProperty(BKDL_UNPARTITIONED_STREAM_NAME, streamName);
-        return this;
-    }
-
-    //
-    // LogSegment Cache Settings
-    //
-
-    /**
-     * Get the log segment cache entry TTL in milliseconds.
-     *
-     * @return log segment cache ttl in milliseconds.
-     */
-    public long getLogSegmentCacheTTLMs() {
-        return getLong(BKDL_LOGSEGMENT_CACHE_TTL_MS, BKDL_LOGSEGMENT_CACHE_MAX_SIZE_DEFAULT);
-    }
-
-    /**
-     * Set the log segment cache entry TTL in milliseconds.
-     *
-     * @param ttlMs TTL in milliseconds
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setLogSegmentCacheTTLMs(long ttlMs) {
-        setProperty(BKDL_LOGSEGMENT_CACHE_TTL_MS, ttlMs);
-        return this;
-    }
-
-    /**
-     * Get the maximum size of the log segment cache.
-     *
-     * @return maximum size of the log segment cache.
-     */
-    public long getLogSegmentCacheMaxSize() {
-        return getLong(BKDL_LOGSEGMENT_CACHE_MAX_SIZE, BKDL_LOGSEGMENT_CACHE_MAX_SIZE_DEFAULT);
-    }
-
-    /**
-     * Set the maximum size of the log segment cache.
-     *
-     * @param maxSize maximum size of the log segment cache.
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setLogSegmentCacheMaxSize(long maxSize) {
-        setProperty(BKDL_LOGSEGMENT_CACHE_MAX_SIZE, maxSize);
-        return this;
-    }
-
-    /**
-     * Is log segment cache enabled?
-     *
-     * @return true if log segment cache is enabled; otherwise false
-     */
-    public boolean isLogSegmentCacheEnabled() {
-        return getBoolean(BKDL_LOGSEGMENT_CACHE_ENABLED, BKDL_LOGSEGMENT_CACHE_ENABLED_DEFAULT);
-    }
-
-    /**
-     * Enable/disable log segment cache.
-     *
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setLogSegmentCacheEnabled(boolean enabled) {
-        setProperty(BKDL_LOGSEGMENT_CACHE_ENABLED, enabled);
-        return this;
-    }
-
-    //
-    // DL Writer General Settings
-    //
-
-    /**
-     * Whether to create stream if not exists. By default it is true.
-     *
-     * @return true if it is abled to create stream if not exists.
-     */
-    public boolean getCreateStreamIfNotExists() {
-        return getBoolean(BKDL_CREATE_STREAM_IF_NOT_EXISTS,
-                BKDL_CREATE_STREAM_IF_NOT_EXISTS_DEFAULT);
-    }
-
-    /**
-     * Enable/Disable creating stream if not exists.
-     *
-     * @param enabled
-     *          enable/disable sanity check txn id.
-     * @return distributed log configuration.
-     * @see #getCreateStreamIfNotExists()
-     */
-    public DistributedLogConfiguration setCreateStreamIfNotExists(boolean enabled) {
-        setProperty(BKDL_CREATE_STREAM_IF_NOT_EXISTS, enabled);
-        return this;
-    }
-
-    /**
-     * Get Log Flush timeout in seconds.
-     * <p>This is a setting used by DL writer on flushing data. It is typically used
-     * by synchronous writer and log segment writer. By default it is 30 seconds.
-     *
-     * @return log flush timeout in seconds.
-     */
-    // @Deprecated
-    public int getLogFlushTimeoutSeconds() {
-        return this.getInt(BKDL_LOG_FLUSH_TIMEOUT, BKDL_LOG_FLUSH_TIMEOUT_DEFAULT);
-    }
-
-    /**
-     * Set Log Flush Timeout in seconds.
-     *
-     * @param logFlushTimeoutSeconds log flush timeout.
-     * @return distributed log configuration
-     * @see #getLogFlushTimeoutSeconds()
-     */
-    public DistributedLogConfiguration setLogFlushTimeoutSeconds(int logFlushTimeoutSeconds) {
-        setProperty(BKDL_LOG_FLUSH_TIMEOUT, logFlushTimeoutSeconds);
-        return this;
-    }
-
-    /**
-     * The compression type to use while sending data to bookkeeper.
-     *
-     * @return compression type to use
-     * @see com.twitter.distributedlog.io.CompressionCodec
-     */
-    public String getCompressionType() {
-        return getString(BKDL_COMPRESSION_TYPE, BKDL_COMPRESSION_TYPE_DEFAULT);
-    }
-
-    /**
-     * Set the compression type to use while sending data to bookkeeper.
-     *
-     * @param compressionType compression type
-     * @return distributedlog configuration
-     * @see #getCompressionType()
-     */
-    public DistributedLogConfiguration setCompressionType(String compressionType) {
-        Preconditions.checkArgument(null != compressionType && !compressionType.isEmpty());
-        setProperty(BKDL_COMPRESSION_TYPE, compressionType);
-        return this;
-    }
-
-    /**
-     * Whether to fail immediately if the stream is not ready rather than queueing the request.
-     * <p>If it is enabled, it would fail the write request immediately if the stream isn't ready.
-     * Consider turning it on for the use cases that could retry writing to other streams
-     * (aka non-strict ordering guarantee). It would result fast failure hence the client would
-     * retry immediately.
-     *
-     * @return true if should fail fast. otherwise, false.
-     */
-    public boolean getFailFastOnStreamNotReady() {
-        return getBoolean(BKDL_FAILFAST_ON_STREAM_NOT_READY,
-                BKDL_FAILFAST_ON_STREAM_NOT_READY_DEFAULT);
-    }
-
-    /**
-     * Set the failfast on stream not ready flag.
-     *
-     * @param failFastOnStreamNotReady
-     *        set failfast flag
-     * @return dl configuration.
-     * @see #getFailFastOnStreamNotReady()
-     */
-    public DistributedLogConfiguration setFailFastOnStreamNotReady(boolean failFastOnStreamNotReady) {
-        setProperty(BKDL_FAILFAST_ON_STREAM_NOT_READY, failFastOnStreamNotReady);
-        return this;
-    }
-
-    /**
-     * If this option is set, the log writer won't reset the segment writer if an error
-     * is encountered.
-     *
-     * @return true if we should disable automatic rolling
-     */
-    public boolean getDisableRollingOnLogSegmentError() {
-        return getBoolean(BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR,
-                BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR_DEFAULT);
-    }
-
-    /**
-     * Set the roll on segment error flag.
-     *
-     * @param disableRollingOnLogSegmentError
-     *        set roll on error flag
-     * @return dl configuration.
-     * @see #getDisableRollingOnLogSegmentError()
-     */
-    public DistributedLogConfiguration setDisableRollingOnLogSegmentError(boolean disableRollingOnLogSegmentError) {
-        setProperty(BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR, disableRollingOnLogSegmentError);
-        return this;
-    }
-
-    //
-    // DL Durability Settings
-    //
-
-    /**
-     * Check whether the durable write is enabled.
-     * <p>It is enabled by default.
-     *
-     * @return true if durable write is enabled. otherwise, false.
-     */
-    public boolean isDurableWriteEnabled() {
-        return this.getBoolean(BKDL_IS_DURABLE_WRITE_ENABLED, BKDL_IS_DURABLE_WRITE_ENABLED_DEFAULT);
-    }
-
-    /**
-     * Enable/Disable durable writes in writers.
-     *
-     * @param enabled
-     *          flag to enable/disable durable writes in writers.
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setDurableWriteEnabled(boolean enabled) {
-        setProperty(BKDL_IS_DURABLE_WRITE_ENABLED, enabled);
-        return this;
-    }
-
-    //
-    // DL Writer Transmit Settings
-    //
-
-    /**
-     * Get output buffer size for DL writers, in bytes.
-     * <p>Large buffer will result in higher compression ratio and
-     * it would use the bandwidth more efficiently and improve throughput.
-     * Set it to 0 would ask DL writers to transmit the data immediately,
-     * which it could achieve low latency.
-     * <p>The default value is 1KB.
-     *
-     * @return buffer size in byes.
-     */
-    public int getOutputBufferSize() {
-        return this.getInt(BKDL_OUTPUT_BUFFER_SIZE,
-                getInt(BKDL_OUTPUT_BUFFER_SIZE_OLD, BKDL_OUTPUT_BUFFER_SIZE_DEFAULT));
-    }
-
-    /**
-     * Set output buffer size for DL writers, in bytes.
-     *
-     * @param opBufferSize output buffer size.
-     * @return distributed log configuration
-     * @see #getOutputBufferSize()
-     */
-    public DistributedLogConfiguration setOutputBufferSize(int opBufferSize) {
-        setProperty(BKDL_OUTPUT_BUFFER_SIZE, opBufferSize);
-        return this;
-    }
-
-    /**
-     * Get Periodic Log Flush Frequency in milliseconds.
-     * <p>If the setting is set with a positive value, the data in output buffer
-     * will be flushed in this provided interval. The default value is 0.
-     *
-     * @return periodic flush frequency in milliseconds.
-     * @see #getOutputBufferSize()
-     */
-    public int getPeriodicFlushFrequencyMilliSeconds() {
-        return this.getInt(BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
-                BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT);
-    }
-
-    /**
-     * Set Periodic Log Flush Frequency in milliseconds.
-     *
-     * @param flushFrequencyMs periodic flush frequency in milliseconds.
-     * @return distributed log configuration
-     * @see #getPeriodicFlushFrequencyMilliSeconds()
-     */
-    public DistributedLogConfiguration setPeriodicFlushFrequencyMilliSeconds(int flushFrequencyMs) {
-        setProperty(BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS, flushFrequencyMs);
-        return this;
-    }
-
-    /**
-     * Is immediate flush enabled.
-     * <p>If it is enabled, it would flush control record immediately after adding
-     * data completed. The default value is false.
-     *
-     * @return whether immediate flush is enabled
-     */
-    public boolean getImmediateFlushEnabled() {
-        return getBoolean(BKDL_ENABLE_IMMEDIATE_FLUSH, BKDL_ENABLE_IMMEDIATE_FLUSH_DEFAULT);
-    }
-
-    /**
-     * Enable/Disable immediate flush
-     *
-     * @param enabled
-     *          flag to enable/disable immediate flush.
-     * @return configuration instance.
-     * @see #getImmediateFlushEnabled()
-     */
-    public DistributedLogConfiguration setImmediateFlushEnabled(boolean enabled) {
-        setProperty(BKDL_ENABLE_IMMEDIATE_FLUSH, enabled);
-        return this;
-    }
-
-    /**
-     * Get minimum delay between immediate flushes in milliseconds.
-     * <p>This setting only takes effects when {@link #getImmediateFlushEnabled()}
-     * is enabled. It torelants the bursty of traffic when immediate flush is enabled,
-     * which prevents sending too many control records to the bookkeeper.
-     *
-     * @return minimum delay between immediate flushes in milliseconds
-     * @see #getImmediateFlushEnabled()
-     */
-    public int getMinDelayBetweenImmediateFlushMs() {
-        return this.getInt(BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS, BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS_DEFAULT);
-    }
-
-    /**
-     * Set minimum delay between immediate flushes in milliseconds
-     *
-     * @param minDelayMs minimum delay between immediate flushes in milliseconds.
-     * @return distributed log configuration
-     * @see #getMinDelayBetweenImmediateFlushMs()
-     */
-    public DistributedLogConfiguration setMinDelayBetweenImmediateFlushMs(int minDelayMs) {
-        setProperty(BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS, minDelayMs);
-        return this;
-    }
-
-    /**
-     * Get Periodic Keep Alive Frequency in milliseconds.
-     * <p>If the setting is set with a positive value, it would periodically write a control record
-     * to keep the stream active. The default value is 0.
-     *
-     * @return periodic keep alive frequency in milliseconds.
-     */
-    public int getPeriodicKeepAliveMilliSeconds() {
-        return this.getInt(BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS, BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS_DEFAULT);
-    }
-
-    /**
-     * Set Periodic Keep Alive Frequency in milliseconds.
-     *
-     * @param keepAliveMs keep alive frequency in milliseconds.
-     * @return distributedlog configuration
-     * @see #getPeriodicKeepAliveMilliSeconds()
-     */
-    public DistributedLogConfiguration setPeriodicKeepAliveMilliSeconds(int keepAliveMs) {
-        setProperty(BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS, keepAliveMs);
-        return this;
-    }
-
-    //
-    // DL Retention/Truncation Settings
-    //
-
-    /**
-     * Get log segment retention period in hours.
-     * The default value is 3 days.
-     *
-     * @return log segment retention period in hours
-     */
-    public int getRetentionPeriodHours() {
-        return this.getInt(BKDL_RETENTION_PERIOD_IN_HOURS,
-                getInt(BKDL_RETENTION_PERIOD_IN_HOURS_OLD,
-                        BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT));
-    }
-
-    /**
-     * Set log segment retention period in hours.
-     *
-     * @param retentionHours retention period in hours.
-     * @return distributed log configuration
-     */
-    public DistributedLogConfiguration setRetentionPeriodHours(int retentionHours) {
-        setProperty(BKDL_RETENTION_PERIOD_IN_HOURS, retentionHours);
-        return this;
-    }
-
-    /**
-     * Is truncation managed explicitly by the application.
-     * <p>If this is set then time based retention is only a hint to perform
-     * deferred cleanup. However we never remove a segment that has not been
-     * already marked truncated.
-     * <p>It is disabled by default.
-     *
-     * @return whether truncation managed explicitly by the application
-     * @see com.twitter.distributedlog.LogSegmentMetadata.TruncationStatus
-     */
-    public boolean getExplicitTruncationByApplication() {
-        return getBoolean(BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION,
-                BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION_DEFAULT);
-    }
-
-    /**
-     * Enable/Disable whether truncation is managed explicitly by the application.
-     *
-     * @param enabled
-     *          flag to enable/disable whether truncation is managed explicitly by the application.
-     * @return configuration instance.
-     */
-    public DistributedLogConfiguration setExplicitTruncationByApplication(boolean enabled) {
-        setProperty(BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION, enabled);
-        return this;
-    }
-
-    //
-    // Log Segment Rolling Settings
-    //
-
-    /**
-     * Get log segment rolling interval in minutes.
-     * <p>If the setting is set to a positive value, DL writer will roll log segments
-     * based on time. Otherwise, it will roll log segments based on size.
-     * <p>The default value is 2 hours.
-     *
-     * @return log segment rolling interval in minutes
-     * @see #getMaxLogSegmentBytes()
-     */
-    public int getLogSegmentRollingIntervalMinutes() {
-        return this.getInt(BKDL_ROLLING_INTERVAL_IN_MINUTES,
-                getInt(BKDL_ROLLING_INTERVAL_IN_MINUTES_OLD,
-                        BKDL_ROLLING_INTERVAL_IN_MINUTES_DEFAULT));
-    }
-
-    /**
-     * Set log segment rolling interval in minutes.
-     *
-     * @param rollingMinutes rolling interval in minutes.
-     * @return distributed log configuration
-     * @see #getLogSegmentRollingIntervalMinutes()
-     */
-    public DistributedLogConfiguration setLogSegmentRollingIntervalMinutes(int rollingMinutes) {
-        setProperty(BKDL_ROLLING_INTERVAL_IN_MINUTES, rollingMinutes);
-        return this;
-    }
-
-    /**
-     * Get Max LogSegment Size in Bytes.
-     * <p>This setting only takes effects when time based rolling is disabled.
-     * DL writer will roll into a new log segment only after current one reaches
-     * this threshold.
-     * <p>The default value is 256MB.
-     *
-     * @return max logsegment size in bytes.
-     * @see #getLogSegmentRollingIntervalMinutes()
-     */
-    public long getMaxLogSegmentBytes() {
-        long maxBytes = this.getLong(BKDL_MAX_LOGSEGMENT_BYTES, BKDL_MAX_LOGSEGMENT_BYTES_DEFAULT);
-        if (maxBytes <= 0) {
-            maxBytes = BKDL_MAX_LOGSEGMENT_BYTES_DEFAULT;
-        }
-        return maxBytes;
-    }
-
-    /**
-     * Set Max LogSegment Size in Bytes.
-     *
-     * @param maxBytes
-     *          max logsegment size in bytes.
-     * @return configuration.
-     * @see #getMaxLogSegmentBytes()
-     */
-    public DistributedLogConfiguration setMaxLogSegmentBytes(long maxBytes) {
-        setProperty(BKDL_MAX_LOGSEGMENT_BYTES, maxBytes);
-        return this;
-    }
-
-    /**
-     * Get log segment rolling concurrency.
-     * <p>It limits how many writers could roll log segments concurrently.
-     * The default value is 1.
-     *
-     * @return log segment rolling concurrency.
-     * @see #setLogSegmentRollingConcurrency(int)
-     */
-    public int getLogSegmentRollingConcurrency() {
-        return getInt(BKDL_LOGSEGMENT_ROLLING_CONCURRENCY, BKDL_LOGSEGMENT_ROLLING_CONCURRENCY_DEFAULT);
-    }
-
-    /**
-     * Set log segment rolling concurrency. <i>0</i> means disable rolling concurrency.
-     * <i>larger than 0</i> means how many log segment could be rolled at the same time.
-     * <i>less than 0</i> means unlimited concurrency on rolling log segments.
-     *
-     * @param concurrency
-     *          log segment rolling concurrency.
-     * @return distributed log configuration.
-     * @see #getLogSegmentRollingConcurrency()
-     */
-    public DistributedLogConfiguration setLogSegmentRollingConcurrency(int concurrency) {
-        setProperty(BKDL_LOGSEGMENT_ROLLING_CONCURRENCY, concurrency);
-        return this;
-    }
-
-    //
-    // Lock Settings
-    //
-
-    /**
-     * Is lock enabled when opening a writer to write a stream?
-     * <p> We don't generally require a lock to write a stream to guarantee correctness. The lock
-     * is more on tracking ownerships. The built-in fencing mechanism is used guarantee correctness
-     * during stream owner failover. It is okay to disable lock if your application knows which nodes
-     * have to write which streams.
-     *
-     * @return true if lock is enabled, otherwise false.
-     */
-    public boolean isWriteLockEnabled() {
-        return this.getBoolean(BKDL_WRITE_LOCK_ENABLED, BKDL_WRITE_LOCK_ENABLED_DEFAULT);
-    }
-
-    /**
-     * Enable lock for opening a writer to write a stream?
-     *
-     * @param enabled flag to enable or disable lock for opening a writer to write a stream.
-     * @return distributedlog configuration.
-     */
-    public DistributedLogConfiguration setWriteLockEnabled(boolean enabled) {
-        setProperty(BKDL_WRITE_LOCK_ENABLED, enabled);
-        return this;
-    }
-
-    /**
-     * Get lock timeout in milliseconds. The default value is 30.
-     *
-     * @return lock timeout in milliseconds
-     */
-    public long getLockTimeoutMilliSeconds() {
-        return this.getLong(BKDL_LOCK_TIMEOUT, BKDL_LOCK_TIMEOUT_DEFAULT) * 1000;
-    }
-
-    /**
-     * Set lock timeout in seconds.
-     *
-     * @param lockTimeout lock timeout in seconds.
-     * @return distributed log configuration
-     * @see #getLockTimeoutMilliSeconds()
-     */
-    public DistributedLogConfiguration setLockTimeout(long lockTimeout) {
-        setProperty(BKDL_LOCK_TIMEOUT, lockTimeout);
-        return this;
-    }
-
-    /**
-     * Get lock reacquire timeout in milliseconds. The default value is 120 seconds.
-     *
-     * @return lock reacquire timeout in milliseconds
-     */
-    public long getLockReacquireTimeoutMilliSeconds() {
-        return this.getLong(BKDL_LOCK_REACQUIRE_TIMEOUT, BKDL_LOCK_REACQUIRE_TIMEOUT_DEFAULT) * 1000;
-    }
-
-    /**
-     * Set lock reacquire timeout in seconds.
-     *
-     * @param lockReacquireTimeout lock reacquire timeout in seconds.
-     * @return distributed log configuration
-     * @see #getLockReacquireTimeoutMilliSeconds()
-     */
-    public DistributedLogConfiguration setLockReacquireTimeoutSeconds(long lockReacquireTimeout) {
-        setProperty(BKDL_LOCK_REACQUIRE_TIMEOUT, lockReacquireTimeout);
-        return this;
-    }
-
-    /**
-     * Get lock internal operation timeout in milliseconds.
-     * The default value is 120 seconds.
-     *
-     * @return lock internal operation timeout in milliseconds.
-     */
-    public long getLockOpTimeoutMilliSeconds() {
-        return this.getLong(BKDL_LOCK_OP_TIMEOUT, BKDL_LOCK_OP_TIMEOUT_DEFAULT) * 1000;
-    }
-
-    /**
-     * Set lock internal operation timeout in seconds.
-     *
-     * @param lockOpTimeout lock internal operation timeout in seconds.
-     * @return distributed log configuration
-     * @see #getLockOpTimeoutMilliSeconds()
-     */
-    public DistributedLogConfiguration setLockOpTimeoutSeconds(long lockOpTimeout) {
-        setProperty(BKDL_LOCK_OP_TIMEOUT, lockOpTimeout);
-        return this;
-    }
-
-    //
-    // Ledger Allocator Settings
-    //
-
-    /**
-     * Whether to enable ledger allocator pool or not.
-     * It is disabled by default.
-     *
-     * @return whether using ledger allocator pool or not.
-     */
-    public boolean getEnableLedgerAllocatorPool() {
-        return getBoolean(BKDL_ENABLE_LEDGER_ALLOCATOR_POOL, BKDL_ENABLE_LEDGER_ALLOCATOR_POOL_DEFAULT);
-    }
-
-    /**
-     * Enable/Disable ledger allocator pool.
-     *
-     * @param enabled
-     *          enable/disable ledger allocator pool.
-     * @return configuration.
-     * @see #getEnableLedgerAllocatorPool()
-     */
-    public DistributedLogConfiguration setEnableLedgerAllocatorPool(boolean enabled) {
-        setProperty(BKDL_ENABLE_LEDGER_ALLOCATOR_POOL, enabled);
-        return this;
-    }
-
-    /**
-     * Get the path of ledger allocator pool.
-     * The default value is ".allocation_pool".
-     *
-     * @return path of ledger allocator pool.
-     */
-    public String getLedgerAllocatorPoolPath() {
-        return getString(BKDL_LEDGER_ALLOCATOR_POOL_PATH, BKDL_LEDGER_ALLOCATOR_POOL_PATH_DEFAULT);
-    }
-
-    /**
-     * Set the root path of ledger allocator pool
-     *
-     * @param path
-     *          path of ledger allocator pool.
-     * @return configuration
-     * @see #getLedgerAllocatorPoolPath()
-     */
-    public DistributedLogConfiguration setLedgerAllocatorPoolPath(String path) {
-        setProperty(BKDL_LEDGER_ALLOCATOR_POOL_PATH, path);
-        return this;
-    }
-
-    /**
-     * Get the name of ledger allocator pool.
-     *
-     * @return name of ledger allocator pool.
-     */
-    public String getLedgerAllocatorPoolName() {
-        return getString(BKDL_LEDGER_ALLOCATOR_POOL_NAME, BKDL_LEDGER_ALLOCATOR_POOL_NAME_DEFAULT);
-    }
-
-    /**
-     * Set name of ledger allocator pool.
-     *
-     * @param name
-     *          name of ledger allocator pool.
-     * @return configuration.
-     */
-    public DistributedLogConfiguration setLedgerAllocatorPoolName(String name) {
-        setProperty(BKDL_LEDGER_ALLOCATOR_POOL_NAME, name);
-        return this;
-    }
-
-    /**
-     * Get the core size of ledger allocator pool.
-     * The default value is 20.
-     *
-     * @return core size of ledger allocator pool.
-     */
-    public int getLedgerAllocatorPoolCoreSize() {
-        return getInt(BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE, BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE_DEFAULT);
-    }
-
-    /**
-     * Set core size of ledger allocator pool.
-     *
-     * @param poolSize
-     *          core size of ledger allocator pool.
-     * @return distributedlog configuration.
-     * @see #getLedgerAllocatorPoolCoreSize()
-     */
-    public DistributedLogConfiguration setLedgerAllocatorPoolCoreSize(int poolSize) {
-        setProperty(BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE, poolSize);
-        return this;
-    }
-
-    //
-    // Write Limit Settings
-    //
-
-    /**
-     * Get the per stream outstanding write limit for dl.
-     * <p>If the setting is set with a positive value, the per stream
-     * write limiting is enabled. By default it is disabled.
-     *
-     * @return the per stream outstanding write limit for dl
-     * @see #getGlobalOutstandingWriteLimit()
-     */
-    public int getPerWriterOutstandingWriteLimit() {
-        return getInt(BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT,
-                BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT_DEFAULT);
-    }
-
-    /**
-     * Set the per stream outstanding write limit for dl.
-     *
-     * @param limit
-     *          per stream outstanding write limit for dl
-     * @return dl configuration
-     * @see #getPerWriterOutstandingWriteLimit()
-     */
-    public DistributedLogConfiguration setPerWriterOutstandingWriteLimit(int limit) {
-        setProperty(BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT, limit);
-        return this;
-    }
-
-    /**
-     * Get the global write limit for dl.
-     * <p>If the setting is set with a positive value, the global
-     * write limiting is enabled. By default it is disabled.
-     *
-     * @return the global write limit for dl
-     * @see #getPerWriterOutstandingWriteLimit()
-     */
-    public int getGlobalOutstandingWriteLimit() {
-        return getInt(BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT, BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT_DEFAULT);
-    }
-
-    /**
-     * Set the global write limit for dl.
-     *
-     * @param limit
-     *          global write limit for dl
-     * @return dl configuration
-     * @see #getGlobalOutstandingWriteLimit()
-     */
-    public DistributedLogConfiguration setGlobalOutstandingWriteLimit(int limit) {
-        setProperty(BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT, limit);
-        return this;
-    }
-
-    /**
-     * Whether to darkmode outstanding writes limit.
-     * <p>If it is running in darkmode, it would not reject requests when
-     * it is over limit, but just record them in the stats.
-     * <p>By default, it is in darkmode.
-     *
-     * @return flag to darmkode pending write limit.
-     */
-    public boolean getOutstandingWriteLimitDarkmode() {
-        return getBoolean(BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE,
-                BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE_DEFAULT);
-    }
-
-    /**
-     * Set the flag to darkmode outstanding writes limit.
-     *
-     * @param darkmoded
-     *          flag to darmkode pending write limit
-     * @return dl configuration.
-     * @see #getOutstandingWriteLimitDarkmode()
-     */
-    public DistributedLogConfiguration setOutstandingWriteLimitDarkmode(boolean darkmoded) {
-        setProperty(BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE, darkmoded);
-        return this;
-    }
-
-    //
-    // DL Reader General Settings
-    //
-
-    /**
-     * Get the long poll time out for read last add confirmed requests, in milliseconds.
-     * The default value is 1 second.
-     *
-     * @return long poll timeout in milliseconds
-     * @see #getReadLACLongPollTimeout()
-     */
-    public int getReadLACLongPollTimeout() {
-        return this.getInt(BKDL_READLACLONGPOLL_TIMEOUT, BKDL_READLACLONGPOLL_TIMEOUT_DEFAULT);
-    }
-
-    /**
-     * Set the long poll time out for read last add confirmed requests, in milliseconds.
-     *
-     * @param readAheadLongPollTimeout long poll timeout in milliseconds
-     * @return distributed log configuration
-     * @see #getReadLACLongPollTimeout()
-     */
-    public DistributedLogConfiguration setReadLACLongPollTimeout(int readAheadLongPollTimeout) {
-        setProperty(BKDL_READLACLONGPOLL_TIMEOUT, readAheadLongPollTimeout);
-        return this;
-    }
-
-    /**
-     * Get the flag whether to deserialize record set on reads.
-     *
-     * @return true if it should deserialize, otherwise false.
-     */
-    public boolean getDeserializeRecordSetOnReads() {
-        return getBoolean(BKDL_DESERIALIZE_RECORDSET_ON_READS, BKDL_DESERIALIZE_RECORDSET_ON_READS_DEFAULT);
-    }
-
-    /**
-     * Enable or disable deserialize recordset on reads.
-     *
-     * @param enabled
-     *          flag whether to deserialize recordset
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setDeserializeRecordSetOnReads(boolean enabled) {
-        setProperty(BKDL_DESERIALIZE_RECORDSET_ON_READS, enabled);
-        return this;
-    }
-
-    //
-    // Idle reader settings
-    //
-
-    /**
-     * Get the time in milliseconds as the threshold for when an idle reader should dump warnings
-     * <p>The default value is 2 minutes.
-     *
-     * @return reader idle warn threshold in millis.
-     * @see #getReaderIdleErrorThresholdMillis()
-     */
-    public int getReaderIdleWarnThresholdMillis() {
-        return getInt(BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS,
-                BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS_DEFAULT);
-    }
-
-    /**
-     * Set the time in milliseconds as the threshold for when an idle reader should dump warnings
-     *
-     * @param warnThreshold time after which we should dump the read ahead state
-     * @return distributed log configuration
-     * @see #getReaderIdleWarnThresholdMillis()
-     */
-    public DistributedLogConfiguration setReaderIdleWarnThresholdMillis(int warnThreshold) {
-        setProperty(BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS, warnThreshold);
-        return this;
-    }
-
-    /**
-     * Get the time in milliseconds as the threshold for when an idle reader should throw errors
-     * <p>The default value is <i>Integer.MAX_VALUE</i>.
-     *
-     * @return reader idle error threshold in millis
-     * @see #getReaderIdleWarnThresholdMillis()
-     */
-    public int getReaderIdleErrorThresholdMillis() {
-        return getInt(BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS,
-                BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT);
-    }
-
-    /**
-     * Set the time in milliseconds as the threshold for when an idle reader should throw errors
-     *
-     * @param warnThreshold time after which we should throw idle reader errors
-     * @return distributed log configuration
-     * @see #getReaderIdleErrorThresholdMillis()
-     */
-    public DistributedLogConfiguration setReaderIdleErrorThresholdMillis(int warnThreshold) {
-        setProperty(BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS, warnThreshold);
-        return this;
-    }
-
-    //
-    // Reader Constraint Settings
-    //
-
-    /**
-     * Get if we should ignore truncation status when reading the records
-     *
-     * @return if we should ignore truncation status
-     */
-    public boolean getIgnoreTruncationStatus() {
-        return getBoolean(BKDL_READER_IGNORE_TRUNCATION_STATUS, BKDL_READER_IGNORE_TRUNCATION_STATUS_DEFAULT);
-    }
-
-    /**
-     * Set if we should ignore truncation status when reading the records
-     *
-     * @param ignoreTruncationStatus
-     *          if we should ignore truncation status
-     */
-    public DistributedLogConfiguration setIgnoreTruncationStatus(boolean ignoreTruncationStatus) {
-        setProperty(BKDL_READER_IGNORE_TRUNCATION_STATUS, ignoreTruncationStatus);
-        return this;
-    }
-
-    /**
-     * Get if we should alert when reader is positioned on a truncated segment
-     *
-     * @return if we should alert when reader is positioned on a truncated segment
-     */
-    public boolean getAlertWhenPositioningOnTruncated() {
-        return getBoolean(BKDL_READER_ALERT_POSITION_ON_TRUNCATED, BKDL_READER_ALERT_POSITION_ON_TRUNCATED_DEFAULT);
-    }
-
-    /**
-     * Set if we should alert when reader is positioned on a truncated segment
-     *
-     * @param alertWhenPositioningOnTruncated
-     *          if we should alert when reader is positioned on a truncated segment
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setAlertWhenPositioningOnTruncated(boolean alertWhenPositioningOnTruncated) {
-        setProperty(BKDL_READER_ALERT_POSITION_ON_TRUNCATED, alertWhenPositioningOnTruncated);
-        return this;
-    }
-
-    /**
-     * Get whether position gap detection for reader enabled.
-     * @return whether position gap detection for reader enabled.
-     */
-    public boolean getPositionGapDetectionEnabled() {
-        return getBoolean(BKDL_READER_POSITION_GAP_DETECTION_ENABLED, BKDL_READER_POSITION_GAP_DETECTION_ENABLED_DEFAULT);
-    }
-
-    /**
-     * Set if enable position gap detection for reader.
-     *
-     * @param enabled
-     *          flag to enable/disable position gap detection on reader.
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setPositionGapDetectionEnabled(boolean enabled) {
-        setProperty(BKDL_READER_POSITION_GAP_DETECTION_ENABLED, enabled);
-        return this;
-    }
-
-    //
-    // ReadAhead Settings
-    //
-
-    /**
-     * Set if we should enable read ahead.
-     * By default is it enabled.
-     *
-     * @param enableReadAhead
-     *          Enable read ahead
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setEnableReadAhead(boolean enableReadAhead) {
-        setProperty(BKDL_ENABLE_READAHEAD, enableReadAhead);
-        return this;
-    }
-
-    /**
-     * Get if we should enable read ahead
-     *
-     * @return if read ahead is enabled
-     */
-    public boolean getEnableReadAhead() {
-        return getBoolean(BKDL_ENABLE_READAHEAD, BKDL_ENABLE_READAHEAD_DEFAULT);
-    }
-
-    /**
-     * Set if we should enable force read
-     *
-     * @param enableForceRead
-     *          Enable force read
-     */
-    public DistributedLogConfiguration setEnableForceRead(boolean enableForceRead) {
-        setProperty(BKDL_ENABLE_FORCEREAD, enableForceRead);
-        return this;
-    }
-
-    /**
-     * Get if we should enable force read
-     *
-     * @return if should use separate ZK Clients
-     */
-    public boolean getEnableForceRead() {
-        return getBoolean(BKDL_ENABLE_FORCEREAD, BKDL_ENABLE_FORCEREAD_DEFAULT);
-    }
-
-    /**
-     * Get the max records cached by readahead cache.
-     * <p>The default value is 10. Increase this value to improve throughput,
-     * but be careful about the memory.
-     *
-     * @return max records cached by readahead cache.
-     */
-    public int getReadAheadMaxRecords() {
-        return this.getInt(BKDL_READAHEAD_MAX_RECORDS,
-                getInt(BKDL_READAHEAD_MAX_RECORDS_OLD,
-                        BKDL_READAHEAD_MAX_RECORDS_DEFAULT));
-    }
-
-    /**
-     * Set the maximum records allowed to be cached by read ahead worker.
-     *
-     * @param readAheadMaxEntries max records to cache.
-     * @return distributed log configuration
-     * @see #getReadAheadMaxRecords()
-     */
-    public DistributedLogConfiguration setReadAheadMaxRecords(int readAheadMaxEntries) {
-        setProperty(BKDL_READAHEAD_MAX_RECORDS, readAheadMaxEntries);
-        return this;
-    }
-
-    /**
-     * Get number of entries read as a batch by readahead worker.
-     * <p>The default value is 2. Increase the value to increase the concurrency
-     * of reading entries from bookkeeper.
-     *
-     * @return number of entries read as a batch.
-     */
-    public int getReadAheadBatchSize() {
-        return this.getInt(BKDL_READAHEAD_BATCHSIZE,
-                getInt(BKDL_READAHEAD_BATCHSIZE_OLD,
-                        BKDL_READAHEAD_BATCHSIZE_DEFAULT));
-    }
-
-    /**
-     * Set number of entries read as a batch by readahead worker.
-     *
-     * @param readAheadBatchSize
-     *          Read ahead batch size.
-     * @return distributed log configuration
-     * @see #getReadAheadBatchSize()
-     */
-    public DistributedLogConfiguration setReadAheadBatchSize(int readAheadBatchSize) {
-        setProperty(BKDL_READAHEAD_BATCHSIZE, readAheadBatchSize);
-        return this;
-    }
-
-    /**
-     * Get the wait time between successive attempts to poll for new log records, in milliseconds.
-     * The default value is 200 ms.
-     *
-     * @return read ahead wait time
-     */
-    public int getReadAheadWaitTime() {
-        return this.getInt(BKDL_READAHEAD_WAITTIME,
-                getInt(BKDL_READAHEAD_WAITTIME_OLD, BKDL_READAHEAD_WAITTIME_DEFAULT));
-    }
-
-    /**
-     * Set the wait time between successive attempts to poll for new log records, in milliseconds
-     *
-     * @param readAheadWaitTime read ahead wait time
-     * @return distributed log configuration
-     * @see #getReadAheadWaitTime()
-     */
-    public DistributedLogConfiguration setReadAheadWaitTime(int readAheadWaitTime) {
-        setProperty(BKDL_READAHEAD_WAITTIME, readAheadWaitTime);
-        return this;
-    }
-
-    /**
-     * Get the wait time if it reaches end of stream and
-     * <b>there isn't any inprogress logsegment in the stream</b>, in millis.
-     * <p>The default value is 10 seconds.
-     *
-     * @see #setReadAheadWaitTimeOnEndOfStream(int)
-     * @return the wait time if it reaches end of stream and there isn't
-     * any inprogress logsegment in the stream, in millis.
-     */
-    public int getReadAheadWaitTimeOnEndOfStream() {
-        return this.getInt(BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM,
-                getInt(BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM_OLD,
-                        BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM_DEFAULT));
-    }
-
-    /**
-     * Set the wait time that would be used for readahead to backoff polling
-     * logsegments from zookeeper when it reaches end of stream and there isn't
-     * any inprogress logsegment in the stream. The unit is millis.
-     *
-     * @param waitTime
-     *          wait time that readahead used to backoff when reaching end of stream.
-     * @return distributedlog configuration
-     * @see #getReadAheadWaitTimeOnEndOfStream()
-     */
-    public DistributedLogConfiguration setReadAheadWaitTimeOnEndOfStream(int waitTime) {
-        setProperty(BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM, waitTime);
-        return this;
-    }
-
-    /**
-     * If readahead keeps receiving {@link org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException} on
-     * reading last add confirmed in given period, it would stop polling last add confirmed and re-initialize the ledger
-     * handle and retry. The threshold is specified in milliseconds.
-     * <p>The default value is 10 seconds.
-     *
-     * @return error threshold in milliseconds, that readahead will reinitialize ledger handle after keeping receiving
-     * no such ledger exceptions.
-     */
-    public int getReadAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis() {
-        return this.getInt(BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS,
-                           BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS_DEFAULT);
-    }
-
-    /**
-     * Set the error threshold that readahead will reinitialize ledger handle after keeping receiving no such ledger exceptions.
-     *
-     * @see #getReadAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis()
-     * @param thresholdMillis
-     *          error threshold in milliseconds, that readahead will reinitialize ledger handle after keeping receiving
-     *          no such ledger exceptions.
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setReadAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis(long thresholdMillis) {
-        setProperty(BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS, thresholdMillis);
-        return this;
-    }
-
-    /**
-     * When corruption is encountered in an entry, skip it and move on. Must disable gap detection for
-     * this to work.
-     *
-     * @return should broken records be skipped
-     */
-    public boolean getReadAheadSkipBrokenEntries() {
-        return getBoolean(BKDL_READAHEAD_SKIP_BROKEN_ENTRIES, BKDL_READAHEAD_SKIP_BROKEN_ENTRIES_DEFAULT);
-    }
-
-    /**
-     * Set the percentage of operations to delay in read ahead.
-     *
-     * @param enabled
-     *          should brokenn records be skipped
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setReadAheadSkipBrokenEntries(boolean enabled) {
-        setProperty(BKDL_READAHEAD_SKIP_BROKEN_ENTRIES, enabled);
-        return this;
-    }
-
-    /**
-     * Get the number prefetch entries per log segment. Default value is 4.
-     *
-     * @return the number prefetch entries per log segment.
-     */
-    public int getNumPrefetchEntriesPerLogSegment() {
-        return getInt(BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT, BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT);
-    }
-
-    /**
-     * Set the number prefetch entries per log segment.
-     *
-     * @param numEntries the number prefetch entries per log segment.
-     * @return configuration
-     */
-    public DistributedLogConfiguration setNumPrefetchEntriesPerLogSegment(int numEntries) {
-        setProperty(BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT, numEntries);
-        return this;
-    }
-
-    /**
-     * Get the max prefetch entries per log segment. Default value is 4.
-     *
-     * @return the max prefetch entries per log segment.
-     */
-    public int getMaxPrefetchEntriesPerLogSegment() {
-        return getInt(BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT, BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT);
-    }
-
-    /**
-     * Set the max prefetch entries per log segment.
-     *
-     * @param numEntries the max prefetch entries per log segment.
-     * @return configuration
-     */
-    public DistributedLogConfiguration setMaxPrefetchEntriesPerLogSegment(int numEntries) {
-        setProperty(BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT, numEntries);
-        return this;
-    }
-
-    //
-    // DL Reader Scan Settings
-    //
-
-    /**
-     * Number of entries to scan for first scan of reading last record.
-     *
-     * @return number of entries to scan for first scan of reading last record.
-     */
-    public int getFirstNumEntriesPerReadLastRecordScan() {
-        return getInt(BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN, BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT);
-    }
-
-    /**
-     * Set number of entries to scan for first scan of reading last record.
-     *
-     * @param numEntries
-     *          number of entries to scan
-     * @return distributedlog configuration.
-     */
-    public DistributedLogConfiguration setFirstNumEntriesPerReadLastRecordScan(int numEntries) {
-        setProperty(BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN, numEntries);
-        return this;
-    }
-
-    /**
-     * Max number of entries for each scan to read last record.
-     *
-     * @return max number of entries for each scan to read last record.
-     */
-    public int getMaxNumEntriesPerReadLastRecordScan() {
-        return getInt(BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN, BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT);
-    }
-
-    /**
-     * Set max number of entries for each scan to read last record.
-     *
-     * @param numEntries
-     *          number of entries to scan
-     * @return distributedlog configuration.
-     */
-    public DistributedLogConfiguration setMaxNumEntriesPerReadLastRecordScan(int numEntries) {
-        setProperty(BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN, numEntries);
-        return this;
-    }
-
-    //
-    // DL Reader Log Existence Checking Settings
-    //
-
-    /**
-     * Get the backoff start time to check log existence if the log doesn't exist.
-     *
-     * @return the backoff start time to check log existence if the log doesn't exist.
-     */
-    public long getCheckLogExistenceBackoffStartMillis() {
-        return getLong(BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS, BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS_DEFAULT);
-    }
-
-    /**
-     * Set the backoff start time to check log existence if the log doesn't exist.
-     *
-     * @param backoffMillis
-     *          backoff time in millis
-     * @return dl configuration
-     */
-    public DistributedLogConfiguration setCheckLogExistenceBackoffStartMillis(long backoffMillis) {
-        setProperty(BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS, backoffMillis);
-        return this;
-    }
-
-    /**
-     * Get the backoff max time to check log existence if the log doesn't exist.
-     *
-     * @return the backoff max time to check log existence if the log doesn't exist.
-     */
-    public long getCheckLogExistenceBackoffMaxMillis() {
-        return getLong(BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS, BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS_DEFAULT);
-    }
-
-    /**
-     * Set the backoff max time to check log existence if the log doesn't exist.
-     *
-     * @param backoffMillis
-     *          backoff time in millis
-     * @return dl configuration
-     */
-    public DistributedLogConfiguration setCheckLogExistenceBackoffMaxMillis(long backoffMillis) {
-        setProperty(BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS, backoffMillis);
-        return this;
-    }
-
-    //
-    // Tracing/Stats Settings
-    //
-
-    /**
-     * Whether to trace read ahead delivery latency or not?
-     *
-     * @return flag to trace read ahead delivery latency.
-     */
-    public boolean getTraceReadAheadDeliveryLatency() {
-        return getBoolean(BKDL_TRACE_READAHEAD_DELIVERY_LATENCY, BKDL_TRACE_READAHEAD_DELIVERY_LATENCY_DEFAULT);
-    }
-
-    /**
-     * Set the flag to trace readahead delivery latency.
-     *
-     * @param enabled
-     *          flag to trace readahead delivery latency.
-     * @return dl configuration.
-     */
-    public DistributedLogConfiguration setTraceReadAheadDeliveryLatency(boolean enabled) {
-        setProperty(BKDL_TRACE_READAHEAD_DELIVERY_LATENCY, enabled);
-        return this;
-    }
-
-    /**
-     * Get the warn threshold (in millis) of metadata access latency.
-     *
-     * @return warn threshold of metadata access latency, in millis.
-     */
-    public long getMetadataLatencyWarnThresholdMillis() {
-        return getLong(BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS, BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT);
-    }
-
-    /**
-     * Set the warn threshold of metadata access latency, in millis.
-     *
-     * @param warnThresholdMillis
-     *          warn threshold of metadata access latency, in millis
-     * @return dl configuration
-     */
-    public DistributedLogConfiguration setMetadataLatencyWarnThresholdMillis(long warnThresholdMillis) {
-        setProperty(BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS, warnThresholdMillis);
-        return this;
-    }
-
-    /**
-     * Get the warn threshold (in millis) of data access latency.
-     *
-     * @return warn threshold of data access latency, in millis.
-     */
-    public long getDataLatencyWarnThresholdMillis() {
-        return getLong(BKDL_DATA_LATENCY_WARN_THRESHOLD_MS, BKDL_DATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT);
-    }
-
-    /**
-     * Set the warn threshold of data access latency, in millis.
-     *
-     * @param warnThresholdMillis
-     *          warn threshold of data access latency, in millis
-     * @return dl configuration
-     */
-    public DistributedLogConfiguration setDataLatencyWarnThresholdMillis(long warnThresholdMillis) {
-        setProperty(BKDL_DATA_LATENCY_WARN_THRESHOLD_MS, warnThresholdMillis);
-        return this;
-    }
-
-    /**
-     * Whether to trace read ahead changes? If enabled, it will log readahead metadata changes with timestamp.
-     * It is helpful when you are troubleshooting latency related issues.
-     *
-     * @return flag to trace read ahead delivery latency.
-     */
-    public boolean getTraceReadAheadMetadataChanges() {
-        return getBoolean(BKDL_TRACE_READAHEAD_METADATA_CHANGES, BKDL_TRACE_READAHEAD_MEATDATA_CHANGES_DEFAULT);
-    }
-
-    /**
-     * Set the flag to trace readahead metadata changes.
-     *
-     * @see #getTraceReadAheadMetadataChanges()
-     *
-     * @param enabled
-     *          flag to trace readahead metadata changes.
-     * @return dl configuration.
-     */
-    public DistributedLogConfiguration setTraceReadAheadMetadataChanges(boolean enabled) {
-        setProperty(BKDL_TRACE_READAHEAD_METADATA_CHANGES, enabled);
-        return this;
-    }
-
-    /**
-     * Whether to trace long running tasks and record task execution stats in thread pools.
-     *
-     * @return flag to enable task execution stats
-     */
-    public boolean getEnableTaskExecutionStats() {
-        return getBoolean(BKDL_ENABLE_TASK_EXECUTION_STATS, BKDL_ENABLE_TASK_EXECUTION_STATS_DEFAULT);
-    }
-
-    /**
-     * Set to trace long running tasks and record task execution stats in thread pools.
-     *
-     * @see #getEnableTaskExecutionStats()
-     *
-     * @param enabled
-     *          flag to enable task execution stats.
-     * @return dl configuration.
-     */
-    public DistributedLogConfiguration setEnableTaskExecutionStats(boolean enabled) {
-        setProperty(BKDL_ENABLE_TASK_EXECUTION_STATS, enabled);
-        return this;
-    }
-
-    /**
-     * Report long running task after execution takes longer than the given interval.
-     *
-     * @return warn time for long running tasks
-     */
-    public long getTaskExecutionWarnTimeMicros() {
-        return getLong(BKDL_TASK_EXECUTION_WARN_TIME_MICROS, BKDL_TASK_EXECUTION_WARN_TIME_MICROS_DEFAULT);
-    }
-
-    /**
-     * Set warn time for reporting long running tasks.
-     *
-     * @see #getTaskExecutionWarnTimeMicros()
-     *
-     * @param warnTimeMicros
-     *          warn time for long running tasks.
-     * @return dl configuration.
-     */
-    public DistributedLogConfiguration setTaskExecutionWarnTimeMicros(long warnTimeMicros) {
-        setProperty(BKDL_TASK_EXECUTION_WARN_TIME_MICROS, warnTimeMicros);
-        return this;
-    }
-
-    /**
-     * Whether to enable per stream stat or not.
-     *
-     * @deprecated please use {@link DistributedLogNamespaceBuilder#perLogStatsLogger(StatsLogger)}
-     * @return flag to enable per stream stat.
-     */
-    public boolean getEnablePerStreamStat() {
-        return getBoolean(BKDL_ENABLE_PERSTREAM_STAT, BKDL_ENABLE_PERSTREAM_STAT_DEFAULT);
-    }
-
-    /**
-     * Set the flag to enable per stream stat or not.
-     *
-     * @deprecated please use {@link DistributedLogNamespaceBuilder#perLogStatsLogger(StatsLogger)}
-     * @param enabled
-     *          flag to enable/disable per stream stat.
-     * @return dl configuration.
-     */
-    public DistributedLogConfiguration setEnablePerStreamStat(boolean enabled) {
-        setProperty(BKDL_ENABLE_PERSTREAM_STAT, enabled);
-        return this;
-    }
-
-    //
-    // Settings for Feature Providers
-    //
-
-    /**
-     * Get feature provider class.
-     *
-     * @return feature provider class.
-     * @throws ConfigurationException
-     */
-    public Class<? extends FeatureProvider> getFeatureProviderClass()
-            throws ConfigurationException {
-        return ReflectionUtils.getClass(this, BKDL_FEATURE_PROVIDER_CLASS, DefaultFeatureProvider.class,
-                FeatureProvider.class, FeatureProvider.class.getClassLoader());
-    }
-
-    /**
-     * Set feature provider class.
-     *
-     * @param providerClass
-     *          feature provider class.
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setFeatureProviderClass(Class<? extends FeatureProvider> providerClass) {
-        setProperty(BKDL_FEATURE_PROVIDER_CLASS, providerClass.getName());
-        return this;
-    }
-
-    /**
-     * Get the base config path for file feature provider.
-     *
-     * @return base config path for file feature provider.
-     */
-    public String getFileFeatureProviderBaseConfigPath() {
-        return getString(BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH,
-                BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH_DEFAULT);
-    }
-
-    /**
-     * Set the base config path for file feature provider.
-     *
-     * @param configPath
-     *          base config path for file feature provider.
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setFileFeatureProviderBaseConfigPath(String configPath) {
-        setProperty(BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH, configPath);
-        return this;
-    }
-
-    /**
-     * Get the overlay config path for file feature provider.
-     *
-     * @return overlay config path for file feature provider.
-     */
-    public String getFileFeatureProviderOverlayConfigPath() {
-        return getString(BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH,
-                BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH_DEFAULT);
-    }
-
-    /**
-     * Set the overlay config path for file feature provider.
-     *
-     * @param configPath
-     *          overlay config path for file feature provider.
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setFileFeatureProviderOverlayConfigPath(String configPath) {
-        setProperty(BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH,
-                configPath);
-        return this;
-    }
-
-    //
-    // Settings for Namespaces
-    //
-
-    /**
-     * Is federated namespace implementation enabled.
-     *
-     * @return true if federated namespace is enabled. otherwise, false.
-     */
-    public boolean isFederatedNamespaceEnabled() {
-        return getBoolean(BKDL_FEDERATED_NAMESPACE_ENABLED, BKDL_FEDERATED_NAMESPACE_ENABLED_DEFAULT);
-    }
-
-    /**
-     * Use federated namespace implementation if this flag is enabled.
-     *
-     * @param enabled flag to enable federated namespace implementation
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setFederatedNamespaceEnabled(boolean enabled) {
-        setProperty(BKDL_FEDERATED_NAMESPACE_ENABLED, enabled);
-        return this;
-    }
-
-    /**
-     * Get the max logs per sub namespace for federated namespace.
-     *
-     * @return max logs per sub namespace
-     */
-    public int getFederatedMaxLogsPerSubnamespace() {
-        return getInt(BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE, BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE_DEFAULT);
-    }
-
-    /**
-     * Set the max logs per sub namespace for federated namespace.
-     *
-     * @param maxLogs
-     *          max logs per sub namespace
-     * @return distributedlog configuration.
-     */
-    public DistributedLogConfiguration setFederatedMaxLogsPerSubnamespace(int maxLogs) {
-        setProperty(BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE, maxLogs);
-        return this;
-    }
-
-    /**
-     * Whether check the existence of a log if querying local cache of a federated namespace missed.
-     * Enabling it will issue zookeeper queries to check all sub namespaces under a federated namespace.
-     *
-     * NOTE: by default it is on for all admin related tools. for write proxies, consider turning off for
-     * performance.
-     *
-     * @return true if it needs to check existence of a log when querying local cache misses. otherwise false.
-     */
-    public boolean getFederatedCheckExistenceWhenCacheMiss() {
-        return getBoolean(BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS,
-                BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS_DEFAULT);
-    }
-
-    /**
-     * Enable check existence of a log if quering local cache of a federated namespace missed.
-     *
-     * @param enabled
-     *          flag to enable/disable this feature.
-     * @return distributedlog configuration.
-     */
-    public DistributedLogConfiguration setFederatedCheckExistenceWhenCacheMiss(boolean enabled) {
-        setProperty(BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS, enabled);
-        return this;
-    }
-
-    //
-    // Settings for Configurations
-    //
-
-    /**
-     * Get dynamic configuration reload interval in seconds.
-     *
-     * @return dynamic configuration reload interval
-     */
-    public int getDynamicConfigReloadIntervalSec() {
-        return getInt(BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC, BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC_DEFAULT);
-    }
-
-    /**
-     * Get dynamic configuration reload interval in seconds.
-     *
-     * @param intervalSec dynamic configuration reload interval in seconds
-     * @return distributedlog configuration.
-     */
-    public DistributedLogConfiguration setDynamicConfigReloadIntervalSec(int intervalSec) {
-        setProperty(BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC, intervalSec);
-        return this;
-    }
-
-    /**
-     * Get config router class which determines how stream name is mapped to configuration.
-     *
-     * @return config router class.
-     */
-    public String getStreamConfigRouterClass() {
-        return getString(BKDL_STREAM_CONFIG_ROUTER_CLASS, BKDL_STREAM_CONFIG_ROUTER_CLASS_DEFAULT);
-    }
-
-    /**
-     * Set config router class.
-     *
-     * @param routerClass
-     *          config router class.
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setStreamConfigRouterClass(String routerClass) {
-        setProperty(BKDL_STREAM_CONFIG_ROUTER_CLASS, routerClass);
-        return this;
-    }
-
-    //
-    // Settings for RateLimit
-    //
-
-    /**
-     * A lower threshold bytes per second limit on writes to the distributedlog proxy.
-     *
-     * @return Bytes per second write limit
-     */
-    public int getBpsSoftWriteLimit() {
-        return getInt(BKDL_BPS_SOFT_WRITE_LIMIT, BKDL_BPS_SOFT_WRITE_LIMIT_DEFAULT);
-    }
-
-    /**
-     * An upper threshold bytes per second limit on writes to the distributedlog proxy.
-     *
-     * @return Bytes per second write limit
-     */
-    public int getBpsHardWriteLimit() {
-        return getInt(BKDL_BPS_HARD_WRITE_LIMIT, BKDL_BPS_HARD_WRITE_LIMIT_DEFAULT);
-    }
-
-    /**
-     * A lower threshold requests per second limit on writes to the distributedlog proxy.
-     *
-     * @return Requests per second write limit
-     */
-    public int getRpsSoftWriteLimit() {
-        return getInt(BKDL_RPS_SOFT_WRITE_LIMIT, BKDL_RPS_SOFT_WRITE_LIMIT_DEFAULT);
-    }
-
-    /**
-     * An upper threshold requests per second limit on writes to the distributedlog proxy.
-     *
-     * @return Requests per second write limit
-     */
-    public int getRpsHardWriteLimit() {
-        return getInt(BKDL_RPS_HARD_WRITE_LIMIT, BKDL_RPS_HARD_WRITE_LIMIT_DEFAULT);
-    }
-
-    //
-    // Settings for partitioning
-    //
-
-    /**
-     * Get the maximum number of partitions of each stream allowed to be acquired per proxy.
-     * <p>This setting is able to configure per stream. This is the default setting if it is
-     * not configured per stream. Default value is -1, which means no limit on the number of
-     * partitions could be acquired each stream.
-     *
-     * @return maximum number of partitions of each stream allowed to be acquired per proxy.
-     */
-    public int getMaxAcquiredPartitionsPerProxy() {
-        return getInt(BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY, BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY_DEFAULT);
-    }
-
-    /**
-     * Set the maximum number of partitions of each stream allowed to be acquired per proxy.
-     *
-     * @param numPartitions
-     *          number of partitions of each stream allowed to be acquired
-     * @return distributedlog configuration
-     * @see #getMaxAcquiredPartitionsPerProxy()
-     */
-    public DistributedLogConfiguration setMaxAcquiredPartitionsPerProxy(int numPartitions) {
-        setProperty(BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY, numPartitions);
-        return this;
-    }
-
-    /**
-     * Get the maximum number of partitions of each stream allowed to cache per proxy.
-     * <p>This setting is able to configure per stream. This is the default setting if it is
-     * not configured per stream. Default value is -1, which means no limit on the number of
-     * partitions could be acquired each stream.
-     *
-     * @return maximum number of partitions of each stream allowed to be acquired per proxy.
-     */
-    public int getMaxCachedPartitionsPerProxy() {
-        return getInt(BKDL_MAX_CACHED_PARTITIONS_PER_PROXY, BKDL_MAX_CACHED_PARTITIONS_PER_PROXY_DEFAULT);
-    }
-
-    /**
-     * Set the maximum number of partitions of each stream allowed to cache per proxy.
-     *
-     * @param numPartitions
-     *          number of partitions of each stream allowed to cache
-     * @return distributedlog configuration
-     * @see #getMaxAcquiredPartitionsPerProxy()
-     */
-    public DistributedLogConfiguration setMaxCachedPartitionsPerProxy(int numPartitions) {
-        setProperty(BKDL_MAX_CACHED_PARTITIONS_PER_PROXY, numPartitions);
-        return this;
-    }
-
-    // Error Injection Settings
-
-    /**
-     * Should we enable write delay injection? If false we won't check other write delay settings.
-     *
-     * @return true if write delay injection is enabled.
-     */
-    public boolean getEIInjectWriteDelay() {
-        return getBoolean(BKDL_EI_INJECT_WRITE_DELAY, BKDL_EI_INJECT_WRITE_DELAY_DEFAULT);
-    }
-
-    /**
-     * Get percent of write requests which should be delayed by BKDL_EI_INJECTED_WRITE_DELAY_MS.
-     *
-     * @return percent of writes to delay.
-     */
-    public double getEIInjectedWriteDelayPercent() {
-        return getDouble(BKDL_EI_INJECTED_WRITE_DELAY_PERCENT, BKDL_EI_INJECTED_WRITE_DELAY_PERCENT_DEFAULT);
-    }
-
-    /**
-     * Set percent of write requests which should be delayed by BKDL_EI_INJECTED_WRITE_DELAY_MS. 0 disables
-     * write delay.
-     *
-     * @param percent
-     *          percent of writes to delay.
-     * @return dl configuration.
-     */
-    public DistributedLogConfiguration setEIInjectedWriteDelayPercent(double percent) {
-        setProperty(BKDL_EI_INJECTED_WRITE_DELAY_PERCENT, percent);
-        return this;
-    }
-
-    /**
-     * Get amount of time to delay writes for in writer failure injection.
-     *
-     * @return millis to delay writes for.
-     */
-    public int getEIInjectedWriteDelayMs() {
-        return getInt(BKDL_EI_INJECTED_WRITE_DELAY_MS, BKDL_EI_INJECTED_WRITE_DELAY_MS_DEFAULT);
-    }
-
-    /**
-     * Set amount of time to delay writes for in writer failure injection. 0 disables write delay.
-     *
-     * @param delayMs
-     *          ms to delay writes for.
-     * @return dl configuration.
-     */
-    public DistributedLogConfiguration setEIInjectedWriteDelayMs(int delayMs) {
-        setProperty(BKDL_EI_INJECTED_WRITE_DELAY_MS, delayMs);
-        return this;
-    }
-
-    /**
-     * Get the flag whether to inject stalls in read ahead.
-     *
-     * @return true if to inject stalls in read ahead, otherwise false.
-     */
-    public boolean getEIInjectReadAheadStall() {
-        return getBoolean(BKDL_EI_INJECT_READAHEAD_STALL, BKDL_EI_INJECT_READAHEAD_STALL_DEFAULT);
-    }
-
-    /**
-     * Set the flag whether to inject stalls in read ahead.
-     *
-     * @param enabled
-     *          flag to inject stalls in read ahead.
-     * @return distributedlog configuration.
-     */
-    public DistributedLogConfiguration setEIInjectReadAheadStall(boolean enabled) {
-        setProperty(BKDL_EI_INJECT_READAHEAD_STALL, enabled);
-        return this;
-    }
-
-    /**
-     * Get the flag whether to inject broken entries in readahead.
-     *
-     * @return true if to inject corruption in read ahead, otherwise false.
-     */
-    public boolean getEIInjectReadAheadBrokenEntries() {
-        return getBoolean(BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES, BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES_DEFAULT);
-    }
-
-    /**
-     * Set the flag whether to inject broken entries in read ahead.
-     *
-     * @param enabled
-     *          flag to inject corruption in read ahead.
-     * @return distributedlog configuration.
-     */
-    public DistributedLogConfiguration setEIInjectReadAheadBrokenEntries(boolean enabled) {
-        setProperty(BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES, enabled);
-        return this;
-    }
-
-    /**
-     * Get the flag whether to inject delay in read ahead.
-     *
-     * @return true if to inject delays in read ahead, otherwise false.
-     */
-    public boolean getEIInjectReadAheadDelay() {
-        return getBoolean(BKDL_EI_INJECT_READAHEAD_DELAY, BKDL_EI_INJECT_READAHEAD_DELAY_DEFAULT);
-    }
-
-    /**
-     * Set the flag whether to inject delays in read ahead.
-     *
-     * @param enabled
-     *          flag to inject delays in read ahead.
-     * @return distributedlog configuration.
-     */
-    public DistributedLogConfiguration setEIInjectReadAheadDelay(boolean enabled) {
-        setProperty(BKDL_EI_INJECT_READAHEAD_DELAY, enabled);
-        return this;
-    }
-
-    /**
-     * Get the max injected delay in read ahead, in millis.
-     *
-     * @return max injected delay in read ahead, in millis.
-     */
-    public int getEIInjectMaxReadAheadDelayMs() {
-        return getInt(BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS, BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS_DEFAULT);
-    }
-
-    /**
-     * Set the max injected delay in read ahead, in millis.
-     *
-     * @param delayMs
-     *          max injected delay in read ahead, in millis.
-     * @return distributedlog configuration.
-     */
-    public DistributedLogConfiguration setEIInjectMaxReadAheadDelayMs(int delayMs) {
-        setProperty(BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS, delayMs);
-        return this;
-    }
-
-    /**
-     * Get the percentage of operations to delay in read ahead.
-     *
-     * @return the percentage of operations to delay in read ahead.
-     */
-    public int getEIInjectReadAheadDelayPercent() {
-        return getInt(BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT, BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT_DEFAULT);
-    }
-
-    /**
-     * Set the percentage of operations to delay in read ahead.
-     *
-     * @param percent
-     *          the percentage of operations to delay in read ahead.
-     * @return distributedlog configuration
-     */
-    public DistributedLogConfiguration setEIInjectReadAheadDelayPercent(int percent) {
-        setProperty(BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT, percent);
-        return this;
-    }
-
-    /**
-     * Validate the configuration
-     */
-    public void validate() {
-        Preconditions.checkArgument(getBKClientReadTimeout() * 1000 >= getReadLACLongPollTimeout(),
-            "Invalid timeout configuration: bkcReadTimeoutSeconds ("+getBKClientReadTimeout()+
-                ") should be longer than readLACLongPollTimeout ("+getReadLACLongPollTimeout()+")");
-        long readerIdleWarnThresholdMs = getReaderIdleWarnThresholdMillis();
-        if (readerIdleWarnThresholdMs > 0) { // NOTE: some test cases set the idle warn threshold to 0
-            Preconditions.checkArgument(readerIdleWarnThresholdMs > 2 * getReadLACLongPollTimeout(),
-                    "Invalid configuration: ReaderIdleWarnThreshold should be 2x larget than readLACLongPollTimeout");
-        }
-    }
-
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConstants.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConstants.java
deleted file mode 100644
index e798a0f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConstants.java
+++ /dev/null
@@ -1,69 +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 com.twitter.distributedlog;
-
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import com.google.common.collect.ImmutableList;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.data.ACL;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-public class DistributedLogConstants {
-    public static final byte[] EMPTY_BYTES = new byte[0];
-    public static final String SCHEME_PREFIX = "distributedlog";
-    public static final String BACKEND_BK = "bk";
-    public static final long INVALID_TXID = -999;
-    public static final long EMPTY_LOGSEGMENT_TX_ID = -99;
-    public static final long MAX_TXID = Long.MAX_VALUE;
-    public static final long SMALL_LOGSEGMENT_THRESHOLD = 10;
-    public static final int LOGSEGMENT_NAME_VERSION = 1;
-    public static final int FUTURE_TIMEOUT_IMMEDIATE = 0;
-    public static final int FUTURE_TIMEOUT_INFINITE = -1;
-    public static final long LOCK_IMMEDIATE = FUTURE_TIMEOUT_IMMEDIATE;
-    public static final long LOCK_TIMEOUT_INFINITE = FUTURE_TIMEOUT_INFINITE;
-    public static final long LOCK_OP_TIMEOUT_DEFAULT = 120;
-    public static final long LOCK_REACQUIRE_TIMEOUT_DEFAULT = 120;
-    public static final String UNKNOWN_CLIENT_ID = "Unknown-ClientId";
-    public static final int LOCAL_REGION_ID = 0;
-    public static final long LOGSEGMENT_DEFAULT_STATUS = 0;
-    public static final long UNASSIGNED_LOGSEGMENT_SEQNO = 0;
-    public static final long UNASSIGNED_SEQUENCE_ID = -1L;
-    public static final long FIRST_LOGSEGMENT_SEQNO = 1;
-    public static final long UNRESOLVED_LEDGER_ID = -1;
-    public static final long LATENCY_WARN_THRESHOLD_IN_MILLIS = TimeUnit.SECONDS.toMillis(1);
-    public static final int DL_INTERRUPTED_EXCEPTION_RESULT_CODE = Integer.MIN_VALUE + 1;
-    public static final int ZK_CONNECTION_EXCEPTION_RESULT_CODE = Integer.MIN_VALUE + 2;
-
-    public static final String ALLOCATION_POOL_NODE = ".allocation_pool";
-    // log segment prefix
-    public static final String INPROGRESS_LOGSEGMENT_PREFIX = "inprogress";
-    public static final String COMPLETED_LOGSEGMENT_PREFIX = "logrecs";
-    public static final String DISALLOW_PLACEMENT_IN_REGION_FEATURE_NAME = "disallow_bookie_placement";
-    static final byte[] CONTROL_RECORD_CONTENT = "control".getBytes(UTF_8);
-    static final byte[] KEEPALIVE_RECORD_CONTENT = "keepalive".getBytes(UTF_8);
-
-    // An ACL that gives all permissions to node creators and read permissions only to everyone else.
-    public static final List<ACL> EVERYONE_READ_CREATOR_ALL =
-        ImmutableList.<ACL>builder()
-            .addAll(Ids.CREATOR_ALL_ACL)
-            .addAll(Ids.READ_ACL_UNSAFE)
-            .build();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java
deleted file mode 100644
index 34cfb65..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java
+++ /dev/null
@@ -1,308 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.callback.LogSegmentListener;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.namespace.NamespaceDriver;
-import com.twitter.distributedlog.subscription.SubscriptionStateStore;
-import com.twitter.distributedlog.subscription.SubscriptionsStore;
-import com.twitter.util.Future;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-
-/**
- * A DistributedLogManager is responsible for managing a single place of storing
- * edit logs. It may correspond to multiple files, a backup node, etc.
- * Even when the actual underlying storage is rolled, or failed and restored,
- * 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 {
-
-    /**
-     * Get the name of the stream managed by this log manager
-     * @return streamName
-     */
-    public String getStreamName();
-
-    /**
-     * Get the namespace driver used by this manager.
-     *
-     * @return the namespace driver
-     */
-    public NamespaceDriver getNamespaceDriver();
-
-    /**
-     * Get log segments.
-     *
-     * @return log segments
-     * @throws IOException
-     */
-    public List<LogSegmentMetadata> getLogSegments() throws IOException;
-
-    /**
-     * Register <i>listener</i> on log segment updates of this stream.
-     *
-     * @param listener
-     *          listener to receive update log segment list.
-     */
-    public void registerListener(LogSegmentListener listener) throws IOException ;
-
-    /**
-     * Unregister <i>listener</i> on log segment updates from this stream.
-     *
-     * @param listener
-     *          listener to receive update log segment list.
-     */
-    public 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();
-
-    /**
-     * Begin writing to the log stream identified by the name
-     *
-     * @return the writer interface to generate log records
-     */
-    public LogWriter startLogSegmentNonPartitioned() throws IOException;
-
-    /**
-     * Begin writing to the log stream identified by the name
-     *
-     * @return the writer interface to generate log records
-     */
-    // @Deprecated
-    public AsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException;
-
-    /**
-     * 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;
-
-    /**
-     * 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;
-
-    /**
-     * 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)
-        throws IOException;
-
-    public LogReader getInputStream(DLSN fromDLSN) throws IOException;
-
-    /**
-     * Open an async log reader to read records from a log starting from <code>fromTxnId</code>.
-     *
-     * @param fromTxnId
-     *          transaction id to start reading from
-     * @return async log reader
-     */
-    public Future<AsyncLogReader> openAsyncLogReader(long fromTxnId);
-
-    /**
-     * 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);
-
-    // @Deprecated
-    public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException;
-
-    // @Deprecated
-    public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException;
-
-    public Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN);
-
-    /**
-     * Get a log reader with lock starting from <i>fromDLSN</i> and using <i>subscriberId</i>.
-     * If two readers tried to open using same subscriberId, one would succeed, while the other
-     * will be blocked until it gets the lock.
-     *
-     * @param fromDLSN
-     *          start dlsn
-     * @param subscriberId
-     *          subscriber id
-     * @return async log reader
-     */
-    public 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
-     * will be blocked until it gets the lock.
-     *
-     * @param subscriberId
-     *          subscriber id
-     * @return async log reader
-     */
-    public Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId);
-
-    /**
-     * Get the {@link DLSN} of first log record whose transaction id is not less than <code>transactionId</code>.
-     *
-     * @param transactionId
-     *          transaction id
-     * @return dlsn of first log record whose transaction id is not less than transactionId.
-     */
-    public Future<DLSN> getDLSNNotLessThanTxId(long transactionId);
-
-    /**
-     * 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()
-        throws IOException;
-
-    /**
-     * Get the earliest Transaction Id available in the log
-     *
-     * @return earliest transaction id
-     * @throws IOException
-     */
-    public long getFirstTxId() throws IOException;
-
-    /**
-     * Get Latest Transaction Id in the log
-     *
-     * @return latest transaction id
-     * @throws IOException
-     */
-    public long getLastTxId() throws IOException;
-
-    /**
-     * Get Latest DLSN in the log
-     *
-     * @return last dlsn
-     * @throws IOException
-     */
-    public DLSN getLastDLSN() throws IOException;
-
-    /**
-     * Get Latest log record with DLSN in the log - async
-     *
-     * @return latest log record with DLSN
-     */
-    public Future<LogRecordWithDLSN> getLastLogRecordAsync();
-
-    /**
-     * Get Latest Transaction Id in the log - async
-     *
-     * @return latest transaction id
-     */
-    public Future<Long> getLastTxIdAsync();
-
-    /**
-     * Get first DLSN in the log.
-     *
-     * @return first dlsn in the stream
-     */
-    public Future<DLSN> getFirstDLSNAsync();
-
-    /**
-     * Get Latest DLSN in the log - async
-     *
-     * @return latest transaction id
-     */
-    public 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
-     *
-     * @return number of log records
-     * @throws IOException
-     */
-    public 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
-     *
-     * @return future number of log records
-     * @throws IOException
-     */
-    public Future<Long> getLogRecordCountAsync(final DLSN beginDLSN);
-
-    /**
-     * Run recovery on the log.
-     *
-     * @throws IOException
-     */
-    public 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
-     *
-     * @return true if the marker was added to the stream, false otherwise
-     * @throws IOException
-     */
-    public boolean isEndOfStreamMarked() throws IOException;
-
-    /**
-     * Delete the log.
-     *
-     * @throws IOException if the deletion fails
-     */
-    public void delete() throws IOException;
-
-    /**
-     * The DistributedLogManager may archive/purge any logs for transactionId
-     * less than or equal to minImageTxId.
-     * This is to be used only when the client explicitly manages deletion. If
-     * the cleanup policy is based on sliding time window, then this method need
-     * not be called.
-     *
-     * @param minTxIdToKeep the earliest txid that must be retained
-     * @throws IOException if purging fails
-     */
-    public 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();
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/Entry.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/Entry.java
deleted file mode 100644
index bf315fc..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/Entry.java
+++ /dev/null
@@ -1,403 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-import javax.annotation.Nullable;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * A set of {@link LogRecord}s.
- */
-public class Entry {
-
-    /**
-     * Create a new log record set.
-     *
-     * @param logName
-     *          name of the log
-     * @param initialBufferSize
-     *          initial buffer size
-     * @param envelopeBeforeTransmit
-     *          if envelope the buffer before transmit
-     * @param codec
-     *          compression codec
-     * @param statsLogger
-     *          stats logger to receive stats
-     * @return writer to build a log record set.
-     */
-    public static Writer newEntry(
-            String logName,
-            int initialBufferSize,
-            boolean envelopeBeforeTransmit,
-            CompressionCodec.Type codec,
-            StatsLogger statsLogger) {
-        return new EnvelopedEntryWriter(
-                logName,
-                initialBufferSize,
-                envelopeBeforeTransmit,
-                codec,
-                statsLogger);
-    }
-
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    /**
-     * Build the record set object.
-     */
-    public static class Builder {
-
-        private long logSegmentSequenceNumber = -1;
-        private long entryId = -1;
-        private long startSequenceId = Long.MIN_VALUE;
-        private boolean envelopeEntry = true;
-        // input stream
-        private InputStream in = null;
-        // or bytes array
-        private byte[] data = null;
-        private int offset = -1;
-        private int length = -1;
-        private Optional<Long> txidToSkipTo = Optional.absent();
-        private Optional<DLSN> dlsnToSkipTo = Optional.absent();
-        private boolean deserializeRecordSet = true;
-
-        private Builder() {}
-
-        /**
-         * Reset the builder.
-         *
-         * @return builder
-         */
-        public Builder reset() {
-            logSegmentSequenceNumber = -1;
-            entryId = -1;
-            startSequenceId = Long.MIN_VALUE;
-            envelopeEntry = true;
-            // input stream
-            in = null;
-            // or bytes array
-            data = null;
-            offset = -1;
-            length = -1;
-            txidToSkipTo = Optional.absent();
-            dlsnToSkipTo = Optional.absent();
-            return this;
-        }
-
-        /**
-         * Set the segment info of the log segment that this record
-         * set belongs to.
-         *
-         * @param lssn
-         *          log segment sequence number
-         * @param startSequenceId
-         *          start sequence id of this log segment
-         * @return builder
-         */
-        public Builder setLogSegmentInfo(long lssn, long startSequenceId) {
-            this.logSegmentSequenceNumber = lssn;
-            this.startSequenceId = startSequenceId;
-            return this;
-        }
-
-        /**
-         * Set the entry id of this log record set.
-         *
-         * @param entryId
-         *          entry id assigned for this log record set.
-         * @return builder
-         */
-        public Builder setEntryId(long entryId) {
-            this.entryId = entryId;
-            return this;
-        }
-
-        /**
-         * Set whether this record set is enveloped or not.
-         *
-         * @param enabled
-         *          flag indicates whether this record set is enveloped or not.
-         * @return builder
-         */
-        public Builder setEnvelopeEntry(boolean enabled) {
-            this.envelopeEntry = enabled;
-            return this;
-        }
-
-        /**
-         * Set the serialized bytes data of this record set.
-         *
-         * @param data
-         *          serialized bytes data of this record set.
-         * @param offset
-         *          offset of the bytes data
-         * @param length
-         *          length of the bytes data
-         * @return builder
-         */
-        public Builder setData(byte[] data, int offset, int length) {
-            this.data = data;
-            this.offset = offset;
-            this.length = length;
-            return this;
-        }
-
-        /**
-         * Set the input stream of the serialized bytes data of this record set.
-         *
-         * @param in
-         *          input stream
-         * @return builder
-         */
-        public Builder setInputStream(InputStream in) {
-            this.in = in;
-            return this;
-        }
-
-        /**
-         * Set the record set starts from <code>dlsn</code>.
-         *
-         * @param dlsn
-         *          dlsn to skip to
-         * @return builder
-         */
-        public Builder skipTo(@Nullable DLSN dlsn) {
-            this.dlsnToSkipTo = Optional.fromNullable(dlsn);
-            return this;
-        }
-
-        /**
-         * Set the record set starts from <code>txid</code>.
-         *
-         * @param txid
-         *          txid to skip to
-         * @return builder
-         */
-        public Builder skipTo(long txid) {
-            this.txidToSkipTo = Optional.of(txid);
-            return this;
-        }
-
-        /**
-         * Enable/disable deserialize record set.
-         *
-         * @param enabled
-         *          flag to enable/disable dserialize record set.
-         * @return builder
-         */
-        public Builder deserializeRecordSet(boolean enabled) {
-            this.deserializeRecordSet = enabled;
-            return this;
-        }
-
-        public Entry build() {
-            Preconditions.checkNotNull(data, "Serialized data isn't provided");
-            Preconditions.checkArgument(offset >= 0 && length >= 0
-                    && (offset + length) <= data.length,
-                    "Invalid offset or length of serialized data");
-            return new Entry(
-                    logSegmentSequenceNumber,
-                    entryId,
-                    startSequenceId,
-                    envelopeEntry,
-                    deserializeRecordSet,
-                    data,
-                    offset,
-                    length,
-                    txidToSkipTo,
-                    dlsnToSkipTo);
-        }
-
-        public Entry.Reader buildReader() throws IOException {
-            Preconditions.checkArgument(data != null || in != null,
-                    "Serialized data or input stream isn't provided");
-            InputStream in;
-            if (null != this.in) {
-                in = this.in;
-            } else {
-                Preconditions.checkArgument(offset >= 0 && length >= 0
-                                && (offset + length) <= data.length,
-                        "Invalid offset or length of serialized data");
-                in = new ByteArrayInputStream(data, offset, length);
-            }
-            return new EnvelopedEntryReader(
-                    logSegmentSequenceNumber,
-                    entryId,
-                    startSequenceId,
-                    in,
-                    envelopeEntry,
-                    deserializeRecordSet,
-                    NullStatsLogger.INSTANCE);
-        }
-
-    }
-
-    private final long logSegmentSequenceNumber;
-    private final long entryId;
-    private final long startSequenceId;
-    private final boolean envelopedEntry;
-    private final boolean deserializeRecordSet;
-    private final byte[] data;
-    private final int offset;
-    private final int length;
-    private final Optional<Long> txidToSkipTo;
-    private final Optional<DLSN> dlsnToSkipTo;
-
-    private Entry(long logSegmentSequenceNumber,
-                  long entryId,
-                  long startSequenceId,
-                  boolean envelopedEntry,
-                  boolean deserializeRecordSet,
-                  byte[] data,
-                  int offset,
-                  int length,
-                  Optional<Long> txidToSkipTo,
-                  Optional<DLSN> dlsnToSkipTo) {
-        this.logSegmentSequenceNumber = logSegmentSequenceNumber;
-        this.entryId = entryId;
-        this.startSequenceId = startSequenceId;
-        this.envelopedEntry = envelopedEntry;
-        this.deserializeRecordSet = deserializeRecordSet;
-        this.data = data;
-        this.offset = offset;
-        this.length = length;
-        this.txidToSkipTo = txidToSkipTo;
-        this.dlsnToSkipTo = dlsnToSkipTo;
-    }
-
-    /**
-     * Get raw data of this record set.
-     *
-     * @return raw data representation of this record set.
-     */
-    public byte[] getRawData() {
-        return data;
-    }
-
-    /**
-     * Create reader to iterate over this record set.
-     *
-     * @return reader to iterate over this record set.
-     * @throws IOException if the record set is invalid record set.
-     */
-    public Reader reader() throws IOException {
-        InputStream in = new ByteArrayInputStream(data, offset, length);
-        Reader reader = new EnvelopedEntryReader(
-                logSegmentSequenceNumber,
-                entryId,
-                startSequenceId,
-                in,
-                envelopedEntry,
-                deserializeRecordSet,
-                NullStatsLogger.INSTANCE);
-        if (txidToSkipTo.isPresent()) {
-            reader.skipTo(txidToSkipTo.get());
-        }
-        if (dlsnToSkipTo.isPresent()) {
-            reader.skipTo(dlsnToSkipTo.get());
-        }
-        return reader;
-    }
-
-    /**
-     * Writer to append {@link LogRecord}s to {@link Entry}.
-     */
-    public interface Writer extends EntryBuffer {
-
-        /**
-         * Write a {@link LogRecord} to this record set.
-         *
-         * @param record
-         *          record to write
-         * @param transmitPromise
-         *          callback for transmit result. the promise is only
-         *          satisfied when this record set is transmitted.
-         * @throws LogRecordTooLongException if the record is too long
-         * @throws WriteException when encountered exception writing the record
-         */
-        void writeRecord(LogRecord record, Promise<DLSN> transmitPromise)
-                throws LogRecordTooLongException, WriteException;
-
-        /**
-         * Reset the writer to write records.
-         */
-        void reset();
-
-    }
-
-    /**
-     * Reader to read {@link LogRecord}s from this record set.
-     */
-    public interface Reader {
-
-        /**
-         * Get the log segment sequence number.
-         *
-         * @return the log segment sequence number.
-         */
-        long getLSSN();
-
-        /**
-         * Return the entry id.
-         *
-         * @return the entry id.
-         */
-        long getEntryId();
-
-        /**
-         * Read next log record from this record set.
-         *
-         * @return next log record from this record set.
-         */
-        LogRecordWithDLSN nextRecord() throws IOException;
-
-        /**
-         * Skip the reader to the record whose transaction id is <code>txId</code>.
-         *
-         * @param txId
-         *          transaction id to skip to.
-         * @return true if skip succeeds, otherwise false.
-         * @throws IOException
-         */
-        boolean skipTo(long txId) throws IOException;
-
-        /**
-         * Skip the reader to the record whose DLSN is <code>dlsn</code>.
-         *
-         * @param dlsn
-         *          DLSN to skip to.
-         * @return true if skip succeeds, otherwise false.
-         * @throws IOException
-         */
-        boolean skipTo(DLSN dlsn) throws IOException;
-
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryBuffer.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryBuffer.java
deleted file mode 100644
index 394fbad..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryBuffer.java
+++ /dev/null
@@ -1,70 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.exceptions.InvalidEnvelopedEntryException;
-import com.twitter.distributedlog.io.Buffer;
-import com.twitter.distributedlog.io.TransmitListener;
-
-import java.io.IOException;
-
-/**
- * Write representation of a {@link Entry}.
- * It is a buffer of log record set, used for transmission.
- */
-public interface EntryBuffer extends TransmitListener {
-
-    /**
-     * Return if this record set contains user records.
-     *
-     * @return true if this record set contains user records, otherwise
-     * return false.
-     */
-    boolean hasUserRecords();
-
-    /**
-     * Return number of records in current record set.
-     *
-     * @return number of records in current record set.
-     */
-    int getNumRecords();
-
-    /**
-     * Return number of bytes in current record set.
-     *
-     * @return number of bytes in current record set.
-     */
-    int getNumBytes();
-
-    /**
-     * Return max tx id in current record set.
-     *
-     * @return max tx id.
-     */
-    long getMaxTxId();
-
-    /**
-     * Get the buffer to transmit.
-     *
-     * @return the buffer to transmit.
-     * @throws InvalidEnvelopedEntryException if the record set buffer is invalid
-     * @throws IOException when encountered IOException during serialization
-     */
-    Buffer getBuffer() throws InvalidEnvelopedEntryException, IOException;
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryPosition.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryPosition.java
deleted file mode 100644
index 0a15d29..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryPosition.java
+++ /dev/null
@@ -1,63 +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 com.twitter.distributedlog;
-
-/**
- * The position of an entry, identified by log segment sequence number and entry id.
- */
-class EntryPosition {
-
-    private long lssn;
-    private long entryId;
-
-    EntryPosition(long lssn, long entryId) {
-        this.lssn = lssn;
-        this.entryId = entryId;
-    }
-
-    public synchronized long getLogSegmentSequenceNumber() {
-        return lssn;
-    }
-
-    public synchronized long getEntryId() {
-        return entryId;
-    }
-
-    public synchronized boolean advance(long lssn, long entryId) {
-        if (lssn == this.lssn) {
-            if (entryId <= this.entryId) {
-                return false;
-            }
-            this.entryId = entryId;
-            return true;
-        } else if (lssn > this.lssn) {
-            this.lssn = lssn;
-            this.entryId = entryId;
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("(").append(lssn).append(", ").append(entryId).append(")");
-        return sb.toString();
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntry.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntry.java
deleted file mode 100644
index 55d3be9..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntry.java
+++ /dev/null
@@ -1,296 +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 com.twitter.distributedlog;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-import com.google.common.base.Preconditions;
-
-import com.twitter.distributedlog.exceptions.InvalidEnvelopedEntryException;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-import com.twitter.distributedlog.annotations.DistributedLogAnnotations.Compression;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.distributedlog.io.CompressionUtils;
-import com.twitter.distributedlog.util.BitMaskUtils;
-
-/**
- * An enveloped entry written to BookKeeper.
- *
- * Data type in brackets. Interpretation should be on the basis of data types and not individual
- * bytes to honor Endianness.
- *
- * Entry Structure:
- * ---------------
- * Bytes 0                                  : Version (Byte)
- * Bytes 1 - (DATA = 1+Header.length-1)     : Header (Integer)
- * Bytes DATA - DATA+3                      : Payload Length (Integer)
- * BYTES DATA+4 - DATA+4+payload.length-1   : Payload (Byte[])
- *
- * V1 Header Structure: // Offsets relative to the start of the header.
- * -------------------
- * Bytes 0 - 3                              : Flags (Integer)
- * Bytes 4 - 7                              : Original payload size before compression (Integer)
- *
- *      Flags: // 32 Bits
- *      -----
- *      0 ... 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
- *                                      |_|
- *                                       |
- *                               Compression Type
- *
- *      Compression Type: // 2 Bits (Least significant)
- *      ----------------
- *      00      : No Compression
- *      01      : LZ4 Compression
- *      10      : Unused
- *      11      : Unused
- */
-public class EnvelopedEntry {
-
-    public static final int VERSION_LENGTH = 1; // One byte long
-    public static final byte VERSION_ONE = 1;
-
-    public static final byte LOWEST_SUPPORTED_VERSION = VERSION_ONE;
-    public static final byte HIGHEST_SUPPORTED_VERSION = VERSION_ONE;
-    public static final byte CURRENT_VERSION = VERSION_ONE;
-
-    private final OpStatsLogger compressionStat;
-    private final OpStatsLogger decompressionStat;
-    private final Counter compressedEntryBytes;
-    private final Counter decompressedEntryBytes;
-    private final byte version;
-
-    private Header header = new Header();
-    private Payload payloadCompressed = new Payload();
-    private Payload payloadDecompressed = new Payload();
-
-    public EnvelopedEntry(byte version,
-                          StatsLogger statsLogger) throws InvalidEnvelopedEntryException {
-        Preconditions.checkNotNull(statsLogger);
-        if (version < LOWEST_SUPPORTED_VERSION || version > HIGHEST_SUPPORTED_VERSION) {
-            throw new InvalidEnvelopedEntryException("Invalid enveloped entry version " + version + ", expected to be in [ "
-                    + LOWEST_SUPPORTED_VERSION + " ~ " + HIGHEST_SUPPORTED_VERSION + " ]");
-        }
-        this.version = version;
-        this.compressionStat = statsLogger.getOpStatsLogger("compression_time");
-        this.decompressionStat = statsLogger.getOpStatsLogger("decompression_time");
-        this.compressedEntryBytes = statsLogger.getCounter("compressed_bytes");
-        this.decompressedEntryBytes = statsLogger.getCounter("decompressed_bytes");
-    }
-
-    /**
-     * @param statsLogger
-     *          Used for getting stats for (de)compression time
-     * @param compressionType
-     *          The compression type to use
-     * @param decompressed
-     *          The decompressed payload
-     *          NOTE: The size of the byte array passed as the decompressed payload can be larger
-     *                than the actual contents to be compressed.
-     */
-    public EnvelopedEntry(byte version,
-                          CompressionCodec.Type compressionType,
-                          byte[] decompressed,
-                          int length,
-                          StatsLogger statsLogger)
-            throws InvalidEnvelopedEntryException {
-        this(version, statsLogger);
-        Preconditions.checkNotNull(compressionType);
-        Preconditions.checkNotNull(decompressed);
-        Preconditions.checkArgument(length >= 0, "Invalid bytes length " + length);
-
-        this.header = new Header(compressionType, length);
-        this.payloadDecompressed = new Payload(length, decompressed);
-    }
-
-    private boolean isReady() {
-        return (header.ready && payloadDecompressed.ready);
-    }
-
-    @Compression
-    public void writeFully(DataOutputStream out) throws IOException {
-        Preconditions.checkNotNull(out);
-        if (!isReady()) {
-            throw new IOException("Entry not writable");
-        }
-        // Version
-        out.writeByte(version);
-        // Header
-        header.write(out);
-        // Compress
-        CompressionCodec codec = CompressionUtils.getCompressionCodec(header.compressionType);
-        byte[] compressed = codec.compress(
-                payloadDecompressed.payload,
-                0,
-                payloadDecompressed.length,
-                compressionStat);
-        this.payloadCompressed = new Payload(compressed.length, compressed);
-        this.compressedEntryBytes.add(payloadCompressed.length);
-        this.decompressedEntryBytes.add(payloadDecompressed.length);
-        payloadCompressed.write(out);
-    }
-
-    @Compression
-    public void readFully(DataInputStream in) throws IOException {
-        Preconditions.checkNotNull(in);
-        // Make sure we're reading the right versioned entry.
-        byte version = in.readByte();
-        if (version != this.version) {
-            throw new IOException(String.format("Version mismatch while reading. Received: %d," +
-                    " Required: %d", version, this.version));
-        }
-        header.read(in);
-        payloadCompressed.read(in);
-        // Decompress
-        CompressionCodec codec = CompressionUtils.getCompressionCodec(header.compressionType);
-        byte[] decompressed = codec.decompress(
-                payloadCompressed.payload,
-                0,
-                payloadCompressed.length,
-                header.decompressedSize,
-                decompressionStat);
-        this.payloadDecompressed = new Payload(decompressed.length, decompressed);
-        this.compressedEntryBytes.add(payloadCompressed.length);
-        this.decompressedEntryBytes.add(payloadDecompressed.length);
-    }
-
-    public byte[] getDecompressedPayload() throws IOException {
-        if (!isReady()) {
-            throw new IOException("Decompressed payload is not initialized");
-        }
-        return payloadDecompressed.payload;
-    }
-
-    public static class Header {
-        public static final int COMPRESSION_CODEC_MASK = 0x3;
-        public static final int COMPRESSION_CODEC_NONE = 0x0;
-        public static final int COMPRESSION_CODEC_LZ4 = 0x1;
-
-        private int flags = 0;
-        private int decompressedSize = 0;
-        private CompressionCodec.Type compressionType = CompressionCodec.Type.UNKNOWN;
-
-        // Whether this struct is ready for reading/writing.
-        private boolean ready = false;
-
-        // Used while reading.
-        public Header() {
-        }
-
-        public Header(CompressionCodec.Type compressionType,
-                      int decompressedSize) {
-            this.compressionType = compressionType;
-            this.decompressedSize = decompressedSize;
-            this.flags = 0;
-            switch (compressionType) {
-                case NONE:
-                    this.flags = (int) BitMaskUtils.set(flags, COMPRESSION_CODEC_MASK,
-                                                        COMPRESSION_CODEC_NONE);
-                    break;
-                case LZ4:
-                    this.flags = (int) BitMaskUtils.set(flags, COMPRESSION_CODEC_MASK,
-                                                        COMPRESSION_CODEC_LZ4);
-                    break;
-                default:
-                    throw new RuntimeException(String.format("Unknown Compression Type: %s",
-                                                             compressionType));
-            }
-            // This can now be written.
-            this.ready = true;
-        }
-
-        private void write(DataOutputStream out) throws IOException {
-            out.writeInt(flags);
-            out.writeInt(decompressedSize);
-        }
-
-        private void read(DataInputStream in) throws IOException {
-            this.flags = in.readInt();
-            int compressionType = (int) BitMaskUtils.get(flags, COMPRESSION_CODEC_MASK);
-            if (compressionType == COMPRESSION_CODEC_NONE) {
-                this.compressionType = CompressionCodec.Type.NONE;
-            } else if (compressionType == COMPRESSION_CODEC_LZ4) {
-                this.compressionType = CompressionCodec.Type.LZ4;
-            } else {
-                throw new IOException(String.format("Unsupported Compression Type: %s",
-                                                    compressionType));
-            }
-            this.decompressedSize = in.readInt();
-            // Values can now be read.
-            this.ready = true;
-        }
-    }
-
-    public static class Payload {
-        private int length = 0;
-        private byte[] payload = null;
-
-        // Whether this struct is ready for reading/writing.
-        private boolean ready = false;
-
-        // Used for reading
-        Payload() {
-        }
-
-        Payload(int length, byte[] payload) {
-            this.length = length;
-            this.payload = payload;
-            this.ready = true;
-        }
-
-        private void write(DataOutputStream out) throws IOException {
-            out.writeInt(length);
-            out.write(payload, 0, length);
-        }
-
-        private void read(DataInputStream in) throws IOException {
-            this.length = in.readInt();
-            this.payload = new byte[length];
-            in.readFully(payload);
-            this.ready = true;
-        }
-    }
-
-    /**
-     * Return an InputStream that reads from the provided InputStream, decompresses the data
-     * and returns a new InputStream wrapping the underlying payload.
-     *
-     * Note that src is modified by this call.
-     *
-     * @return
-     *      New Input stream with the underlying payload.
-     * @throws Exception
-     */
-    public static InputStream fromInputStream(InputStream src,
-                                              StatsLogger statsLogger) throws IOException {
-        src.mark(VERSION_LENGTH);
-        byte version = new DataInputStream(src).readByte();
-        src.reset();
-        EnvelopedEntry entry = new EnvelopedEntry(version, statsLogger);
-        entry.readFully(new DataInputStream(src));
-        return new ByteArrayInputStream(entry.getDecompressedPayload());
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryReader.java
deleted file mode 100644
index 038bb18..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryReader.java
+++ /dev/null
@@ -1,102 +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 com.twitter.distributedlog;
-
-import org.apache.bookkeeper.stats.StatsLogger;
-
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-
-/**
- * Record reader to read records from an enveloped entry buffer.
- */
-class EnvelopedEntryReader implements Entry.Reader, RecordStream {
-
-    private final long logSegmentSeqNo;
-    private final long entryId;
-    private final LogRecord.Reader reader;
-
-    // slot id
-    private long slotId = 0;
-
-    EnvelopedEntryReader(long logSegmentSeqNo,
-                         long entryId,
-                         long startSequenceId,
-                         InputStream in,
-                         boolean envelopedEntry,
-                         boolean deserializeRecordSet,
-                         StatsLogger statsLogger)
-            throws IOException {
-        this.logSegmentSeqNo = logSegmentSeqNo;
-        this.entryId = entryId;
-        InputStream src = in;
-        if (envelopedEntry) {
-            src = EnvelopedEntry.fromInputStream(in, statsLogger);
-        }
-        this.reader = new LogRecord.Reader(
-                this,
-                new DataInputStream(src),
-                startSequenceId,
-                deserializeRecordSet);
-    }
-
-    @Override
-    public long getLSSN() {
-        return logSegmentSeqNo;
-    }
-
-    @Override
-    public long getEntryId() {
-        return entryId;
-    }
-
-    @Override
-    public LogRecordWithDLSN nextRecord() throws IOException {
-        return reader.readOp();
-    }
-
-    @Override
-    public boolean skipTo(long txId) throws IOException {
-        return reader.skipTo(txId, true);
-    }
-
-    @Override
-    public boolean skipTo(DLSN dlsn) throws IOException {
-        return reader.skipTo(dlsn);
-    }
-
-    //
-    // Record Stream
-    //
-
-    @Override
-    public void advance(int numRecords) {
-        slotId += numRecords;
-    }
-
-    @Override
-    public DLSN getCurrentPosition() {
-        return new DLSN(logSegmentSeqNo, entryId, slotId);
-    }
-
-    @Override
-    public String getName() {
-        return "EnvelopedReader";
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryWriter.java
deleted file mode 100644
index 01a91ab..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryWriter.java
+++ /dev/null
@@ -1,192 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.Entry.Writer;
-import com.twitter.distributedlog.exceptions.InvalidEnvelopedEntryException;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.WriteCancelledException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.io.Buffer;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.util.LinkedList;
-import java.util.List;
-
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-
-/**
- * {@link com.twitter.distributedlog.io.Buffer} based log record set writer.
- */
-class EnvelopedEntryWriter implements Writer {
-
-    static final Logger logger = LoggerFactory.getLogger(EnvelopedEntryWriter.class);
-
-    private static class WriteRequest {
-
-        private final int numRecords;
-        private final Promise<DLSN> promise;
-
-        WriteRequest(int numRecords, Promise<DLSN> promise) {
-            this.numRecords = numRecords;
-            this.promise = promise;
-        }
-
-    }
-
-    private final String logName;
-    private final Buffer buffer;
-    private final LogRecord.Writer writer;
-    private final List<WriteRequest> writeRequests;
-    private final boolean envelopeBeforeTransmit;
-    private final CompressionCodec.Type codec;
-    private final StatsLogger statsLogger;
-    private int count = 0;
-    private boolean hasUserData = false;
-    private long maxTxId = Long.MIN_VALUE;
-
-    EnvelopedEntryWriter(String logName,
-                         int initialBufferSize,
-                         boolean envelopeBeforeTransmit,
-                         CompressionCodec.Type codec,
-                         StatsLogger statsLogger) {
-        this.logName = logName;
-        this.buffer = new Buffer(initialBufferSize * 6 / 5);
-        this.writer = new LogRecord.Writer(new DataOutputStream(buffer));
-        this.writeRequests = new LinkedList<WriteRequest>();
-        this.envelopeBeforeTransmit = envelopeBeforeTransmit;
-        this.codec = codec;
-        this.statsLogger = statsLogger;
-    }
-
-    @Override
-    public synchronized void reset() {
-        cancelPromises(new WriteCancelledException(logName, "Record Set is reset"));
-        count = 0;
-        this.buffer.reset();
-    }
-
-    @Override
-    public synchronized void writeRecord(LogRecord record,
-                                         Promise<DLSN> transmitPromise)
-            throws LogRecordTooLongException, WriteException {
-        int logRecordSize = record.getPersistentSize();
-        if (logRecordSize > MAX_LOGRECORD_SIZE) {
-            throw new LogRecordTooLongException(
-                    "Log Record of size " + logRecordSize + " written when only "
-                            + MAX_LOGRECORD_SIZE + " is allowed");
-        }
-
-        try {
-            this.writer.writeOp(record);
-            int numRecords = 1;
-            if (!record.isControl()) {
-                hasUserData = true;
-            }
-            if (record.isRecordSet()) {
-                numRecords = LogRecordSet.numRecords(record);
-            }
-            count += numRecords;
-            writeRequests.add(new WriteRequest(numRecords, transmitPromise));
-            maxTxId = Math.max(maxTxId, record.getTransactionId());
-        } catch (IOException e) {
-            logger.error("Failed to append record to record set of {} : ",
-                    logName, e);
-            throw new WriteException(logName, "Failed to append record to record set of "
-                    + logName);
-        }
-    }
-
-    private synchronized void satisfyPromises(long lssn, long entryId) {
-        long nextSlotId = 0;
-        for (WriteRequest request : writeRequests) {
-            request.promise.setValue(new DLSN(lssn, entryId, nextSlotId));
-            nextSlotId += request.numRecords;
-        }
-        writeRequests.clear();
-    }
-
-    private synchronized void cancelPromises(Throwable reason) {
-        for (WriteRequest request : writeRequests) {
-            request.promise.setException(reason);
-        }
-        writeRequests.clear();
-    }
-
-    @Override
-    public synchronized long getMaxTxId() {
-        return maxTxId;
-    }
-
-    @Override
-    public synchronized boolean hasUserRecords() {
-        return hasUserData;
-    }
-
-    @Override
-    public int getNumBytes() {
-        return buffer.size();
-    }
-
-    @Override
-    public synchronized int getNumRecords() {
-        return count;
-    }
-
-    @Override
-    public synchronized Buffer getBuffer() throws InvalidEnvelopedEntryException, IOException {
-        if (!envelopeBeforeTransmit) {
-            return buffer;
-        }
-        // We can't escape this allocation because things need to be read from one byte array
-        // and then written to another. This is the destination.
-        Buffer toSend = new Buffer(buffer.size());
-        byte[] decompressed = buffer.getData();
-        int length = buffer.size();
-        EnvelopedEntry entry = new EnvelopedEntry(EnvelopedEntry.CURRENT_VERSION,
-                                                  codec,
-                                                  decompressed,
-                                                  length,
-                                                  statsLogger);
-        // This will cause an allocation of a byte[] for compression. This can be avoided
-        // but we can do that later only if needed.
-        entry.writeFully(new DataOutputStream(toSend));
-        return toSend;
-    }
-
-    @Override
-    public synchronized DLSN finalizeTransmit(long lssn, long entryId) {
-        return new DLSN(lssn, entryId, count - 1);
-    }
-
-    @Override
-    public void completeTransmit(long lssn, long entryId) {
-        satisfyPromises(lssn, entryId);
-    }
-
-    @Override
-    public void abortTransmit(Throwable reason) {
-        cancelPromises(reason);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/LedgerReadPosition.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/LedgerReadPosition.java
deleted file mode 100644
index 550d314..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/LedgerReadPosition.java
+++ /dev/null
@@ -1,171 +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 com.twitter.distributedlog;
-
-import java.io.Serializable;
-import java.util.Comparator;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class LedgerReadPosition {
-    static final Logger LOG = LoggerFactory.getLogger(LedgerReadPosition.class);
-
-    private static enum PartialOrderingComparisonResult {
-        NotComparable,
-        GreaterThan,
-        LessThan,
-        EqualTo
-    }
-
-    long ledgerId = DistributedLogConstants.UNRESOLVED_LEDGER_ID;
-    long logSegmentSequenceNo;
-    long entryId;
-
-    public LedgerReadPosition(long ledgerId, long logSegmentSequenceNo, long entryId) {
-        this.ledgerId = ledgerId;
-        this.logSegmentSequenceNo = logSegmentSequenceNo;
-        this.entryId = entryId;
-    }
-
-    public LedgerReadPosition(LedgerReadPosition that) {
-        this.ledgerId = that.ledgerId;
-        this.logSegmentSequenceNo = that.logSegmentSequenceNo;
-        this.entryId = that.entryId;
-    }
-
-
-    public LedgerReadPosition(final DLSN dlsn) {
-        this(dlsn.getLogSegmentSequenceNo(), dlsn.getEntryId());
-    }
-
-    public LedgerReadPosition(long logSegmentSequenceNo, long entryId) {
-        this.logSegmentSequenceNo = logSegmentSequenceNo;
-        this.entryId = entryId;
-    }
-
-    public long getLedgerId() {
-        if (DistributedLogConstants.UNRESOLVED_LEDGER_ID == ledgerId) {
-            LOG.trace("Ledger Id is not initialized");
-            throw new IllegalStateException("Ledger Id is not initialized");
-        }
-        return ledgerId;
-    }
-
-    public long getLogSegmentSequenceNumber() {
-        return logSegmentSequenceNo;
-    }
-
-    public long getEntryId() {
-        return entryId;
-    }
-
-    public void advance() {
-        entryId++;
-    }
-
-    public void positionOnNewLogSegment(long ledgerId, long logSegmentSequenceNo) {
-        this.ledgerId = ledgerId;
-        this.logSegmentSequenceNo = logSegmentSequenceNo;
-        this.entryId = 0L;
-    }
-
-    @Override
-    public String toString() {
-        return String.format("(lid=%d, lseqNo=%d, eid=%d)", ledgerId, logSegmentSequenceNo, entryId);
-    }
-
-    public boolean definitelyLessThanOrEqualTo(LedgerReadPosition threshold) {
-        PartialOrderingComparisonResult result = comparePartiallyOrdered(threshold);
-        return ((result == PartialOrderingComparisonResult.LessThan) ||
-            (result == PartialOrderingComparisonResult.EqualTo));
-    }
-
-    public boolean definitelyLessThan(LedgerReadPosition threshold) {
-        PartialOrderingComparisonResult result = comparePartiallyOrdered(threshold);
-        return result == PartialOrderingComparisonResult.LessThan;
-    }
-
-    private PartialOrderingComparisonResult comparePartiallyOrdered(LedgerReadPosition threshold) {
-        // If no threshold is passed we cannot make a definitive comparison
-        if (null == threshold) {
-            return PartialOrderingComparisonResult.NotComparable;
-        }
-
-        if (this.logSegmentSequenceNo != threshold.logSegmentSequenceNo) {
-            if (this.logSegmentSequenceNo < threshold.logSegmentSequenceNo) {
-                return PartialOrderingComparisonResult.LessThan;
-            } else {
-                return PartialOrderingComparisonResult.GreaterThan;
-            }
-        } else if (this.ledgerId != threshold.ledgerId) {
-            // When logSegmentSequenceNo is equal we cannot definitely say that this
-            // position is less than the threshold unless ledgerIds are equal
-            // since LogSegmentSequenceNumber maybe inferred from transactionIds in older
-            // versions of the metadata.
-            return PartialOrderingComparisonResult.NotComparable;
-        } else if (this.getEntryId() < threshold.getEntryId()) {
-            return PartialOrderingComparisonResult.LessThan;
-        } else if (this.getEntryId() > threshold.getEntryId()) {
-            return PartialOrderingComparisonResult.GreaterThan;
-        } else {
-            return PartialOrderingComparisonResult.EqualTo;
-        }
-    }
-
-    /**
-     * Comparator for the key portion
-     */
-    public static final ReadAheadCacheKeyComparator COMPARATOR = new ReadAheadCacheKeyComparator();
-
-    // Only compares the key portion
-    @Override
-    public boolean equals(Object other) {
-        if (!(other instanceof LedgerReadPosition)) {
-            return false;
-        }
-        LedgerReadPosition key = (LedgerReadPosition) other;
-        return ledgerId == key.ledgerId &&
-            entryId == key.entryId;
-    }
-
-    @Override
-    public int hashCode() {
-        return (int) (ledgerId * 13 ^ entryId * 17);
-    }
-
-    /**
-     * Compare EntryKey.
-     */
-    protected static class ReadAheadCacheKeyComparator implements Comparator<LedgerReadPosition>, Serializable {
-
-        private static final long serialVersionUID = 0L;
-
-        @Override
-        public int compare(LedgerReadPosition left, LedgerReadPosition right) {
-            long ret = left.ledgerId - right.ledgerId;
-            if (ret == 0) {
-                ret = left.entryId - right.entryId;
-            }
-            return (ret < 0) ? -1 : ((ret > 0) ? 1 : 0);
-        }
-    }
-
-}
-
-
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java
deleted file mode 100644
index f4a1e41..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java
+++ /dev/null
@@ -1,364 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.DLMetadata;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.proto.BookieServer;
-import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.bookkeeper.util.LocalBookKeeper;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.BindException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.UUID;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Utility class for setting up bookkeeper ensembles
- * and bringing individual bookies up and down
- */
-public class LocalDLMEmulator {
-    private static final Logger LOG = LoggerFactory.getLogger(LocalDLMEmulator.class);
-
-    public static final String DLOG_NAMESPACE = "/messaging/distributedlog";
-
-    private static final int DEFAULT_BOOKIE_INITIAL_PORT = 0; // Use ephemeral ports
-    private static final int DEFAULT_ZK_TIMEOUT_SEC = 10;
-    private static final int DEFAULT_ZK_PORT = 2181;
-    private static final String DEFAULT_ZK_HOST = "127.0.0.1";
-    private static final String DEFAULT_ZK_ENSEMBLE = DEFAULT_ZK_HOST + ":" + DEFAULT_ZK_PORT;
-    private static final int DEFAULT_NUM_BOOKIES = 3;
-    private static final ServerConfiguration DEFAULT_SERVER_CONFIGURATION = new ServerConfiguration();
-
-    private final String zkEnsemble;
-    private final URI uri;
-    private final List<File> tmpDirs = new ArrayList<File>();
-    private final int zkTimeoutSec;
-    private final Thread bkStartupThread;
-    private final String zkHost;
-    private final int zkPort;
-    private final int numBookies;
-
-    public static class Builder {
-        private int zkTimeoutSec = DEFAULT_ZK_TIMEOUT_SEC;
-        private int numBookies = DEFAULT_NUM_BOOKIES;
-        private String zkHost = DEFAULT_ZK_HOST;
-        private int zkPort = DEFAULT_ZK_PORT;
-        private int initialBookiePort = DEFAULT_BOOKIE_INITIAL_PORT;
-        private boolean shouldStartZK = true;
-        private Optional<ServerConfiguration> serverConf = Optional.absent();
-
-        public Builder numBookies(int numBookies) {
-            this.numBookies = numBookies;
-            return this;
-        }
-        public Builder zkHost(String zkHost) {
-            this.zkHost = zkHost;
-            return this;
-        }
-        public Builder zkPort(int zkPort) {
-            this.zkPort = zkPort;
-            return this;
-        }
-        public Builder zkTimeoutSec(int zkTimeoutSec) {
-            this.zkTimeoutSec = zkTimeoutSec;
-            return this;
-        }
-        public Builder initialBookiePort(int initialBookiePort) {
-            this.initialBookiePort = initialBookiePort;
-            return this;
-        }
-        public Builder shouldStartZK(boolean shouldStartZK) {
-            this.shouldStartZK = shouldStartZK;
-            return this;
-        }
-        public Builder serverConf(ServerConfiguration serverConf) {
-            this.serverConf = Optional.of(serverConf);
-            return this;
-        }
-
-        public LocalDLMEmulator build() throws Exception {
-            ServerConfiguration conf = null;
-            if (serverConf.isPresent()) {
-                conf = serverConf.get();
-            } else {
-                conf = (ServerConfiguration) DEFAULT_SERVER_CONFIGURATION.clone();
-                conf.setZkTimeout(zkTimeoutSec * 1000);
-            }
-            ServerConfiguration newConf = new ServerConfiguration();
-            newConf.loadConf(conf);
-            newConf.setAllowLoopback(true);
-
-            return new LocalDLMEmulator(numBookies, shouldStartZK, zkHost, zkPort,
-                initialBookiePort, zkTimeoutSec, newConf);
-        }
-    }
-
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    private LocalDLMEmulator(final int numBookies, final boolean shouldStartZK, final String zkHost, final int zkPort, final int initialBookiePort, final int zkTimeoutSec, final ServerConfiguration serverConf) throws Exception {
-        this.numBookies = numBookies;
-        this.zkHost = zkHost;
-        this.zkPort = zkPort;
-        this.zkEnsemble = zkHost + ":" + zkPort;
-        this.uri = URI.create("distributedlog://" + zkEnsemble + DLOG_NAMESPACE);
-        this.zkTimeoutSec = zkTimeoutSec;
-        this.bkStartupThread = new Thread() {
-            public void run() {
-                try {
-                    LOG.info("Starting {} bookies : allowLoopback = {}", numBookies, serverConf.getAllowLoopback());
-                    LocalBookKeeper.startLocalBookies(zkHost, zkPort, numBookies, shouldStartZK, initialBookiePort, serverConf);
-                    LOG.info("{} bookies are started.");
-                } catch (InterruptedException e) {
-                    // go away quietly
-                } catch (Exception e) {
-                    LOG.error("Error starting local bk", e);
-                }
-            }
-        };
-    }
-
-    public void start() throws Exception {
-        bkStartupThread.start();
-        if (!LocalBookKeeper.waitForServerUp(zkEnsemble, zkTimeoutSec*1000)) {
-            throw new Exception("Error starting zookeeper/bookkeeper");
-        }
-        int bookiesUp = checkBookiesUp(numBookies, zkTimeoutSec);
-        assert (numBookies == bookiesUp);
-        // Provision "/messaging/distributedlog" namespace
-        DLMetadata.create(new BKDLConfig(zkEnsemble, "/ledgers")).create(uri);
-    }
-
-    public void teardown() throws Exception {
-        if (bkStartupThread != null) {
-            bkStartupThread.interrupt();
-            bkStartupThread.join();
-        }
-        for (File dir : tmpDirs) {
-            FileUtils.deleteDirectory(dir);
-        }
-    }
-
-    public String getZkServers() {
-        return zkEnsemble;
-    }
-
-    public URI getUri() {
-        return uri;
-    }
-
-    public BookieServer newBookie() throws Exception {
-        ServerConfiguration bookieConf = new ServerConfiguration();
-        bookieConf.setZkTimeout(zkTimeoutSec * 1000);
-        bookieConf.setBookiePort(0);
-        bookieConf.setAllowLoopback(true);
-        File tmpdir = File.createTempFile("bookie" + UUID.randomUUID() + "_",
-            "test");
-        if (!tmpdir.delete()) {
-            LOG.debug("Fail to delete tmpdir " + tmpdir);
-        }
-        if (!tmpdir.mkdir()) {
-            throw new IOException("Fail to create tmpdir " + tmpdir);
-        }
-        tmpDirs.add(tmpdir);
-
-        bookieConf.setZkServers(zkEnsemble);
-        bookieConf.setJournalDirName(tmpdir.getPath());
-        bookieConf.setLedgerDirNames(new String[]{tmpdir.getPath()});
-
-        BookieServer b = new BookieServer(bookieConf);
-        b.start();
-        for (int i = 0; i < 10 && !b.isRunning(); i++) {
-            Thread.sleep(10000);
-        }
-        if (!b.isRunning()) {
-            throw new IOException("Bookie would not start");
-        }
-        return b;
-    }
-
-    /**
-     * Check that a number of bookies are available
-     *
-     * @param count number of bookies required
-     * @param timeout number of seconds to wait for bookies to start
-     * @throws java.io.IOException if bookies are not started by the time the timeout hits
-     */
-    public int checkBookiesUp(int count, int timeout) throws Exception {
-        ZooKeeper zkc = connectZooKeeper(zkHost, zkPort, zkTimeoutSec);
-        try {
-            int mostRecentSize = 0;
-            for (int i = 0; i < timeout; i++) {
-                try {
-                    List<String> children = zkc.getChildren("/ledgers/available",
-                        false);
-                    children.remove("readonly");
-                    mostRecentSize = children.size();
-                    if ((mostRecentSize > count) || LOG.isDebugEnabled()) {
-                        LOG.info("Found " + mostRecentSize + " bookies up, "
-                            + "waiting for " + count);
-                        if ((mostRecentSize > count) || LOG.isTraceEnabled()) {
-                            for (String child : children) {
-                                LOG.info(" server: " + child);
-                            }
-                        }
-                    }
-                    if (mostRecentSize == count) {
-                        break;
-                    }
-                } catch (KeeperException e) {
-                    // ignore
-                }
-                Thread.sleep(1000);
-            }
-            return mostRecentSize;
-        } finally {
-            zkc.close();
-        }
-    }
-
-    public static String getBkLedgerPath() {
-        return "/ledgers";
-    }
-
-    public static ZooKeeper connectZooKeeper(String zkHost, int zkPort)
-        throws IOException, KeeperException, InterruptedException {
-            return connectZooKeeper(zkHost, zkPort, DEFAULT_ZK_TIMEOUT_SEC);
-    }
-
-    public static ZooKeeper connectZooKeeper(String zkHost, int zkPort, int zkTimeoutSec)
-        throws IOException, KeeperException, InterruptedException {
-        final CountDownLatch latch = new CountDownLatch(1);
-        final String zkHostPort = zkHost + ":" + zkPort;
-
-        ZooKeeper zkc = new ZooKeeper(zkHostPort, zkTimeoutSec * 1000, new Watcher() {
-            public void process(WatchedEvent event) {
-                if (event.getState() == Event.KeeperState.SyncConnected) {
-                    latch.countDown();
-                }
-            }
-        });
-        if (!latch.await(zkTimeoutSec, TimeUnit.SECONDS)) {
-            throw new IOException("Zookeeper took too long to connect");
-        }
-        return zkc;
-    }
-
-    public static URI createDLMURI(String path) throws Exception {
-        return createDLMURI(DEFAULT_ZK_ENSEMBLE, path);
-    }
-
-    public static URI createDLMURI(String zkServers, String path) throws Exception {
-        return URI.create("distributedlog://" + zkServers + DLOG_NAMESPACE + path);
-    }
-
-    /**
-     * Try to start zookkeeper locally on any port.
-     */
-    public static Pair<ZooKeeperServerShim, Integer> runZookeeperOnAnyPort(File zkDir) throws Exception {
-        return runZookeeperOnAnyPort((int) (Math.random()*10000+7000), zkDir);
-    }
-
-    /**
-     * Try to start zookkeeper locally on any port beginning with some base port.
-     * Dump some socket info when bind fails.
-     */
-    public static Pair<ZooKeeperServerShim, Integer> runZookeeperOnAnyPort(int basePort, File zkDir) throws Exception {
-
-        final int MAX_RETRIES = 20;
-        final int MIN_PORT = 1025;
-        final int MAX_PORT = 65535;
-        ZooKeeperServerShim zks = null;
-        int zkPort = basePort;
-        boolean success = false;
-        int retries = 0;
-
-        while (!success) {
-            try {
-                LOG.info("zk trying to bind to port " + zkPort);
-                zks = LocalBookKeeper.runZookeeper(1000, zkPort, zkDir);
-                success = true;
-            } catch (BindException be) {
-                retries++;
-                if (retries > MAX_RETRIES) {
-                    throw be;
-                }
-                zkPort++;
-                if (zkPort > MAX_PORT) {
-                    zkPort = MIN_PORT;
-                }
-            }
-        }
-
-        return Pair.of(zks, zkPort);
-    }
-
-    public static void main(String[] args) throws Exception {
-        try {
-            if (args.length < 1) {
-                System.out.println("Usage: LocalDLEmulator <zk_port>");
-                System.exit(-1);
-            }
-
-            final int zkPort = Integer.parseInt(args[0]);
-            final File zkDir = IOUtils.createTempDir("distrlog", "zookeeper");
-            final LocalDLMEmulator localDlm = LocalDLMEmulator.newBuilder()
-                .zkPort(zkPort)
-                .build();
-
-            Runtime.getRuntime().addShutdownHook(new Thread() {
-                @Override
-                public void run() {
-                    try {
-                        localDlm.teardown();
-                        FileUtils.deleteDirectory(zkDir);
-                        System.out.println("ByeBye!");
-                    } catch (Exception e) {
-                        // do nothing
-                    }
-                }
-            });
-            localDlm.start();
-
-            System.out.println(String.format(
-                "DistributedLog Sandbox is running now. You could access distributedlog://%s:%s",
-                DEFAULT_ZK_HOST,
-                zkPort));
-        } catch (Exception ex) {
-            System.out.println("Exception occurred running emulator " + ex);
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/LogReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/LogReader.java
deleted file mode 100644
index c12de29..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/LogReader.java
+++ /dev/null
@@ -1,195 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.io.AsyncCloseable;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-
-/**
- * <i>LogReader</i> is a `synchronous` reader reading records from a DL log.
- *
- * <h3>Lifecycle of a Reader</h3>
- *
- * 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)}
- * to read records out the log from provided position.
- * <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,
- * 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:
- * <pre>
- * DistributedLogManager dlm = ...;
- * long nextTxId = ...;
- * LogReader reader = dlm.getInputStream(nextTxId);
- *
- * while (true) { // keep reading & processing records
- *     LogRecord record;
- *     try {
- *         record = reader.readNext(false);
- *         nextTxId = record.getTransactionId();
- *         // process the record
- *         ...
- *     } catch (IOException ioe) {
- *         // handle the exception
- *         ...
- *         reader = dlm.getInputStream(nextTxId + 1);
- *     }
- * }
- *
- * </pre>
- *
- * <h3>Read Records</h3>
- *
- * 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.
- *
- * <h4>Blocking vs NonBlocking</h4>
- *
- * <i>Blocking</i> (nonBlocking = false) means the reads will wait for records
- * 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
- * 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
- * will wait for a small period of time (defined in
- * {@link DistributedLogConfiguration#getReadAheadWaitTime()} and return whatever
- * 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
- * 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.
- *
- * <h4>Read Single Record</h4>
- *
- * {@link #readNext(boolean)} is reading individual records from a DL log.
- *
- * <pre>
- * LogReader reader = ...
- *
- * // keep reading records in blocking way until no records available in the log
- * LogRecord record = reader.readNext(false);
- * while (null != record) {
- *     // process the record
- *     ...
- *     // read next record
- *     records = reader.readNext(false);
- * }
- *
- * ...
- *
- * // reader is caught up with writer, doing non-blocking reads to tail the log
- * while (true) {
- *     record = reader.readNext(true)
- *     // process the new records
- *     ...
- * }
- * </pre>
- *
- * <h4>Read Batch of Records</h4>
- *
- * {@link #readBulk(boolean, int)} is a convenient way to read a batch of records
- * from a DL log.
- *
- * <pre>
- * LogReader reader = ...
- * int N = 10;
- *
- * // keep reading N records in blocking way until no records available in the log
- * List<LogRecord> records = reader.readBulk(false, N);
- * while (!records.isEmpty()) {
- *     // process the list of records
- *     ...
- *     if (records.size() < N) { // no more records available in the log
- *         break;
- *     }
- *     // read next N records
- *     records = reader.readBulk(false, N);
- * }
- *
- * ...
- *
- * // reader is caught up with writer, doing non-blocking reads to tail the log
- * while (true) {
- *     records = reader.readBulk(true, N)
- *     // process the new records
- *     ...
- * }
- *
- * </pre>
- *
- * @see AsyncLogReader
- *
- * NOTE:
- * 1. Extending {@link AsyncCloseable}: BKSyncLogReader is implemented based on BKAsyncLogReader, exposing
- *    the {@link AsyncCloseable} interface so the reader could be closed asynchronously
- */
-public interface LogReader extends Closeable, AsyncCloseable {
-
-    /**
-     * Read the next log record from the stream.
-     * <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
-     * 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.
-     *
-     * @param nonBlocking should the read make blocking calls to the backend or rely on the
-     * readAhead cache
-     * @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;
-
-    /**
-     * 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
-     * @param numLogRecords maximum number of log records returned by this call.
-     * @return an operation from the stream or empty list if at end of stream
-     * @throws IOException if there is an error reading from the stream
-     * @see #readNext(boolean)
-     */
-    public List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords) throws IOException;
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/LogSegmentMetadata.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/LogSegmentMetadata.java
deleted file mode 100644
index a8d9e6d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/LogSegmentMetadata.java
+++ /dev/null
@@ -1,1125 +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 com.twitter.distributedlog;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Comparator;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.LogSegmentNotFoundException;
-import com.twitter.distributedlog.exceptions.UnsupportedMetadataVersionException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.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;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Utility class for storing the metadata associated
- * with a single edit log segment, stored in a single ledger
- */
-public class LogSegmentMetadata {
-    static final Logger LOG = LoggerFactory.getLogger(LogSegmentMetadata.class);
-
-    public static enum LogSegmentMetadataVersion {
-        VERSION_INVALID(0),
-        VERSION_V1_ORIGINAL(1),
-        VERSION_V2_LEDGER_SEQNO(2),
-        VERSION_V3_MIN_ACTIVE_DLSN(3),
-        VERSION_V4_ENVELOPED_ENTRIES(4),
-        VERSION_V5_SEQUENCE_ID(5);
-
-        public final int value;
-
-        private LogSegmentMetadataVersion(int value) {
-            this.value = value;
-        }
-
-        public static LogSegmentMetadataVersion of(int version) {
-            switch (version) {
-                case 5:
-                    return VERSION_V5_SEQUENCE_ID;
-                case 4:
-                    return VERSION_V4_ENVELOPED_ENTRIES;
-                case 3:
-                    return VERSION_V3_MIN_ACTIVE_DLSN;
-                case 2:
-                    return VERSION_V2_LEDGER_SEQNO;
-                case 1:
-                    return VERSION_V1_ORIGINAL;
-                case 0:
-                    return VERSION_INVALID;
-                default:
-                    throw new IllegalArgumentException("unknown version " + version);
-            }
-        }
-    }
-
-    public static enum TruncationStatus {
-        ACTIVE (0), PARTIALLY_TRUNCATED(1), TRUNCATED (2);
-        private final int value;
-
-        private TruncationStatus(int value) {
-            this.value = value;
-        }
-    }
-
-    public static class LogSegmentMetadataBuilder {
-        protected String zkPath;
-        protected long logSegmentId;
-        protected LogSegmentMetadataVersion version;
-        protected long firstTxId;
-        protected int regionId;
-        protected long status;
-        protected long lastTxId;
-        protected long completionTime;
-        protected int recordCount;
-        protected long logSegmentSequenceNo;
-        protected long lastEntryId;
-        protected long lastSlotId;
-        protected long minActiveEntryId;
-        protected long minActiveSlotId;
-        protected long startSequenceId;
-        protected boolean inprogress;
-
-        // This is a derived attribute.
-        // Since we overwrite the original version with the target version, information that is
-        // derived from the original version (e.g. does it support enveloping of entries)
-        // is lost while parsing.
-        // NOTE: This value is not stored in the Metadata store.
-        protected boolean envelopeEntries = false;
-
-        LogSegmentMetadataBuilder(String zkPath,
-                                  LogSegmentMetadataVersion version,
-                                  long logSegmentId,
-                                  long firstTxId) {
-            initialize();
-            this.zkPath = zkPath;
-            this.version = version;
-            this.logSegmentId = logSegmentId;
-            this.firstTxId = firstTxId;
-        }
-
-        LogSegmentMetadataBuilder(String zkPath,
-                                  int version,
-                                  long logSegmentId,
-                                  long firstTxId) {
-            this(zkPath, LogSegmentMetadataVersion.values()[version], logSegmentId, firstTxId);
-        }
-
-        private void initialize() {
-            regionId = DistributedLogConstants.LOCAL_REGION_ID;
-            status = DistributedLogConstants.LOGSEGMENT_DEFAULT_STATUS;
-            lastTxId = DistributedLogConstants.INVALID_TXID;
-            completionTime = 0;
-            recordCount = 0;
-            lastEntryId = -1;
-            lastSlotId = -1;
-            minActiveEntryId = 0;
-            minActiveSlotId = 0;
-            startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
-            inprogress = true;
-        }
-
-        LogSegmentMetadataBuilder setRegionId(int regionId) {
-            this.regionId = regionId;
-            return this;
-        }
-
-        LogSegmentMetadataBuilder setStatus(long status) {
-            this.status = status;
-            return this;
-        }
-
-        public LogSegmentMetadataBuilder setLastTxId(long lastTxId) {
-            this.lastTxId = lastTxId;
-            return this;
-        }
-
-        public LogSegmentMetadataBuilder setCompletionTime(long completionTime) {
-            this.completionTime = completionTime;
-            return this;
-        }
-
-        public LogSegmentMetadataBuilder setRecordCount(int recordCount) {
-            this.recordCount = recordCount;
-            return this;
-        }
-
-        public LogSegmentMetadataBuilder setRecordCount(LogRecord record) {
-            this.recordCount = record.getLastPositionWithinLogSegment();
-            return this;
-        }
-
-        public LogSegmentMetadataBuilder setInprogress(boolean inprogress) {
-            this.inprogress = inprogress;
-            return this;
-        }
-
-        LogSegmentMetadataBuilder setLogSegmentSequenceNo(long logSegmentSequenceNo) {
-            this.logSegmentSequenceNo = logSegmentSequenceNo;
-            return this;
-        }
-
-        public LogSegmentMetadataBuilder setLastEntryId(long lastEntryId) {
-            this.lastEntryId = lastEntryId;
-            return this;
-        }
-
-        LogSegmentMetadataBuilder setLastSlotId(long lastSlotId) {
-            this.lastSlotId = lastSlotId;
-            return this;
-        }
-
-        LogSegmentMetadataBuilder setEnvelopeEntries(boolean envelopeEntries) {
-            this.envelopeEntries = envelopeEntries;
-            return this;
-        }
-
-        LogSegmentMetadataBuilder setMinActiveEntryId(long minActiveEntryId) {
-            this.minActiveEntryId = minActiveEntryId;
-            return this;
-        }
-
-        LogSegmentMetadataBuilder setMinActiveSlotId(long minActiveSlotId) {
-            this.minActiveSlotId = minActiveSlotId;
-            return this;
-        }
-
-        LogSegmentMetadataBuilder setStartSequenceId(long startSequenceId) {
-            this.startSequenceId = startSequenceId;
-            return this;
-        }
-
-        public LogSegmentMetadata build() {
-            return new LogSegmentMetadata(
-                zkPath,
-                version,
-                    logSegmentId,
-                firstTxId,
-                lastTxId,
-                completionTime,
-                inprogress,
-                recordCount,
-                    logSegmentSequenceNo,
-                lastEntryId,
-                lastSlotId,
-                regionId,
-                status,
-                minActiveEntryId,
-                minActiveSlotId,
-                startSequenceId,
-                envelopeEntries
-            );
-        }
-
-    }
-
-    /**
-     * Mutator to mutate the metadata of a log segment. This mutator is going to create
-     * a new instance of the log segment metadata without changing the existing one.
-     */
-    public static class Mutator extends LogSegmentMetadataBuilder {
-
-        Mutator(LogSegmentMetadata original) {
-            super(original.getZkPath(), original.getVersion(), original.getLogSegmentId(), original.getFirstTxId());
-            this.inprogress = original.isInProgress();
-            this.logSegmentSequenceNo = original.getLogSegmentSequenceNumber();
-            this.lastEntryId = original.getLastEntryId();
-            this.lastSlotId = original.getLastSlotId();
-            this.lastTxId = original.getLastTxId();
-            this.completionTime = original.getCompletionTime();
-            this.recordCount = original.getRecordCount();
-            this.regionId = original.getRegionId();
-            this.status = original.getStatus();
-            this.minActiveEntryId = original.getMinActiveDLSN().getEntryId();
-            this.minActiveSlotId = original.getMinActiveDLSN().getSlotId();
-            this.startSequenceId = original.getStartSequenceId();
-            this.envelopeEntries = original.getEnvelopeEntries();
-        }
-
-        @VisibleForTesting
-        public Mutator setVersion(LogSegmentMetadataVersion version) {
-            this.version = version;
-            return this;
-        }
-
-        public Mutator setLogSegmentSequenceNumber(long seqNo) {
-            this.logSegmentSequenceNo = seqNo;
-            return this;
-        }
-
-        public Mutator setZkPath(String zkPath) {
-            this.zkPath = zkPath;
-            return this;
-        }
-
-        public Mutator setLastDLSN(DLSN dlsn) {
-            this.logSegmentSequenceNo = dlsn.getLogSegmentSequenceNo();
-            this.lastEntryId = dlsn.getEntryId();
-            this.lastSlotId = dlsn.getSlotId();
-            return this;
-        }
-
-        public Mutator setMinActiveDLSN(DLSN dlsn) {
-            if (this.logSegmentSequenceNo != dlsn.getLogSegmentSequenceNo()) {
-                throw new IllegalArgumentException("Updating minDLSN in an incorrect log segment");
-            }
-            this.minActiveEntryId = dlsn.getEntryId();
-            this.minActiveSlotId = dlsn.getSlotId();
-            return this;
-        }
-
-        public Mutator setTruncationStatus(TruncationStatus truncationStatus) {
-            status &= ~METADATA_TRUNCATION_STATUS_MASK;
-            status |= (truncationStatus.value & METADATA_TRUNCATION_STATUS_MASK);
-            return this;
-        }
-
-        public Mutator setStartSequenceId(long startSequenceId) {
-            this.startSequenceId = startSequenceId;
-            return this;
-        }
-    }
-
-    private final String zkPath;
-    private final long logSegmentId;
-    private final LogSegmentMetadataVersion version;
-    private final long firstTxId;
-    private final int regionId;
-    private final long status;
-    private final long lastTxId;
-    private final long completionTime;
-    private final int recordCount;
-    private final DLSN lastDLSN;
-    private final DLSN minActiveDLSN;
-    private final long startSequenceId;
-    private final boolean inprogress;
-    // This is a derived attribute.
-    // Since we overwrite the original version with the target version, information that is
-    // derived from the original version (e.g. does it support enveloping of entries)
-    // is lost while parsing.
-    // NOTE: This value is not stored in the Metadata store.
-    private final boolean envelopeEntries;
-
-    public static final Comparator<LogSegmentMetadata> COMPARATOR
-        = new Comparator<LogSegmentMetadata>() {
-
-        public int compare(LogSegmentMetadata o1,
-                           LogSegmentMetadata o2) {
-            if ((o1.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO) ||
-                (o2.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)) {
-                if (o1.firstTxId < o2.firstTxId) {
-                    return -1;
-                } else if (o1.firstTxId == o2.firstTxId) {
-                    return 0;
-                } else {
-                    return 1;
-                }
-            } else {
-                if (o1.getLogSegmentSequenceNumber() < o2.getLogSegmentSequenceNumber()) {
-                    return -1;
-                } else if (o1.getLogSegmentSequenceNumber() == o2.getLogSegmentSequenceNumber()) {
-                    // make sure we won't move over inprogress log segment if it still presents in the list
-                    if (o1.isInProgress() && !o2.isInProgress()) {
-                        return -1;
-                    } else if (!o1.isInProgress() && o2.isInProgress()) {
-                        return 1;
-                    } else {
-                        return 0;
-                    }
-                } else {
-                    return 1;
-                }
-            }
-
-
-        }
-    };
-
-    public static final Comparator<LogSegmentMetadata> DESC_COMPARATOR
-        = new Comparator<LogSegmentMetadata>() {
-        public int compare(LogSegmentMetadata o1,
-                           LogSegmentMetadata o2) {
-            if ((o1.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO) ||
-                (o2.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)) {
-                if (o1.firstTxId > o2.firstTxId) {
-                    return -1;
-                } else if (o1.firstTxId == o2.firstTxId) {
-                    return 0;
-                } else {
-                    return 1;
-                }
-            } else {
-                if (o1.getLogSegmentSequenceNumber() > o2.getLogSegmentSequenceNumber()) {
-                    return -1;
-                } else if (o1.getLogSegmentSequenceNumber() == o2.getLogSegmentSequenceNumber()) {
-                    // make sure we won't move over inprogress log segment if it still presents in the list
-                    if (o1.isInProgress() && !o2.isInProgress()) {
-                        return 1;
-                    } else if (!o1.isInProgress() && o2.isInProgress()) {
-                        return -1;
-                    } else {
-                        return 0;
-                    }
-                } else {
-                    return 1;
-                }
-            }
-        }
-    };
-
-    public static final int LEDGER_METADATA_CURRENT_LAYOUT_VERSION =
-                LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value;
-
-    public static final int LEDGER_METADATA_OLDEST_SUPPORTED_VERSION =
-        LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value;
-
-    static final int LOGRECORD_COUNT_SHIFT = 32;
-    static final long LOGRECORD_COUNT_MASK = 0xffffffff00000000L;
-    static final int REGION_SHIFT = 28;
-    static final long MAX_REGION_ID = 0xfL;
-    static final long REGION_MASK = 0x00000000f0000000L;
-    static final int STATUS_BITS_SHIFT = 8;
-    static final long STATUS_BITS_MASK = 0x000000000000ff00L;
-    static final long UNUSED_BITS_MASK = 0x000000000fff0000L;
-    static final long METADATA_VERSION_MASK = 0x00000000000000ffL;
-
-    //Metadata status bits
-    static final long METADATA_TRUNCATION_STATUS_MASK = 0x3L;
-    static final long METADATA_STATUS_BIT_MAX = 0xffL;
-
-    private LogSegmentMetadata(String zkPath,
-                               LogSegmentMetadataVersion version,
-                               long logSegmentId,
-                               long firstTxId,
-                               long lastTxId,
-                               long completionTime,
-                               boolean inprogress,
-                               int recordCount,
-                               long logSegmentSequenceNumber,
-                               long lastEntryId,
-                               long lastSlotId,
-                               int regionId,
-                               long status,
-                               long minActiveEntryId,
-                               long minActiveSlotId,
-                               long startSequenceId,
-                               boolean envelopeEntries) {
-        this.zkPath = zkPath;
-        this.logSegmentId = logSegmentId;
-        this.version = version;
-        this.firstTxId = firstTxId;
-        this.lastTxId = lastTxId;
-        this.inprogress = inprogress;
-        this.completionTime = completionTime;
-        this.recordCount = recordCount;
-        this.lastDLSN = new DLSN(logSegmentSequenceNumber, lastEntryId, lastSlotId);
-        this.minActiveDLSN = new DLSN(logSegmentSequenceNumber, minActiveEntryId, minActiveSlotId);
-        this.startSequenceId = startSequenceId;
-        this.regionId = regionId;
-        this.status = status;
-        this.envelopeEntries = envelopeEntries;
-    }
-
-    public String getZkPath() {
-        return zkPath;
-    }
-
-    public String getZNodeName() {
-        return new File(zkPath).getName();
-    }
-
-    public long getFirstTxId() {
-        return firstTxId;
-    }
-
-    public long getLastTxId() {
-        return lastTxId;
-    }
-
-    public long getCompletionTime() {
-        return completionTime;
-    }
-
-    public long getLogSegmentId() {
-        return logSegmentId;
-    }
-
-    public long getLogSegmentSequenceNumber() {
-        return lastDLSN.getLogSegmentSequenceNo();
-    }
-
-    public int getVersion() {
-        return version.value;
-    }
-
-    public boolean getEnvelopeEntries() {
-        return envelopeEntries;
-    }
-
-    public long getLastEntryId() {
-        return lastDLSN.getEntryId();
-    }
-
-    long getStatus() {
-        return status;
-    }
-
-    public long getStartSequenceId() {
-        // generate negative sequence id for log segments that created <= v4
-        return supportsSequenceId() && startSequenceId != DistributedLogConstants.UNASSIGNED_SEQUENCE_ID ?
-                startSequenceId : Long.MIN_VALUE + (getLogSegmentSequenceNumber() << 32L);
-    }
-
-    public boolean isTruncated() {
-        return ((status & METADATA_TRUNCATION_STATUS_MASK)
-                == TruncationStatus.TRUNCATED.value);
-    }
-
-    public boolean isPartiallyTruncated() {
-        return ((status & METADATA_TRUNCATION_STATUS_MASK)
-                == TruncationStatus.PARTIALLY_TRUNCATED.value);
-    }
-
-    public boolean isNonTruncated() {
-        return ((status & METADATA_TRUNCATION_STATUS_MASK)
-                == TruncationStatus.ACTIVE.value);
-    }
-
-    public long getLastSlotId() {
-        return lastDLSN.getSlotId();
-    }
-
-    public DLSN getLastDLSN() {
-        return lastDLSN;
-    }
-
-    public DLSN getMinActiveDLSN() {
-        return minActiveDLSN;
-    }
-
-    public DLSN getFirstDLSN() {
-        return new DLSN(getLogSegmentSequenceNumber(), 0, 0);
-    }
-
-    public int getRecordCount() {
-        return recordCount;
-    }
-
-    public int getRegionId() {
-        return regionId;
-    }
-
-    public boolean isInProgress() {
-        return this.inprogress;
-    }
-
-    @VisibleForTesting
-    public boolean isDLSNinThisSegment(DLSN dlsn) {
-        return dlsn.getLogSegmentSequenceNo() == getLogSegmentSequenceNumber();
-    }
-
-    @VisibleForTesting
-    public boolean isRecordPositionWithinSegmentScope(LogRecord record) {
-        return record.getLastPositionWithinLogSegment() <= getRecordCount();
-    }
-
-    @VisibleForTesting
-    public boolean isRecordLastPositioninThisSegment(LogRecord record) {
-        return record.getLastPositionWithinLogSegment() == getRecordCount();
-    }
-
-    /**
-     * complete current log segment. A new log segment metadata instance will be returned.
-     *
-     * @param zkPath
-     *          zk path for the completed log segment.
-     * @param newLastTxId
-     *          last tx id
-     * @param recordCount
-     *          record count
-     * @param lastEntryId
-     *          last entry id
-     * @param lastSlotId
-     *          last slot id
-     * @return completed log segment.
-     */
-    LogSegmentMetadata completeLogSegment(String zkPath,
-                                                long newLastTxId,
-                                                int recordCount,
-                                                long lastEntryId,
-                                                long lastSlotId,
-                                                long startSequenceId) {
-        assert this.lastTxId == DistributedLogConstants.INVALID_TXID;
-
-        return new Mutator(this)
-                .setZkPath(zkPath)
-                .setLastDLSN(new DLSN(this.lastDLSN.getLogSegmentSequenceNo(), lastEntryId, lastSlotId))
-                .setLastTxId(newLastTxId)
-                .setInprogress(false)
-                .setCompletionTime(Utils.nowInMillis())
-                .setRecordCount(recordCount)
-                .setStartSequenceId(startSequenceId)
-                .build();
-    }
-
-    public static Future<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>();
-        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));
-                        } else {
-                            FutureUtils.setException(result,
-                                    new ZKException("Failed to read log segment metadata from " + path,
-                                            KeeperException.Code.get(rc)));
-                        }
-                        return;
-                    }
-                    try {
-                        LogSegmentMetadata metadata = parseData(path, data, skipMinVersionCheck);
-                        FutureUtils.setValue(result, metadata);
-                    } catch (IOException ie) {
-                        LOG.error("Error on parsing log segment metadata from {} : ", path, ie);
-                        result.setException(ie);
-                    }
-                }
-            }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            result.setException(FutureUtils.zkException(e, path));
-        } catch (InterruptedException e) {
-            result.setException(FutureUtils.zkException(e, path));
-        }
-        return result;
-    }
-
-    static LogSegmentMetadata parseDataV1(String path, byte[] data, String[] parts)
-        throws IOException {
-        long versionStatusCount = Long.parseLong(parts[0]);
-
-        long version = versionStatusCount & METADATA_VERSION_MASK;
-        assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
-        assert (1 == version);
-
-        LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.VERSION_V1_ORIGINAL;
-
-        int regionId = (int)(versionStatusCount & REGION_MASK) >> REGION_SHIFT;
-        assert (regionId >= 0 && regionId <= 0xf);
-
-        long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
-        assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
-
-        if (parts.length == 3) {
-            long logSegmentId = Long.parseLong(parts[1]);
-            long txId = Long.parseLong(parts[2]);
-            return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
-                    .setRegionId(regionId)
-                    .setStatus(status)
-                    .build();
-        } else if (parts.length == 5) {
-            long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
-            assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
-
-            long logSegmentId = Long.parseLong(parts[1]);
-            long firstTxId = Long.parseLong(parts[2]);
-            long lastTxId = Long.parseLong(parts[3]);
-            long completionTime = Long.parseLong(parts[4]);
-            return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
-                .setInprogress(false)
-                .setLastTxId(lastTxId)
-                .setCompletionTime(completionTime)
-                .setRecordCount((int) recordCount)
-                .setRegionId(regionId)
-                .setStatus(status)
-                .build();
-        } else {
-            throw new IOException("Invalid log segment metadata : "
-                + new String(data, UTF_8));
-        }
-    }
-
-    static LogSegmentMetadata parseDataV2(String path, byte[] data, String[] parts)
-        throws IOException {
-        long versionStatusCount = Long.parseLong(parts[0]);
-
-        long version = versionStatusCount & METADATA_VERSION_MASK;
-        assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
-        assert (2 == version);
-
-        LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO;
-
-        int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
-        assert (regionId >= 0 && regionId <= 0xf);
-
-        long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
-        assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
-
-        if (parts.length == 4) {
-            long logSegmentId = Long.parseLong(parts[1]);
-            long txId = Long.parseLong(parts[2]);
-            long logSegmentSequenceNumber = Long.parseLong(parts[3]);
-            return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
-                .setLogSegmentSequenceNo(logSegmentSequenceNumber)
-                .setRegionId(regionId)
-                .setStatus(status)
-                .build();
-        } else if (parts.length == 8) {
-            long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
-            assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
-
-            long logSegmentId = Long.parseLong(parts[1]);
-            long firstTxId = Long.parseLong(parts[2]);
-            long lastTxId = Long.parseLong(parts[3]);
-            long completionTime = Long.parseLong(parts[4]);
-            long logSegmentSequenceNumber = Long.parseLong(parts[5]);
-            long lastEntryId = Long.parseLong(parts[6]);
-            long lastSlotId = Long.parseLong(parts[7]);
-            return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
-                .setInprogress(false)
-                .setLastTxId(lastTxId)
-                .setCompletionTime(completionTime)
-                .setRecordCount((int) recordCount)
-                .setLogSegmentSequenceNo(logSegmentSequenceNumber)
-                .setLastEntryId(lastEntryId)
-                .setLastSlotId(lastSlotId)
-                .setRegionId(regionId)
-                .setStatus(status)
-                .build();
-        } else {
-            throw new IOException("Invalid logsegment metadata : "
-                + new String(data, UTF_8));
-        }
-
-    }
-
-    static LogSegmentMetadata parseDataVersionsWithMinActiveDLSN(String path, byte[] data, String[] parts)
-        throws IOException {
-        long versionStatusCount = Long.parseLong(parts[0]);
-
-        long version = versionStatusCount & METADATA_VERSION_MASK;
-        assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
-        assert (LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version &&
-                LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version);
-
-        LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.of((int) version);
-
-        int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
-        assert (regionId >= 0 && regionId <= 0xf);
-
-        long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
-        assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
-
-        if (parts.length == 6) {
-            long logSegmentId = Long.parseLong(parts[1]);
-            long txId = Long.parseLong(parts[2]);
-            long logSegmentSequenceNumber = Long.parseLong(parts[3]);
-            long minActiveEntryId = Long.parseLong(parts[4]);
-            long minActiveSlotId = Long.parseLong(parts[5]);
-
-            LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
-                .setLogSegmentSequenceNo(logSegmentSequenceNumber)
-                .setMinActiveEntryId(minActiveEntryId)
-                .setMinActiveSlotId(minActiveSlotId)
-                .setRegionId(regionId)
-                .setStatus(status);
-            if (supportsEnvelopedEntries((int) version)) {
-                builder = builder.setEnvelopeEntries(true);
-            }
-            return builder.build();
-        } else if (parts.length == 10) {
-            long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
-            assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
-
-            long logSegmentId = Long.parseLong(parts[1]);
-            long firstTxId = Long.parseLong(parts[2]);
-            long lastTxId = Long.parseLong(parts[3]);
-            long completionTime = Long.parseLong(parts[4]);
-            long logSegmentSequenceNumber = Long.parseLong(parts[5]);
-            long lastEntryId = Long.parseLong(parts[6]);
-            long lastSlotId = Long.parseLong(parts[7]);
-            long minActiveEntryId = Long.parseLong(parts[8]);
-            long minActiveSlotId = Long.parseLong(parts[9]);
-            LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
-                .setInprogress(false)
-                .setLastTxId(lastTxId)
-                .setCompletionTime(completionTime)
-                .setRecordCount((int) recordCount)
-                .setLogSegmentSequenceNo(logSegmentSequenceNumber)
-                .setLastEntryId(lastEntryId)
-                .setLastSlotId(lastSlotId)
-                .setMinActiveEntryId(minActiveEntryId)
-                .setMinActiveSlotId(minActiveSlotId)
-                .setRegionId(regionId)
-                .setStatus(status);
-            if (supportsEnvelopedEntries((int) version)) {
-                builder = builder.setEnvelopeEntries(true);
-            }
-            return builder.build();
-        } else {
-            throw new IOException("Invalid logsegment metadata : "
-                + new String(data, UTF_8));
-        }
-
-    }
-
-    static LogSegmentMetadata parseDataVersionsWithSequenceId(String path, byte[] data, String[] parts)
-        throws IOException {
-        long versionStatusCount = Long.parseLong(parts[0]);
-
-        long version = versionStatusCount & METADATA_VERSION_MASK;
-        assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
-        assert (LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value <= version &&
-                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION >= version);
-
-        LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.of((int) version);
-
-        int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
-        assert (regionId >= 0 && regionId <= 0xf);
-
-        long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
-        assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
-
-        if (parts.length == 7) {
-            long logSegmentId = Long.parseLong(parts[1]);
-            long txId = Long.parseLong(parts[2]);
-            long logSegmentSequenceNumber = Long.parseLong(parts[3]);
-            long minActiveEntryId = Long.parseLong(parts[4]);
-            long minActiveSlotId = Long.parseLong(parts[5]);
-            long startSequenceId = Long.parseLong(parts[6]);
-
-            LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
-                    .setLogSegmentSequenceNo(logSegmentSequenceNumber)
-                    .setMinActiveEntryId(minActiveEntryId)
-                    .setMinActiveSlotId(minActiveSlotId)
-                    .setRegionId(regionId)
-                    .setStatus(status)
-                    .setStartSequenceId(startSequenceId)
-                    .setEnvelopeEntries(true);
-            return builder.build();
-        } else if (parts.length == 11) {
-            long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
-            assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
-
-            long logSegmentId = Long.parseLong(parts[1]);
-            long firstTxId = Long.parseLong(parts[2]);
-            long lastTxId = Long.parseLong(parts[3]);
-            long completionTime = Long.parseLong(parts[4]);
-            long logSegmentSequenceNumber = Long.parseLong(parts[5]);
-            long lastEntryId = Long.parseLong(parts[6]);
-            long lastSlotId = Long.parseLong(parts[7]);
-            long minActiveEntryId = Long.parseLong(parts[8]);
-            long minActiveSlotId = Long.parseLong(parts[9]);
-            long startSequenceId = Long.parseLong(parts[10]);
-            LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
-                    .setInprogress(false)
-                    .setLastTxId(lastTxId)
-                    .setCompletionTime(completionTime)
-                    .setRecordCount((int) recordCount)
-                    .setLogSegmentSequenceNo(logSegmentSequenceNumber)
-                    .setLastEntryId(lastEntryId)
-                    .setLastSlotId(lastSlotId)
-                    .setMinActiveEntryId(minActiveEntryId)
-                    .setMinActiveSlotId(minActiveSlotId)
-                    .setRegionId(regionId)
-                    .setStatus(status)
-                    .setStartSequenceId(startSequenceId)
-                    .setEnvelopeEntries(true);
-            return builder.build();
-        } else {
-            throw new IOException("Invalid log segment metadata : "
-                    + new String(data, UTF_8));
-        }
-    }
-
-    public static LogSegmentMetadata parseData(String path, byte[] data)
-            throws IOException {
-        return parseData(path, data, false);
-    }
-
-    static LogSegmentMetadata parseData(String path, byte[] data, boolean skipMinVersionCheck) throws IOException {
-        String[] parts = new String(data, UTF_8).split(";");
-        long version;
-        try {
-            version = Long.parseLong(parts[0]) & METADATA_VERSION_MASK;
-        } catch (Exception exc) {
-            throw new IOException("Invalid ledger entry, "
-                + new String(data, UTF_8));
-        }
-
-        if (!skipMinVersionCheck && version < LogSegmentMetadata.LEDGER_METADATA_OLDEST_SUPPORTED_VERSION) {
-            throw new UnsupportedMetadataVersionException("Ledger metadata version '" + version + "' is no longer supported: "
-                + new String(data, UTF_8));
-        }
-
-        if (version > LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION) {
-            throw new UnsupportedMetadataVersionException("Metadata version '" + version + "' is higher than the highest supported version : "
-                + new String(data, UTF_8));
-        }
-
-        if (LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value == version) {
-            return parseDataV1(path, data, parts);
-        } else if (LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value == version) {
-            return parseDataV2(path, data, parts);
-        } else if (LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version &&
-                   LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version) {
-            return parseDataVersionsWithMinActiveDLSN(path, data, parts);
-        } else {
-            assert(version >= LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value);
-            return parseDataVersionsWithSequenceId(path, data, parts);
-        }
-    }
-
-    public String getFinalisedData() {
-        return getFinalisedData(this.version);
-    }
-
-    public String getFinalisedData(LogSegmentMetadataVersion version) {
-        String finalisedData;
-        final long logSegmentSeqNo = getLogSegmentSequenceNumber();
-        final long lastEntryId = getLastEntryId();
-        final long lastSlotId = getLastSlotId();
-        final long minActiveEntryId = minActiveDLSN.getEntryId();
-        final long minActiveSlotId = minActiveDLSN.getSlotId();
-
-        if (LogSegmentMetadataVersion.VERSION_V1_ORIGINAL == version) {
-            if (inprogress) {
-                finalisedData = String.format("%d;%d;%d",
-                    version.value, logSegmentId, firstTxId);
-            } else {
-                long versionAndCount = ((long) version.value) | ((long)recordCount << LOGRECORD_COUNT_SHIFT);
-                finalisedData = String.format("%d;%d;%d;%d;%d",
-                    versionAndCount, logSegmentId, firstTxId, lastTxId, completionTime);
-            }
-        } else {
-            long versionStatusCount = ((long) version.value);
-            versionStatusCount |= ((status & METADATA_STATUS_BIT_MAX) << STATUS_BITS_SHIFT);
-            versionStatusCount |= (((long) regionId & MAX_REGION_ID) << REGION_SHIFT);
-            if (!inprogress) {
-                versionStatusCount |= ((long)recordCount << LOGRECORD_COUNT_SHIFT);
-            }
-            if (LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO == version) {
-                if (inprogress) {
-                    finalisedData = String.format("%d;%d;%d;%d",
-                        versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo);
-                } else {
-                    finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d",
-                        versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
-                        logSegmentSeqNo, lastEntryId, lastSlotId);
-                }
-            } else if (LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version.value &&
-                        LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version.value) {
-                if (inprogress) {
-                    finalisedData = String.format("%d;%d;%d;%d;%d;%d",
-                        versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo, minActiveEntryId, minActiveSlotId);
-                } else {
-                    finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d;%d;%d",
-                        versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
-                        logSegmentSeqNo, lastEntryId, lastSlotId, minActiveEntryId, minActiveSlotId);
-                }
-            } else if (LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value <= version.value &&
-                        LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION >= version.value) {
-                if (inprogress) {
-                    finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d",
-                        versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo, minActiveEntryId, minActiveSlotId, startSequenceId);
-                } else {
-                    finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d;%d;%d;%d",
-                        versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
-                        logSegmentSeqNo, lastEntryId, lastSlotId, minActiveEntryId, minActiveSlotId, startSequenceId);
-                }
-            } else {
-                throw new IllegalStateException("Unsupported log segment ledger metadata version '" + version + "'");
-            }
-        }
-        return finalisedData;
-    }
-
-    String getSegmentName() {
-        String[] parts = this.zkPath.split("/");
-        if (parts.length <= 0) {
-            throw new IllegalStateException("ZK Path is not valid");
-        }
-        return parts[parts.length - 1];
-    }
-
-    public void write(ZooKeeperClient zkc)
-        throws IOException, KeeperException.NodeExistsException {
-        String finalisedData = getFinalisedData(version);
-        try {
-            zkc.get().create(zkPath, finalisedData.getBytes(UTF_8),
-                zkc.getDefaultACL(), CreateMode.PERSISTENT);
-        } catch (KeeperException.NodeExistsException nee) {
-            throw nee;
-        } catch (InterruptedException ie) {
-            throw new DLInterruptedException("Interrupted on creating ledger znode " + zkPath, ie);
-        } catch (Exception e) {
-            LOG.error("Error creating ledger znode {}", zkPath, e);
-            throw new IOException("Error creating ledger znode " + zkPath);
-        }
-    }
-
-    boolean checkEquivalence(ZooKeeperClient zkc, String path) {
-        try {
-            LogSegmentMetadata other = FutureUtils.result(read(zkc, path));
-            if (LOG.isTraceEnabled()) {
-                LOG.trace("Verifying {} against {}", this, other);
-            }
-
-            boolean retVal;
-
-            // All fields may not be comparable so only compare the ones
-            // that can be compared
-            // completionTime is set when a node is finalized, so that
-            // cannot be compared
-            // if the node is inprogress, don't compare the lastTxId either
-            if (this.getLogSegmentSequenceNumber() != other.getLogSegmentSequenceNumber() ||
-                this.logSegmentId != other.logSegmentId ||
-                this.firstTxId != other.firstTxId) {
-                retVal = false;
-            } else if (this.inprogress) {
-                retVal = other.inprogress;
-            } else {
-                retVal = (!other.inprogress && (this.lastTxId == other.lastTxId));
-            }
-
-            if (!retVal) {
-                LOG.warn("Equivalence check failed between {} and {}", this, other);
-            }
-
-            return retVal;
-        } catch (Exception e) {
-            LOG.error("Could not check equivalence between:" + this + " and data in " + path, e);
-            return false;
-        }
-    }
-
-    public boolean equals(Object o) {
-        if (!(o instanceof LogSegmentMetadata)) {
-            return false;
-        }
-        LogSegmentMetadata ol = (LogSegmentMetadata) o;
-        return getLogSegmentSequenceNumber() == ol.getLogSegmentSequenceNumber()
-            && logSegmentId == ol.logSegmentId
-            && firstTxId == ol.firstTxId
-            && lastTxId == ol.lastTxId
-            && version == ol.version
-            && completionTime == ol.completionTime
-            && Objects.equal(lastDLSN, ol.lastDLSN)
-            && Objects.equal(minActiveDLSN, ol.minActiveDLSN)
-            && startSequenceId == ol.startSequenceId
-            && status == ol.status;
-    }
-
-    public int hashCode() {
-        int hash = 1;
-        hash = hash * 31 + (int) logSegmentId;
-        hash = hash * 31 + (int) firstTxId;
-        hash = hash * 31 + (int) lastTxId;
-        hash = hash * 31 + version.value;
-        hash = hash * 31 + (int) completionTime;
-        hash = hash * 31 + (int) getLogSegmentSequenceNumber();
-        return hash;
-    }
-
-    public String toString() {
-        return "[LogSegmentId:" + logSegmentId +
-            ", firstTxId:" + firstTxId +
-            ", lastTxId:" + lastTxId +
-            ", version:" + version +
-            ", completionTime:" + completionTime +
-            ", recordCount:" + recordCount +
-            ", regionId:" + regionId +
-            ", status:" + status +
-            ", logSegmentSequenceNumber:" + getLogSegmentSequenceNumber() +
-            ", lastEntryId:" + getLastEntryId() +
-            ", lastSlotId:" + getLastSlotId() +
-            ", inprogress:" + inprogress +
-            ", minActiveDLSN:" + minActiveDLSN +
-            ", startSequenceId:" + startSequenceId +
-            "]";
-    }
-
-    public Mutator mutator() {
-        return new Mutator(this);
-    }
-
-
-    //
-    // Version Checking Utilities
-    //
-
-    public boolean supportsLogSegmentSequenceNo() {
-        return supportsLogSegmentSequenceNo(version.value);
-    }
-
-    /**
-     * Whether the provided version supports log segment sequence number.
-     *
-     * @param version
-     *          log segment metadata version
-     * @return true if this log segment supports log segment sequence number.
-     */
-    public static boolean supportsLogSegmentSequenceNo(int version) {
-        return version >= LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value;
-    }
-
-    /**
-     * Whether the provided version supports enveloping entries before writing to bookkeeper.
-     *
-     * @param version
-     *          log segment metadata version
-     * @return true if this log segment supports enveloping entries
-     */
-    public static boolean supportsEnvelopedEntries(int version) {
-        return version >= LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value;
-    }
-
-    public boolean supportsSequenceId() {
-        return supportsSequenceId(version.value);
-    }
-
-    /**
-     * Whether the provided version supports sequence id.
-     *
-     * @param version
-     *          log segment metadata version
-     * @return true if the log segment support sequence id.
-     */
-    public static boolean supportsSequenceId(int version) {
-        return version >= LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value;
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/LogWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/LogWriter.java
deleted file mode 100644
index d7de586..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/LogWriter.java
+++ /dev/null
@@ -1,78 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.io.Abortable;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-
-/*
-* 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.
-     *
-     * @param record single log record
-     * @throws IOException
-     */
-    public void write(LogRecord record) throws IOException;
-
-
-    /**
-     * Write a list of log records to the stream.
-     *
-     * @param records list of log records
-     * @throws IOException
-     */
-    @Deprecated
-    public int writeBulk(List<LogRecord> records) throws IOException;
-
-    /**
-     * All data that has been written to the stream so far will be sent to
-     * persistent storage.
-     * The transmission is asynchronous and new data can be still written to the
-     * stream while flushing is performed.
-     *
-     * TODO: rename this to flush()
-     */
-    public long setReadyToFlush() throws IOException;
-
-    /**
-     * Flush and sync all data that is ready to be flush
-     * {@link #setReadyToFlush()} into underlying persistent store.
-     * @throws IOException
-     *
-     * TODO: rename this to commit()
-     */
-    public long flushAndSync() throws IOException;
-
-    /**
-     * Flushes all the data up to this point,
-     * adds the end of stream marker and marks the stream
-     * as read-only in the metadata. No appends to the
-     * stream will be allowed after this point
-     *
-     * @throws IOException
-     */
-    public void markEndOfStream() throws IOException;
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxLogSegmentSequenceNo.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxLogSegmentSequenceNo.java
deleted file mode 100644
index 9bfaaba..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxLogSegmentSequenceNo.java
+++ /dev/null
@@ -1,72 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.util.DLUtils;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
-
-/**
- * Utility class for storing and reading max ledger sequence number
- */
-class MaxLogSegmentSequenceNo {
-
-    Version version;
-    long maxSeqNo;
-
-    MaxLogSegmentSequenceNo(Versioned<byte[]> logSegmentsData) {
-        if (null != logSegmentsData
-                && null != logSegmentsData.getValue()
-                && null != logSegmentsData.getVersion()) {
-            version = logSegmentsData.getVersion();
-            try {
-                maxSeqNo = DLUtils.deserializeLogSegmentSequenceNumber(logSegmentsData.getValue());
-            } catch (NumberFormatException nfe) {
-                maxSeqNo = DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
-            }
-        } else {
-            maxSeqNo = DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
-            if (null != logSegmentsData && null != logSegmentsData.getVersion()) {
-                version = logSegmentsData.getVersion();
-            } else {
-                throw new IllegalStateException("Invalid MaxLogSegmentSequenceNo found - " + logSegmentsData);
-            }
-        }
-    }
-
-    synchronized Version getVersion() {
-        return version;
-    }
-
-    synchronized long getSequenceNumber() {
-        return maxSeqNo;
-    }
-
-    synchronized MaxLogSegmentSequenceNo update(Version version, long logSegmentSeqNo) {
-        if (version.compare(this.version) == Version.Occurred.AFTER) {
-            this.version = version;
-            this.maxSeqNo = logSegmentSeqNo;
-        }
-        return this;
-    }
-
-    public synchronized Versioned<Long> getVersionedData(long seqNo) {
-        return new Versioned<Long>(seqNo, version);
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxTxId.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxTxId.java
deleted file mode 100644
index 8eabf88..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxTxId.java
+++ /dev/null
@@ -1,72 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.util.DLUtils;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Utility class for storing and reading
- * the max seen txid in zookeeper
- */
-class MaxTxId {
-    static final Logger LOG = LoggerFactory.getLogger(MaxTxId.class);
-
-    private Version version;
-    private long currentMax;
-
-    MaxTxId(Versioned<byte[]> maxTxIdData) {
-        if (null != maxTxIdData
-                && null != maxTxIdData.getValue()
-                && null != maxTxIdData.getVersion()) {
-            this.version = maxTxIdData.getVersion();
-            try {
-                this.currentMax = DLUtils.deserializeTransactionId(maxTxIdData.getValue());
-            } catch (NumberFormatException e) {
-                LOG.warn("Invalid txn id stored in {}", e);
-                this.currentMax = DistributedLogConstants.INVALID_TXID;
-            }
-        } else {
-            this.currentMax = DistributedLogConstants.INVALID_TXID;
-            if (null != maxTxIdData && null != maxTxIdData.getVersion()) {
-                this.version = maxTxIdData.getVersion();
-            } else {
-                throw new IllegalStateException("Invalid MaxTxId found - " + maxTxIdData);
-            }
-        }
-    }
-
-    synchronized void update(Version version, long txId) {
-        if (version.compare(this.version) == Version.Occurred.AFTER) {
-            this.version = version;
-            this.currentMax = txId;
-        }
-    }
-
-    synchronized long get() {
-        return currentMax;
-    }
-
-    public synchronized Versioned<Long> getVersionedData(long txId) {
-        return new Versioned<Long>(Math.max(txId, get()), version);
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/MetadataAccessor.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/MetadataAccessor.java
deleted file mode 100644
index f6ff587..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/MetadataAccessor.java
+++ /dev/null
@@ -1,43 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.io.AsyncCloseable;
-
-import java.io.Closeable;
-import java.io.IOException;
-
-public interface MetadataAccessor extends Closeable, AsyncCloseable {
-    /**
-     * Get the name of the stream managed by this log manager
-     * @return streamName
-     */
-    public String getStreamName();
-
-    public void createOrUpdateMetadata(byte[] metadata) throws IOException;
-
-    public void deleteMetadata() throws IOException;
-
-    public byte[] getMetadata() throws IOException;
-
-    /**
-     * Close the distributed log metadata, freeing any resources it may hold.
-     */
-    public void close() throws IOException;
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java
deleted file mode 100644
index 40e3930..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java
+++ /dev/null
@@ -1,992 +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 com.twitter.distributedlog;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Stopwatch;
-import com.google.common.base.Ticker;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.callback.LogSegmentListener;
-import com.twitter.distributedlog.exceptions.AlreadyTruncatedTransactionException;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.EndOfLogSegmentException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryReader;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import com.twitter.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;
-import java.util.List;
-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;
-
-/**
- * New ReadAhead Reader that uses {@link com.twitter.distributedlog.logsegment.LogSegmentEntryReader}.
- *
- * NOTE: all the state changes happen in the same thread. All *unsafe* methods should be submitted to the order
- * scheduler using stream name as the key.
- */
-public class ReadAheadEntryReader implements
-        AsyncCloseable,
-        LogSegmentListener,
-        LogSegmentEntryReader.StateChangeListener,
-        FutureEventListener<List<Entry.Reader>> {
-
-    private static final Logger logger = LoggerFactory.getLogger(ReadAheadEntryReader.class);
-
-    //
-    // 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;
-        }
-    };
-
-    //
-    // Internal Classes
-    //
-
-    class SegmentReader implements FutureEventListener<LogSegmentEntryReader> {
-
-        private LogSegmentMetadata metadata;
-        private final long startEntryId;
-        private Future<LogSegmentEntryReader> openFuture = null;
-        private LogSegmentEntryReader reader = null;
-        private boolean isStarted = false;
-        private boolean isClosed = false;
-
-        SegmentReader(LogSegmentMetadata metadata,
-                      long startEntryId) {
-            this.metadata = metadata;
-            this.startEntryId = startEntryId;
-        }
-
-        synchronized LogSegmentEntryReader getEntryReader() {
-            return reader;
-        }
-
-        synchronized boolean isBeyondLastAddConfirmed() {
-            return null != reader && reader.isBeyondLastAddConfirmed();
-        }
-
-        synchronized LogSegmentMetadata getSegment() {
-            return metadata;
-        }
-
-        synchronized boolean isReaderOpen() {
-            return null != openFuture;
-        }
-
-        synchronized void openReader() {
-            if (null != openFuture) {
-                return;
-            }
-            openFuture = entryStore.openReader(metadata, startEntryId).addEventListener(this);
-        }
-
-        synchronized boolean isReaderStarted() {
-            return isStarted;
-        }
-
-        synchronized void startRead() {
-            if (isStarted) {
-                return;
-            }
-            isStarted = true;
-            if (null != reader) {
-                reader.start();
-            } else {
-                openFuture.onSuccess(START_READER_FUNC);
-            }
-        }
-
-        synchronized Future<List<Entry.Reader>> readNext() {
-            if (null != reader) {
-                checkCatchingUpStatus(reader);
-                return reader.readNext(numReadAheadEntries);
-            } else {
-                return openFuture.flatMap(readFunc);
-            }
-        }
-
-        synchronized void updateLogSegmentMetadata(final LogSegmentMetadata segment) {
-            if (null != reader) {
-                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;
-                    }
-                });
-            }
-        }
-
-        @Override
-        synchronized public void onSuccess(LogSegmentEntryReader reader) {
-            this.reader = reader;
-            if (reader.getSegment().isInProgress()) {
-                reader.registerListener(ReadAheadEntryReader.this);
-            }
-        }
-
-        @Override
-        public void onFailure(Throwable cause) {
-            // no-op, the failure will be propagated on first read.
-        }
-
-        synchronized boolean isClosed() {
-            return isClosed;
-        }
-
-        synchronized Future<Void> close() {
-            if (null == openFuture) {
-                return Future.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() {
-                    synchronized (SegmentReader.this) {
-                        isClosed = true;
-                    }
-                    return null;
-                }
-            });
-        }
-    }
-
-    private class ReadEntriesFunc extends AbstractFunction1<LogSegmentEntryReader, Future<List<Entry.Reader>>> {
-
-        private final int numEntries;
-
-        ReadEntriesFunc(int numEntries) {
-            this.numEntries = numEntries;
-        }
-
-        @Override
-        public Future<List<Entry.Reader>> apply(LogSegmentEntryReader reader) {
-            checkCatchingUpStatus(reader);
-            return reader.readNext(numEntries);
-        }
-    }
-
-    private abstract class CloseableRunnable implements Runnable {
-
-        @Override
-        public void run() {
-            synchronized (ReadAheadEntryReader.this) {
-                if (null != closePromise) {
-                    return;
-                }
-            }
-            try {
-                safeRun();
-            } catch (Throwable cause) {
-                logger.error("Caught unexpected exception : ", cause);
-            }
-        }
-
-        abstract void safeRun();
-
-    }
-
-    //
-    // 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;
-        }
-    };
-
-    //
-    // Resources
-    //
-    private final DistributedLogConfiguration conf;
-    private final BKLogReadHandler readHandler;
-    private final LogSegmentEntryStore entryStore;
-    private final OrderedScheduler scheduler;
-
-    //
-    // Parameters
-    //
-    private final String streamName;
-    private final DLSN fromDLSN;
-    private final int maxCachedEntries;
-    private final int numReadAheadEntries;
-    private final int idleWarnThresholdMillis;
-
-    //
-    // Cache
-    //
-    private final LinkedBlockingQueue<Entry.Reader> entryQueue;
-
-    //
-    // State of the reader
-    //
-
-    private final AtomicBoolean started = new AtomicBoolean(false);
-    private boolean isInitialized = false;
-    private boolean readAheadPaused = false;
-    private Promise<Void> closePromise = null;
-    // segment readers
-    private long currentSegmentSequenceNumber;
-    private SegmentReader currentSegmentReader;
-    private SegmentReader nextSegmentReader;
-    private DLSN lastDLSN;
-    private final EntryPosition nextEntryPosition;
-    private volatile boolean isCatchingUp = true;
-    private final LinkedList<SegmentReader> segmentReaders;
-    private final LinkedList<SegmentReader> segmentReadersToClose;
-    // last exception that this reader encounters
-    private final AtomicReference<IOException> lastException = new AtomicReference<IOException>(null);
-    // last entry added time
-    private final Stopwatch lastEntryAddedTime;
-    // state change notification
-    private final CopyOnWriteArraySet<AsyncNotification> stateChangeNotifications =
-            new CopyOnWriteArraySet<AsyncNotification>();
-    // idle reader check task
-    private final ScheduledFuture<?> idleReaderCheckTask;
-
-    //
-    // Stats
-    //
-    private final AlertStatsLogger alertStatsLogger;
-
-    public ReadAheadEntryReader(String streamName,
-                                DLSN fromDLSN,
-                                DistributedLogConfiguration conf,
-                                BKLogReadHandler readHandler,
-                                LogSegmentEntryStore entryStore,
-                                OrderedScheduler scheduler,
-                                Ticker ticker,
-                                AlertStatsLogger alertStatsLogger) {
-        this.streamName = streamName;
-        this.fromDLSN = lastDLSN = fromDLSN;
-        this.nextEntryPosition = new EntryPosition(
-                fromDLSN.getLogSegmentSequenceNo(),
-                fromDLSN.getEntryId());
-        this.conf = conf;
-        this.maxCachedEntries = conf.getReadAheadMaxRecords();
-        this.numReadAheadEntries = conf.getReadAheadBatchSize();
-        this.idleWarnThresholdMillis = conf.getReaderIdleWarnThresholdMillis();
-        this.readHandler = readHandler;
-        this.entryStore = entryStore;
-        this.scheduler = scheduler;
-        this.readFunc = new ReadEntriesFunc(numReadAheadEntries);
-        this.alertStatsLogger = alertStatsLogger;
-
-        // create the segment reader list
-        this.segmentReaders = new LinkedList<SegmentReader>();
-        this.segmentReadersToClose = new LinkedList<SegmentReader>();
-        // create the readahead entry queue
-        this.entryQueue = new LinkedBlockingQueue<Entry.Reader>();
-
-        // start the idle reader detection
-        lastEntryAddedTime = Stopwatch.createStarted(ticker);
-        // start the idle reader check task
-        idleReaderCheckTask = scheduleIdleReaderTaskIfNecessary();
-    }
-
-    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();
-                }
-            }, idleWarnThresholdMillis, idleWarnThresholdMillis, TimeUnit.MILLISECONDS);
-        }
-        return null;
-    }
-
-    private void unsafeCheckIfReadAheadIsIdle() {
-        boolean forceReadLogSegments =
-                (null == currentSegmentReader) || currentSegmentReader.isBeyondLastAddConfirmed();
-        if (forceReadLogSegments) {
-            readHandler.readLogSegmentsFromStore(
-                    LogSegmentMetadata.COMPARATOR,
-                    LogSegmentFilter.DEFAULT_FILTER,
-                    null
-            ).addEventListener(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
-                }
-
-                @Override
-                public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
-                    onSegmentsUpdated(segments.getValue());
-                }
-            });
-        }
-    }
-
-    private void cancelIdleReaderTask() {
-        if (null != idleReaderCheckTask) {
-            idleReaderCheckTask.cancel(true);
-        }
-    }
-
-    @VisibleForTesting
-    EntryPosition getNextEntryPosition() {
-        return nextEntryPosition;
-    }
-
-    @VisibleForTesting
-    SegmentReader getCurrentSegmentReader() {
-        return currentSegmentReader;
-    }
-
-    @VisibleForTesting
-    long getCurrentSegmentSequenceNumber() {
-        return currentSegmentSequenceNumber;
-    }
-
-    @VisibleForTesting
-    SegmentReader getNextSegmentReader() {
-        return nextSegmentReader;
-    }
-
-    @VisibleForTesting
-    LinkedList<SegmentReader> getSegmentReaders() {
-        return segmentReaders;
-    }
-
-    @VisibleForTesting
-    boolean isInitialized() {
-        return isInitialized;
-    }
-
-    private void orderedSubmit(Runnable runnable) {
-        synchronized (this) {
-            if (null != closePromise) {
-                return;
-            }
-        }
-        try {
-            scheduler.submit(streamName, runnable);
-        } catch (RejectedExecutionException ree) {
-            logger.debug("Failed to submit and execute an operation for readhead entry reader of {}",
-                    streamName, ree);
-        }
-    }
-
-    public void start(final List<LogSegmentMetadata> segmentList) {
-        logger.info("Starting the readahead entry reader for {} : segments = {}",
-                readHandler.getFullyQualifiedName(), segmentList);
-        started.set(true);
-        processLogSegments(segmentList);
-    }
-
-    private void removeClosedSegmentReaders() {
-        orderedSubmit(new CloseableRunnable() {
-            @Override
-            void safeRun() {
-                unsafeRemoveClosedSegmentReaders();
-            }
-        });
-    }
-
-    private void unsafeRemoveClosedSegmentReaders() {
-        SegmentReader reader = segmentReadersToClose.peekFirst();
-        while (null != reader) {
-            if (reader.isClosed()) {
-                segmentReadersToClose.pollFirst();
-                reader = segmentReadersToClose.peekFirst();
-            } else {
-                break;
-            }
-        }
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        final Promise<Void> closeFuture;
-        synchronized (this) {
-            if (null != closePromise) {
-                return closePromise;
-            }
-            closePromise = closeFuture = new Promise<Void>();
-        }
-
-        // cancel the idle reader task
-        cancelIdleReaderTask();
-
-        // use runnable here instead of CloseableRunnable,
-        // because we need this to be executed
-        try {
-            scheduler.submit(streamName, new Runnable() {
-                @Override
-                public void run() {
-                    unsafeAsyncClose(closeFuture);
-                }
-            });
-        } catch (RejectedExecutionException ree) {
-            logger.warn("Scheduler has been shutdown before closing the readahead entry reader for stream {}",
-                    streamName, ree);
-            unsafeAsyncClose(closeFuture);
-        }
-
-        return closeFuture;
-    }
-
-    private void unsafeAsyncClose(Promise<Void> closePromise) {
-        List<Future<Void>> closeFutures = Lists.newArrayListWithExpectedSize(
-                segmentReaders.size() + segmentReadersToClose.size() + 1);
-        if (null != currentSegmentReader) {
-            segmentReadersToClose.add(currentSegmentReader);
-        }
-        if (null != nextSegmentReader) {
-            segmentReadersToClose.add(nextSegmentReader);
-        }
-        for (SegmentReader reader : segmentReaders) {
-            segmentReadersToClose.add(reader);
-        }
-        segmentReaders.clear();
-        for (SegmentReader reader : segmentReadersToClose) {
-            closeFutures.add(reader.close());
-        }
-        Futures.collect(closeFutures).proxyTo(closePromise);
-    }
-
-    //
-    // Reader State Changes
-    //
-
-    ReadAheadEntryReader addStateChangeNotification(AsyncNotification notification) {
-        this.stateChangeNotifications.add(notification);
-        return this;
-    }
-
-    ReadAheadEntryReader removeStateChangeNotification(AsyncNotification notification) {
-        this.stateChangeNotifications.remove(notification);
-        return this;
-    }
-
-    private void notifyStateChangeOnSuccess() {
-        for (AsyncNotification notification : stateChangeNotifications) {
-            notification.notifyOnOperationComplete();
-        }
-    }
-
-    private void notifyStateChangeOnFailure(Throwable cause) {
-        for (AsyncNotification notification : stateChangeNotifications) {
-            notification.notifyOnError(cause);
-        }
-    }
-
-    void setLastException(IOException cause) {
-        if (!lastException.compareAndSet(null, cause)) {
-            logger.debug("last exception has already been set to ", lastException.get());
-        }
-        // the exception is set and notify the state change
-        notifyStateChangeOnFailure(cause);
-    }
-
-    void checkLastException() throws IOException {
-        if (null != lastException.get()) {
-            throw lastException.get();
-        }
-    }
-
-    void checkCatchingUpStatus(LogSegmentEntryReader reader) {
-        if (reader.getSegment().isInProgress()
-                && isCatchingUp
-                && reader.hasCaughtUpOnInprogress()) {
-            logger.info("ReadAhead for {} is caught up at entry {} @ log segment {}.",
-                    new Object[] { readHandler.getFullyQualifiedName(),
-                            reader.getLastAddConfirmed(), reader.getSegment() });
-            isCatchingUp = false;
-        }
-    }
-
-    void markCaughtup() {
-        if (isCatchingUp) {
-            isCatchingUp = false;
-            logger.info("ReadAhead for {} is caught up", readHandler.getFullyQualifiedName());
-        }
-    }
-
-    public boolean isReadAheadCaughtUp() {
-        return !isCatchingUp;
-    }
-
-    @Override
-    public void onCaughtupOnInprogress() {
-        markCaughtup();
-    }
-
-    //
-    // ReadAhead State Machine
-    //
-
-    @Override
-    public void onSuccess(List<Entry.Reader> entries) {
-        lastEntryAddedTime.reset().start();
-        for (Entry.Reader entry : entries) {
-            entryQueue.add(entry);
-        }
-        if (!entries.isEmpty()) {
-            Entry.Reader lastEntry = entries.get(entries.size() - 1);
-            nextEntryPosition.advance(lastEntry.getLSSN(), lastEntry.getEntryId() + 1);
-        }
-        // notify on data available
-        notifyStateChangeOnSuccess();
-        if (entryQueue.size() >= maxCachedEntries) {
-            pauseReadAheadOnCacheFull();
-        } else {
-            scheduleReadNext();
-        }
-    }
-
-    @Override
-    public void onFailure(Throwable cause) {
-        if (cause instanceof EndOfLogSegmentException) {
-            // we reach end of the log segment
-            moveToNextLogSegment();
-            return;
-        }
-        if (cause instanceof IOException) {
-            setLastException((IOException) cause);
-        } else {
-            setLastException(new UnexpectedException("Unexpected non I/O exception", cause));
-        }
-    }
-
-    private synchronized void invokeReadAhead() {
-        if (readAheadPaused) {
-            scheduleReadNext();
-            readAheadPaused = false;
-        }
-    }
-
-    private synchronized void pauseReadAheadOnCacheFull() {
-        this.readAheadPaused = true;
-        if (!isCacheFull()) {
-            invokeReadAhead();
-        }
-    }
-
-    private synchronized void pauseReadAheadOnNoMoreLogSegments() {
-        this.readAheadPaused = true;
-    }
-
-    //
-    // Cache Related Methods
-    //
-
-    public Entry.Reader getNextReadAheadEntry(long waitTime, TimeUnit waitTimeUnit) throws IOException {
-        if (null != lastException.get()) {
-            throw lastException.get();
-        }
-        Entry.Reader entry;
-        try {
-            entry = entryQueue.poll(waitTime, waitTimeUnit);
-        } catch (InterruptedException e) {
-            throw new DLInterruptedException("Interrupted on waiting next readahead entry : ", e);
-        }
-        try {
-            return entry;
-        } finally {
-            // resume readahead if the cache becomes empty
-            if (null != entry && !isCacheFull()) {
-                invokeReadAhead();
-            }
-        }
-    }
-
-    /**
-     * Return number cached entries.
-     *
-     * @return number cached entries.
-     */
-    public int getNumCachedEntries() {
-        return entryQueue.size();
-    }
-
-    /**
-     * Return if the cache is full.
-     *
-     * @return true if the cache is full, otherwise false.
-     */
-    public boolean isCacheFull() {
-        return getNumCachedEntries() >= maxCachedEntries;
-    }
-
-    @VisibleForTesting
-    public boolean isCacheEmpty() {
-        return entryQueue.isEmpty();
-    }
-
-    /**
-     * Check whether the readahead becomes stall.
-     *
-     * @param idleReaderErrorThreshold idle reader error threshold
-     * @param timeUnit time unit of the idle reader error threshold
-     * @return true if the readahead becomes stall, otherwise false.
-     */
-    public boolean isReaderIdle(int idleReaderErrorThreshold, TimeUnit timeUnit) {
-        return (lastEntryAddedTime.elapsed(timeUnit) > idleReaderErrorThreshold);
-    }
-
-    //
-    // LogSegment Management
-    //
-
-    void processLogSegments(final List<LogSegmentMetadata> segments) {
-        orderedSubmit(new CloseableRunnable() {
-            @Override
-            void safeRun() {
-                unsafeProcessLogSegments(segments);
-            }
-        });
-    }
-
-    private void unsafeProcessLogSegments(List<LogSegmentMetadata> segments) {
-        if (isInitialized) {
-            unsafeReinitializeLogSegments(segments);
-        } else {
-            unsafeInitializeLogSegments(segments);
-        }
-    }
-
-    /**
-     * Update the log segment metadata.
-     *
-     * @param reader the reader to update the metadata
-     * @param newMetadata the new metadata received
-     * @return true if successfully, false on encountering errors
-     */
-    private boolean updateLogSegmentMetadata(SegmentReader reader,
-                                             LogSegmentMetadata newMetadata) {
-        if (reader.getSegment().getLogSegmentSequenceNumber() != newMetadata.getLogSegmentSequenceNumber()) {
-            setLastException(new DLIllegalStateException("Inconsistent state found in entry reader for "
-                    + streamName + " : current segment = " + reader.getSegment() + ", new segment = " + newMetadata));
-            return false;
-        }
-        if (!reader.getSegment().isInProgress() && newMetadata.isInProgress()) {
-            setLastException(new DLIllegalStateException("An inprogress log segment " + newMetadata
-                    + " received after a closed log segment " + reader.getSegment() + " on reading segment "
-                    + newMetadata.getLogSegmentSequenceNumber() + " @ stream " + streamName));
-            return false;
-        }
-        if (reader.getSegment().isInProgress() && !newMetadata.isInProgress()) {
-            reader.updateLogSegmentMetadata(newMetadata);
-        }
-        return true;
-    }
-
-    /**
-     * Reinitialize the log segments
-     */
-    private void unsafeReinitializeLogSegments(List<LogSegmentMetadata> segments) {
-        logger.info("Reinitialize log segments with {}", segments);
-        int segmentIdx = 0;
-        for (; segmentIdx < segments.size(); segmentIdx++) {
-            LogSegmentMetadata segment = segments.get(segmentIdx);
-            if (segment.getLogSegmentSequenceNumber() < currentSegmentSequenceNumber) {
-                continue;
-            }
-            break;
-        }
-        if (segmentIdx >= segments.size()) {
-            return;
-        }
-        LogSegmentMetadata segment = segments.get(segmentIdx);
-        if (null != currentSegmentReader) {
-            if (!updateLogSegmentMetadata(currentSegmentReader, segment)) {
-                return;
-            }
-        } else {
-            if (currentSegmentSequenceNumber != segment.getLogSegmentSequenceNumber()) {
-                setLastException(new DLIllegalStateException("Inconsistent state found in entry reader for "
-                        + streamName + " : current segment sn = " + currentSegmentSequenceNumber
-                        + ", new segment sn = " + segment.getLogSegmentSequenceNumber()));
-                return;
-            }
-        }
-        segmentIdx++;
-        if (segmentIdx >= segments.size()) {
-            return;
-        }
-        // check next segment
-        segment = segments.get(segmentIdx);
-        if (null != nextSegmentReader) {
-            if (!updateLogSegmentMetadata(nextSegmentReader, segment)) {
-                return;
-            }
-            segmentIdx++;
-        }
-        // check the segment readers in the queue
-        for (int readerIdx = 0;
-             readerIdx < segmentReaders.size() && segmentIdx < segments.size();
-             readerIdx++, segmentIdx++) {
-            SegmentReader reader = segmentReaders.get(readerIdx);
-            segment = segments.get(segmentIdx);
-            if (!updateLogSegmentMetadata(reader, segment)) {
-                return;
-            }
-        }
-        // add the remaining segments to the reader queue
-        for (; segmentIdx < segments.size(); segmentIdx++) {
-            segment = segments.get(segmentIdx);
-            SegmentReader reader = new SegmentReader(segment, 0L);
-            reader.openReader();
-            segmentReaders.add(reader);
-        }
-        if (null == currentSegmentReader) {
-            unsafeMoveToNextLogSegment();
-        }
-        // resume readahead if necessary
-        invokeReadAhead();
-    }
-
-    /**
-     * Initialize the reader with the log <i>segments</i>.
-     *
-     * @param segments list of log segments
-     */
-    private void unsafeInitializeLogSegments(List<LogSegmentMetadata> segments) {
-        if (segments.isEmpty()) {
-            // not initialize the background reader, until the first log segment is notified
-            return;
-        }
-        boolean skipTruncatedLogSegments = true;
-        DLSN dlsnToStart = fromDLSN;
-        // positioning the reader
-        for (int i = 0; i < segments.size(); i++) {
-            LogSegmentMetadata segment = segments.get(i);
-            // skip any log segments that have smaller log segment sequence numbers
-            if (segment.getLogSegmentSequenceNumber() < fromDLSN.getLogSegmentSequenceNo()) {
-                continue;
-            }
-            // if the log segment is truncated, skip it.
-            if (skipTruncatedLogSegments &&
-                    !conf.getIgnoreTruncationStatus() &&
-                    segment.isTruncated()) {
-                continue;
-            }
-            // if the log segment is partially truncated, move the start dlsn to the min active dlsn
-            if (skipTruncatedLogSegments &&
-                    !conf.getIgnoreTruncationStatus() &&
-                    segment.isPartiallyTruncated()) {
-                if (segment.getMinActiveDLSN().compareTo(fromDLSN) > 0) {
-                    dlsnToStart = segment.getMinActiveDLSN();
-                }
-            }
-            skipTruncatedLogSegments = false;
-            if (!isAllowedToPosition(segment, dlsnToStart)) {
-                logger.error("segment {} is not allowed to position at {}", segment, dlsnToStart);
-                return;
-            }
-
-            SegmentReader reader = new SegmentReader(segment,
-                    segment.getLogSegmentSequenceNumber() == dlsnToStart.getLogSegmentSequenceNo()
-                            ? dlsnToStart.getEntryId() : 0L);
-            segmentReaders.add(reader);
-        }
-        if (segmentReaders.isEmpty()) {
-            // not initialize the background reader, until the first log segment is available to read
-            return;
-        }
-        currentSegmentReader = segmentReaders.pollFirst();
-        currentSegmentReader.openReader();
-        currentSegmentReader.startRead();
-        currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
-        unsafeReadNext(currentSegmentReader);
-        if (!segmentReaders.isEmpty()) {
-            for (SegmentReader reader : segmentReaders) {
-                reader.openReader();
-            }
-            unsafePrefetchNextSegment(true);
-        }
-        // mark the reader initialized
-        isInitialized = true;
-    }
-
-    private void unsafePrefetchNextSegment(boolean onlyInprogressLogSegment) {
-        SegmentReader nextReader = segmentReaders.peekFirst();
-        // open the next log segment if it is inprogress
-        if (null != nextReader) {
-            if (onlyInprogressLogSegment && !nextReader.getSegment().isInProgress()) {
-                return;
-            }
-            nextReader.startRead();
-            nextSegmentReader = nextReader;
-            segmentReaders.pollFirst();
-        }
-    }
-
-    /**
-     * Check if we are allowed to position the reader at <i>fromDLSN</i>.
-     *
-     * @return true if it is allowed, otherwise false.
-     */
-    private boolean isAllowedToPosition(LogSegmentMetadata segment, DLSN fromDLSN) {
-        if (segment.isTruncated()
-                && segment.getLastDLSN().compareTo(fromDLSN) >= 0
-                && !conf.getIgnoreTruncationStatus()) {
-            setLastException(new AlreadyTruncatedTransactionException(streamName
-                    + " : trying to position read ahead at " + fromDLSN
-                    + " on a segment " + segment + " that is already marked as truncated"));
-            return false;
-        }
-        if (segment.isPartiallyTruncated() &&
-                segment.getMinActiveDLSN().compareTo(fromDLSN) > 0) {
-            if (conf.getAlertWhenPositioningOnTruncated()) {
-                alertStatsLogger.raise("Trying to position reader on {} when {} is marked partially truncated",
-                    fromDLSN, segment);
-            }
-            if (!conf.getIgnoreTruncationStatus()) {
-                logger.error("{}: Trying to position reader on {} when {} is marked partially truncated",
-                        new Object[]{ streamName, fromDLSN, segment });
-
-                setLastException(new AlreadyTruncatedTransactionException(streamName
-                        + " : trying to position read ahead at " + fromDLSN
-                        + " on a segment " + segment + " that is already marked as truncated"));
-                return false;
-            }
-        }
-        return true;
-    }
-
-    void moveToNextLogSegment() {
-        orderedSubmit(new CloseableRunnable() {
-            @Override
-            void safeRun() {
-                unsafeMoveToNextLogSegment();
-            }
-        });
-    }
-
-    private void unsafeMoveToNextLogSegment() {
-        if (null != currentSegmentReader) {
-            segmentReadersToClose.add(currentSegmentReader);
-            currentSegmentReader.close().ensure(removeClosedSegmentReadersFunc);
-            logger.debug("close current segment reader {}", currentSegmentReader.getSegment());
-            currentSegmentReader = null;
-        }
-        boolean hasSegmentToRead = false;
-        if (null != nextSegmentReader) {
-            currentSegmentReader = nextSegmentReader;
-            logger.debug("move to read segment {}", currentSegmentReader.getSegment());
-            currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
-            nextSegmentReader = null;
-            // start reading
-            unsafeReadNext(currentSegmentReader);
-            unsafePrefetchNextSegment(true);
-            hasSegmentToRead = true;
-        } else {
-            unsafePrefetchNextSegment(false);
-            if (null != nextSegmentReader) {
-                currentSegmentReader = nextSegmentReader;
-                logger.debug("move to read segment {}", currentSegmentReader.getSegment());
-                currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
-                nextSegmentReader = null;
-                unsafeReadNext(currentSegmentReader);
-                unsafePrefetchNextSegment(true);
-                hasSegmentToRead = true;
-            }
-        }
-        if (!hasSegmentToRead) { // no more segment to read, wait until new log segment arrive
-            if (isCatchingUp) {
-                logger.info("ReadAhead for {} is caught up and no log segments to read now",
-                        readHandler.getFullyQualifiedName());
-                isCatchingUp = false;
-            }
-            pauseReadAheadOnNoMoreLogSegments();
-        }
-    }
-
-    void scheduleReadNext() {
-        orderedSubmit(new CloseableRunnable() {
-            @Override
-            void safeRun() {
-                if (null == currentSegmentReader) {
-                    pauseReadAheadOnNoMoreLogSegments();
-                    return;
-                }
-                unsafeReadNext(currentSegmentReader);
-            }
-        });
-    }
-
-    private void unsafeReadNext(SegmentReader reader) {
-        reader.readNext().addEventListener(this);
-    }
-
-    @Override
-    public void onSegmentsUpdated(List<LogSegmentMetadata> segments) {
-        if (!started.get()) {
-            return;
-        }
-        logger.info("segments is updated with {}", segments);
-        processLogSegments(segments);
-    }
-
-    @Override
-    public void onLogStreamDeleted() {
-        setLastException(new LogNotFoundException("Log stream "
-                + streamName + " is deleted"));
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadUtils.java
deleted file mode 100644
index f481561..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadUtils.java
+++ /dev/null
@@ -1,782 +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 com.twitter.distributedlog;
-
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.selector.FirstDLSNNotLessThanSelector;
-import com.twitter.distributedlog.selector.FirstTxIdNotLessThanSelector;
-import com.twitter.distributedlog.selector.LastRecordSelector;
-import com.twitter.distributedlog.selector.LogRecordSelector;
-import com.twitter.distributedlog.util.FutureUtils.FutureEventListenerRunnable;
-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.BoxedUnit;
-
-/**
- * Utility function for readers
- */
-public class ReadUtils {
-
-    static final Logger LOG = LoggerFactory.getLogger(ReadUtils.class);
-
-    private static final int MIN_SEARCH_BATCH_SIZE = 2;
-
-    //
-    // Read First & Last Record Functions
-    //
-
-    /**
-     * Read last record from a log segment.
-     *
-     * @param streamName
-     *          fully qualified stream name (used for logging)
-     * @param l
-     *          log segment metadata.
-     * @param fence
-     *          whether to fence the log segment.
-     * @param includeControl
-     *          whether to include control record.
-     * @param includeEndOfStream
-     *          whether to include end of stream.
-     * @param scanStartBatchSize
-     *          first num entries used for read last record scan
-     * @param scanMaxBatchSize
-     *          max num entries used for read last record scan
-     * @param numRecordsScanned
-     *          num of records scanned to get last record
-     * @param executorService
-     *          executor service used for processing entries
-     * @param entryStore
-     *          log segment entry store
-     * @return a future with last record.
-     */
-    public static Future<LogRecordWithDLSN> asyncReadLastRecord(
-            final String streamName,
-            final LogSegmentMetadata l,
-            final boolean fence,
-            final boolean includeControl,
-            final boolean includeEndOfStream,
-            final int scanStartBatchSize,
-            final int scanMaxBatchSize,
-            final AtomicInteger numRecordsScanned,
-            final ExecutorService executorService,
-            final LogSegmentEntryStore entryStore) {
-        final LogRecordSelector selector = new LastRecordSelector();
-        return asyncReadRecord(streamName, l, fence, includeControl, includeEndOfStream, scanStartBatchSize,
-                               scanMaxBatchSize, numRecordsScanned, executorService, entryStore,
-                               selector, true /* backward */, 0L);
-    }
-
-    /**
-     * Read first record from a log segment with a DLSN larger than that given.
-     *
-     * @param streamName
-     *          fully qualified stream name (used for logging)
-     * @param l
-     *          log segment metadata.
-     * @param scanStartBatchSize
-     *          first num entries used for read last record scan
-     * @param scanMaxBatchSize
-     *          max num entries used for read last record scan
-     * @param numRecordsScanned
-     *          num of records scanned to get last record
-     * @param executorService
-     *          executor service used for processing entries
-     * @param entryStore
-     *          log segment entry store
-     * @param dlsn
-     *          threshold dlsn
-     * @return a future with last record.
-     */
-    public static Future<LogRecordWithDLSN> asyncReadFirstUserRecord(
-            final String streamName,
-            final LogSegmentMetadata l,
-            final int scanStartBatchSize,
-            final int scanMaxBatchSize,
-            final AtomicInteger numRecordsScanned,
-            final ExecutorService executorService,
-            final LogSegmentEntryStore entryStore,
-            final DLSN dlsn) {
-        long startEntryId = 0L;
-        if (l.getLogSegmentSequenceNumber() == dlsn.getLogSegmentSequenceNo()) {
-            startEntryId = dlsn.getEntryId();
-        }
-        final LogRecordSelector selector = new FirstDLSNNotLessThanSelector(dlsn);
-        return asyncReadRecord(streamName, l, false, false, false, scanStartBatchSize,
-                               scanMaxBatchSize, numRecordsScanned, executorService, entryStore,
-                               selector, false /* backward */, startEntryId);
-    }
-
-    //
-    // Private methods for scanning log segments
-    //
-
-    private static class ScanContext {
-        // variables to about current scan state
-        final AtomicInteger numEntriesToScan;
-        final AtomicLong curStartEntryId;
-        final AtomicLong curEndEntryId;
-
-        // scan settings
-        final long startEntryId;
-        final long endEntryId;
-        final int scanStartBatchSize;
-        final int scanMaxBatchSize;
-        final boolean includeControl;
-        final boolean includeEndOfStream;
-        final boolean backward;
-
-        // number of records scanned
-        final AtomicInteger numRecordsScanned;
-
-        ScanContext(long startEntryId, long endEntryId,
-                    int scanStartBatchSize,
-                    int scanMaxBatchSize,
-                    boolean includeControl,
-                    boolean includeEndOfStream,
-                    boolean backward,
-                    AtomicInteger numRecordsScanned) {
-            this.startEntryId = startEntryId;
-            this.endEntryId = endEntryId;
-            this.scanStartBatchSize = scanStartBatchSize;
-            this.scanMaxBatchSize = scanMaxBatchSize;
-            this.includeControl = includeControl;
-            this.includeEndOfStream = includeEndOfStream;
-            this.backward = backward;
-            // Scan state
-            this.numEntriesToScan = new AtomicInteger(scanStartBatchSize);
-            if (backward) {
-                this.curStartEntryId = new AtomicLong(
-                        Math.max(startEntryId, (endEntryId - scanStartBatchSize + 1)));
-                this.curEndEntryId = new AtomicLong(endEntryId);
-            } else {
-                this.curStartEntryId = new AtomicLong(startEntryId);
-                this.curEndEntryId = new AtomicLong(
-                        Math.min(endEntryId, (startEntryId + scanStartBatchSize - 1)));
-            }
-            this.numRecordsScanned = numRecordsScanned;
-        }
-
-        boolean moveToNextRange() {
-            if (backward) {
-                return moveBackward();
-            } else {
-                return moveForward();
-            }
-        }
-
-        boolean moveBackward() {
-            long nextEndEntryId = curStartEntryId.get() - 1;
-            if (nextEndEntryId < startEntryId) {
-                // no entries to read again
-                return false;
-            }
-            curEndEntryId.set(nextEndEntryId);
-            // update num entries to scan
-            numEntriesToScan.set(
-                    Math.min(numEntriesToScan.get() * 2, scanMaxBatchSize));
-            // update start entry id
-            curStartEntryId.set(Math.max(startEntryId, nextEndEntryId - numEntriesToScan.get() + 1));
-            return true;
-        }
-
-        boolean moveForward() {
-            long nextStartEntryId = curEndEntryId.get() + 1;
-            if (nextStartEntryId > endEntryId) {
-                // no entries to read again
-                return false;
-            }
-            curStartEntryId.set(nextStartEntryId);
-            // update num entries to scan
-            numEntriesToScan.set(
-                    Math.min(numEntriesToScan.get() * 2, scanMaxBatchSize));
-            // update start entry id
-            curEndEntryId.set(Math.min(endEntryId, nextStartEntryId + numEntriesToScan.get() - 1));
-            return true;
-        }
-    }
-
-    private static class SingleEntryScanContext extends ScanContext {
-        SingleEntryScanContext(long entryId) {
-            super(entryId, entryId, 1, 1, true, true, false, new AtomicInteger(0));
-        }
-    }
-
-    /**
-     * Read record from a given range of log segment entries.
-     *
-     * @param streamName
-     *          fully qualified stream name (used for logging)
-     * @param reader
-     *          log segment random access reader
-     * @param executorService
-     *          executor service used for processing entries
-     * @param context
-     *          scan context
-     * @return a future with the log record.
-     */
-    private static Future<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 long startEntryId = context.curStartEntryId.get();
-        final long endEntryId = context.curEndEntryId.get();
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("{} reading entries [{} - {}] from {}.",
-                    new Object[] { streamName, startEntryId, endEntryId, metadata});
-        }
-        FutureEventListener<List<Entry.Reader>> readEntriesListener =
-            new FutureEventListener<List<Entry.Reader>>() {
-                @Override
-                public void onSuccess(final List<Entry.Reader> entries) {
-                    if (LOG.isDebugEnabled()) {
-                        LOG.debug("{} finished reading entries [{} - {}] from {}",
-                                new Object[]{ streamName, startEntryId, endEntryId, metadata});
-                    }
-                    for (Entry.Reader entry : entries) {
-                        try {
-                            visitEntryRecords(entry, context, selector);
-                        } 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);
-                            return;
-                        }
-                    }
-
-                    LogRecordWithDLSN record = selector.result();
-                    if (LOG.isDebugEnabled()) {
-                        LOG.debug("{} got record from entries [{} - {}] of {} : {}",
-                                new Object[]{streamName, startEntryId, endEntryId,
-                                        metadata, record});
-                    }
-                    promise.setValue(record);
-                }
-
-                @Override
-                public void onFailure(final Throwable cause) {
-                    promise.setException(cause);
-                }
-            };
-        reader.readEntries(startEntryId, endEntryId)
-                .addEventListener(FutureEventListenerRunnable.of(readEntriesListener, executorService));
-        return promise;
-    }
-
-    /**
-     * Process each record using LogRecordSelector.
-     *
-     * @param entry
-     *          ledger entry
-     * @param context
-     *          scan context
-     * @return log record with dlsn inside the ledger entry
-     * @throws IOException
-     */
-    private static void visitEntryRecords(
-            Entry.Reader entry,
-            ScanContext context,
-            LogRecordSelector selector) throws IOException {
-        LogRecordWithDLSN nextRecord = entry.nextRecord();
-        while (nextRecord != null) {
-            LogRecordWithDLSN record = nextRecord;
-            nextRecord = entry.nextRecord();
-            context.numRecordsScanned.incrementAndGet();
-            if (!context.includeControl && record.isControl()) {
-                continue;
-            }
-            if (!context.includeEndOfStream && record.isEndOfStream()) {
-                continue;
-            }
-            selector.process(record);
-        }
-    }
-
-    /**
-     * Scan entries for the given record.
-     *
-     * @param streamName
-     *          fully qualified stream name (used for logging)
-     * @param reader
-     *          log segment random access reader
-     * @param executorService
-     *          executor service used for processing entries
-     * @param promise
-     *          promise to return desired record.
-     * @param context
-     *          scan context
-     */
-    private static void asyncReadRecordFromEntries(
-            final String streamName,
-            final LogSegmentRandomAccessEntryReader reader,
-            final LogSegmentMetadata metadata,
-            final ExecutorService executorService,
-            final Promise<LogRecordWithDLSN> promise,
-            final ScanContext context,
-            final LogRecordSelector selector) {
-        FutureEventListener<LogRecordWithDLSN> readEntriesListener =
-            new FutureEventListener<LogRecordWithDLSN>() {
-                @Override
-                public void onSuccess(LogRecordWithDLSN value) {
-                    if (LOG.isDebugEnabled()) {
-                        LOG.debug("{} read record from [{} - {}] of {} : {}",
-                                new Object[]{streamName, context.curStartEntryId.get(), context.curEndEntryId.get(),
-                                        metadata, value});
-                    }
-                    if (null != value) {
-                        promise.setValue(value);
-                        return;
-                    }
-                    if (!context.moveToNextRange()) {
-                        // no entries to read again
-                        promise.setValue(null);
-                        return;
-                    }
-                    // scan next range
-                    asyncReadRecordFromEntries(streamName,
-                            reader,
-                            metadata,
-                            executorService,
-                            promise,
-                            context,
-                            selector);
-                }
-
-                @Override
-                public void onFailure(Throwable cause) {
-                    promise.setException(cause);
-                }
-            };
-        asyncReadRecordFromEntries(streamName, reader, metadata, executorService, context, selector)
-                .addEventListener(FutureEventListenerRunnable.of(readEntriesListener, executorService));
-    }
-
-    private static void asyncReadRecordFromLogSegment(
-            final String streamName,
-            final LogSegmentRandomAccessEntryReader reader,
-            final LogSegmentMetadata metadata,
-            final ExecutorService executorService,
-            final int scanStartBatchSize,
-            final int scanMaxBatchSize,
-            final boolean includeControl,
-            final boolean includeEndOfStream,
-            final Promise<LogRecordWithDLSN> promise,
-            final AtomicInteger numRecordsScanned,
-            final LogRecordSelector selector,
-            final boolean backward,
-            final long startEntryId) {
-        final long lastAddConfirmed = reader.getLastAddConfirmed();
-        if (lastAddConfirmed < 0) {
-            if (LOG.isDebugEnabled()) {
-                LOG.debug("Log segment {} is empty for {}.", new Object[] { metadata, streamName });
-            }
-            promise.setValue(null);
-            return;
-        }
-        final ScanContext context = new ScanContext(
-                startEntryId, lastAddConfirmed,
-                scanStartBatchSize, scanMaxBatchSize,
-                includeControl, includeEndOfStream, backward, numRecordsScanned);
-        asyncReadRecordFromEntries(streamName, reader, metadata, executorService,
-                                   promise, context, selector);
-    }
-
-    private static Future<LogRecordWithDLSN> asyncReadRecord(
-            final String streamName,
-            final LogSegmentMetadata l,
-            final boolean fence,
-            final boolean includeControl,
-            final boolean includeEndOfStream,
-            final int scanStartBatchSize,
-            final int scanMaxBatchSize,
-            final AtomicInteger numRecordsScanned,
-            final ExecutorService executorService,
-            final LogSegmentEntryStore entryStore,
-            final LogRecordSelector selector,
-            final boolean backward,
-            final long startEntryId) {
-
-        final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
-
-        FutureEventListener<LogSegmentRandomAccessEntryReader> openReaderListener =
-            new FutureEventListener<LogSegmentRandomAccessEntryReader>() {
-                @Override
-                public void onSuccess(final LogSegmentRandomAccessEntryReader reader) {
-                    if (LOG.isDebugEnabled()) {
-                        LOG.debug("{} Opened log segment {} for reading record",
-                                streamName, l);
-                    }
-                    promise.ensure(new AbstractFunction0<BoxedUnit>() {
-                        @Override
-                        public BoxedUnit apply() {
-                            reader.asyncClose();
-                            return BoxedUnit.UNIT;
-                        }
-                    });
-                    if (LOG.isDebugEnabled()) {
-                        LOG.debug("{} {} scanning {}.", new Object[]{
-                                (backward ? "backward" : "forward"), streamName, l});
-                    }
-                    asyncReadRecordFromLogSegment(
-                            streamName, reader, l, executorService,
-                            scanStartBatchSize, scanMaxBatchSize,
-                            includeControl, includeEndOfStream,
-                            promise, numRecordsScanned, selector, backward, startEntryId);
-                }
-
-                @Override
-                public void onFailure(final Throwable cause) {
-                    promise.setException(cause);
-                }
-            };
-        entryStore.openRandomAccessReader(l, fence)
-                .addEventListener(FutureEventListenerRunnable.of(openReaderListener, executorService));
-        return promise;
-    }
-
-    //
-    // Search Functions
-    //
-
-    /**
-     * Get the log record whose transaction id is not less than provided <code>transactionId</code>.
-     *
-     * <p>
-     * It uses a binary-search like algorithm to find the log record whose transaction id is not less than
-     * provided <code>transactionId</code> within a log <code>segment</code>. You could think of a log segment
-     * in terms of a sequence of records whose transaction ids are non-decreasing.
-     *
-     * - The sequence of records within a log segment is divided into N pieces.
-     * - Find the piece of records that contains a record whose transaction id is not less than provided
-     *   <code>transactionId</code>.
-     *
-     * N could be chosen based on trading off concurrency and latency.
-     * </p>
-     *
-     * @param logName
-     *          name of the log
-     * @param segment
-     *          metadata of the log segment
-     * @param transactionId
-     *          transaction id
-     * @param executorService
-     *          executor service used for processing entries
-     * @param entryStore
-     *          log segment entry store
-     * @param nWays
-     *          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(
-            final String logName,
-            final LogSegmentMetadata segment,
-            final long transactionId,
-            final ExecutorService executorService,
-            final LogSegmentEntryStore entryStore,
-            final int nWays) {
-        if (!segment.isInProgress()) {
-            if (segment.getLastTxId() < transactionId) {
-                // all log records whose transaction id is less than provided transactionId
-                // then return none
-                Optional<LogRecordWithDLSN> noneRecord = Optional.absent();
-                return Future.value(noneRecord);
-            }
-        }
-
-        final Promise<Optional<LogRecordWithDLSN>> promise =
-                new Promise<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;
-                        }
-
-                    });
-                    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);
-                        return;
-                    }
-                    // all log records whose transaction id is not less than provided transactionId
-                    if (segment.getFirstTxId() >= transactionId) {
-                        final FirstTxIdNotLessThanSelector selector =
-                                new FirstTxIdNotLessThanSelector(transactionId);
-                        asyncReadRecordFromEntries(
-                                logName,
-                                reader,
-                                segment,
-                                executorService,
-                                new SingleEntryScanContext(0L),
-                                selector
-                        ).addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
-                            @Override
-                            public void onSuccess(LogRecordWithDLSN value) {
-                                promise.setValue(Optional.of(selector.result()));
-                            }
-
-                            @Override
-                            public void onFailure(Throwable cause) {
-                                promise.setException(cause);
-                            }
-                        });
-
-                        return;
-                    }
-                    getLogRecordNotLessThanTxIdFromEntries(
-                            logName,
-                            segment,
-                            transactionId,
-                            executorService,
-                            reader,
-                            Lists.newArrayList(0L, lastEntryId),
-                            nWays,
-                            Optional.<LogRecordWithDLSN>absent(),
-                            promise);
-                }
-
-                @Override
-                public void onFailure(final Throwable cause) {
-                    promise.setException(cause);
-                }
-            };
-
-        entryStore.openRandomAccessReader(segment, false)
-                .addEventListener(FutureEventListenerRunnable.of(openReaderListener, executorService));
-        return promise;
-    }
-
-    /**
-     * Find the log record whose transaction id is not less than provided <code>transactionId</code> from
-     * entries between <code>startEntryId</code> and <code>endEntryId</code>.
-     *
-     * @param logName
-     *          name of the log
-     * @param segment
-     *          log segment
-     * @param transactionId
-     *          provided transaction id to search
-     * @param executorService
-     *          executor service
-     * @param reader
-     *          log segment random access reader
-     * @param entriesToSearch
-     *          list of entries to search
-     * @param nWays
-     *          how many entries to search in parallel
-     * @param prevFoundRecord
-     *          the log record found in previous search
-     * @param promise
-     *          promise to satisfy the result
-     */
-    private static void getLogRecordNotLessThanTxIdFromEntries(
-            final String logName,
-            final LogSegmentMetadata segment,
-            final long transactionId,
-            final ExecutorService executorService,
-            final LogSegmentRandomAccessEntryReader reader,
-            final List<Long> entriesToSearch,
-            final int nWays,
-            final Optional<LogRecordWithDLSN> prevFoundRecord,
-            final Promise<Optional<LogRecordWithDLSN>> promise) {
-        final List<Future<LogRecordWithDLSN>> searchResults =
-                Lists.newArrayListWithExpectedSize(entriesToSearch.size());
-        for (Long entryId : entriesToSearch) {
-            LogRecordSelector selector = new FirstTxIdNotLessThanSelector(transactionId);
-            Future<LogRecordWithDLSN> searchResult = asyncReadRecordFromEntries(
-                    logName,
-                    reader,
-                    segment,
-                    executorService,
-                    new SingleEntryScanContext(entryId),
-                    selector);
-            searchResults.add(searchResult);
-        }
-        FutureEventListener<List<LogRecordWithDLSN>> processSearchResultsListener =
-                new FutureEventListener<List<LogRecordWithDLSN>>() {
-                    @Override
-                    public void onSuccess(List<LogRecordWithDLSN> resultList) {
-                        processSearchResults(
-                                logName,
-                                segment,
-                                transactionId,
-                                executorService,
-                                reader,
-                                resultList,
-                                nWays,
-                                prevFoundRecord,
-                                promise);
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        promise.setException(cause);
-                    }
-                };
-        Future.collect(searchResults).addEventListener(
-                FutureEventListenerRunnable.of(processSearchResultsListener, executorService));
-    }
-
-    /**
-     * Process the search results
-     */
-    static void processSearchResults(
-            final String logName,
-            final LogSegmentMetadata segment,
-            final long transactionId,
-            final ExecutorService executorService,
-            final LogSegmentRandomAccessEntryReader reader,
-            final List<LogRecordWithDLSN> searchResults,
-            final int nWays,
-            final Optional<LogRecordWithDLSN> prevFoundRecord,
-            final Promise<Optional<LogRecordWithDLSN>> promise) {
-        int found = -1;
-        for (int i = 0; i < searchResults.size(); i++) {
-            LogRecordWithDLSN record = searchResults.get(i);
-            if (record.getTransactionId() >= transactionId) {
-                found = i;
-                break;
-            }
-        }
-        if (found == -1) { // all log records' transaction id is less than provided transaction id
-            promise.setValue(prevFoundRecord);
-            return;
-        }
-        // we found a log record
-        LogRecordWithDLSN foundRecord = searchResults.get(found);
-
-        // we found it
-        //   - it is not the first record
-        //   - it is the first record in first search entry
-        //   - its entry is adjacent to previous search entry
-        if (foundRecord.getDlsn().getSlotId() != 0L
-                || found == 0
-                || foundRecord.getDlsn().getEntryId() == (searchResults.get(found - 1).getDlsn().getEntryId() + 1)) {
-            promise.setValue(Optional.of(foundRecord));
-            return;
-        }
-
-        // otherwise, we need to search
-        List<Long> nextSearchBatch = getEntriesToSearch(
-                transactionId,
-                searchResults.get(found - 1),
-                searchResults.get(found),
-                nWays);
-        if (nextSearchBatch.isEmpty()) {
-            promise.setValue(prevFoundRecord);
-            return;
-        }
-        getLogRecordNotLessThanTxIdFromEntries(
-                logName,
-                segment,
-                transactionId,
-                executorService,
-                reader,
-                nextSearchBatch,
-                nWays,
-                Optional.of(foundRecord),
-                promise);
-    }
-
-    /**
-     * Get the entries to search provided <code>transactionId</code> between
-     * <code>firstRecord</code> and <code>lastRecord</code>. <code>firstRecord</code>
-     * and <code>lastRecord</code> are already searched, which the transaction id
-     * of <code>firstRecord</code> is less than <code>transactionId</code> and the
-     * transaction id of <code>lastRecord</code> is not less than <code>transactionId</code>.
-     *
-     * @param transactionId
-     *          transaction id to search
-     * @param firstRecord
-     *          log record that already searched whose transaction id is leass than <code>transactionId</code>.
-     * @param lastRecord
-     *          log record that already searched whose transaction id is not less than <code>transactionId</code>.
-     * @param nWays
-     *          N-ways to search
-     * @return the list of entries to search
-     */
-    static List<Long> getEntriesToSearch(
-            long transactionId,
-            LogRecordWithDLSN firstRecord,
-            LogRecordWithDLSN lastRecord,
-            int nWays) {
-        long txnDiff = lastRecord.getTransactionId() - firstRecord.getTransactionId();
-        if (txnDiff > 0) {
-            if (lastRecord.getTransactionId() == transactionId) {
-                List<Long> entries = getEntriesToSearch(
-                        firstRecord.getDlsn().getEntryId() + 1,
-                        lastRecord.getDlsn().getEntryId() - 2,
-                        Math.max(MIN_SEARCH_BATCH_SIZE, nWays - 1));
-                entries.add(lastRecord.getDlsn().getEntryId() - 1);
-                return entries;
-            } else {
-                // TODO: improve it by estimating transaction ids.
-                return getEntriesToSearch(
-                        firstRecord.getDlsn().getEntryId() + 1,
-                        lastRecord.getDlsn().getEntryId() - 1,
-                        nWays);
-            }
-        } else {
-            // unexpected condition
-            return Lists.newArrayList();
-        }
-    }
-
-    static List<Long> getEntriesToSearch(
-            long startEntryId,
-            long endEntryId,
-            int nWays) {
-        if (startEntryId > endEntryId) {
-            return Lists.newArrayList();
-        }
-        long numEntries = endEntryId - startEntryId + 1;
-        long step = Math.max(1L, numEntries / nWays);
-        List<Long> entryList = Lists.newArrayListWithExpectedSize(nWays);
-        for (long i = startEntryId, j = nWays - 1; i <= endEntryId && j > 0; i += step, j--) {
-            entryList.add(i);
-        }
-        if (entryList.get(entryList.size() - 1) < endEntryId) {
-            entryList.add(endEntryId);
-        }
-        return entryList;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/WriteLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/WriteLimiter.java
deleted file mode 100644
index 0b24c1a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/WriteLimiter.java
+++ /dev/null
@@ -1,62 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-import com.twitter.distributedlog.util.PermitLimiter;
-
-public class WriteLimiter {
-
-    String streamName;
-    final PermitLimiter streamLimiter;
-    final PermitLimiter globalLimiter;
-
-    public WriteLimiter(String streamName, PermitLimiter streamLimiter, PermitLimiter globalLimiter) {
-        this.streamName = streamName;
-        this.streamLimiter = streamLimiter;
-        this.globalLimiter = globalLimiter;
-    }
-
-    public void acquire() throws OverCapacityException {
-        if (!streamLimiter.acquire()) {
-            throw new OverCapacityException(String.format("Stream write capacity exceeded for stream %s", streamName));
-        }
-        try {
-            if (!globalLimiter.acquire()) {
-                throw new OverCapacityException("Global write capacity exceeded");
-            }
-        } catch (OverCapacityException ex) {
-            streamLimiter.release(1);
-            throw ex;
-        }
-    }
-
-    public void release() {
-        release(1);
-    }
-
-    public void release(int permits) {
-        streamLimiter.release(permits);
-        globalLimiter.release(permits);
-    }
-
-    public void close() {
-        streamLimiter.close();
-        globalLimiter.close();
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClient.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClient.java
deleted file mode 100644
index 74cd6cf..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClient.java
+++ /dev/null
@@ -1,402 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.zk.ZKWatcherManager;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.Watcher.Event.EventType;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.data.ACL;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * ZooKeeper Client wrapper over {@link org.apache.bookkeeper.zookeeper.ZooKeeperClient}.
- * It handles retries on session expires and provides a watcher manager {@link ZKWatcherManager}.
- *
- * <h3>Metrics</h3>
- * <ul>
- * <li> zookeeper operation stats are exposed under scope <code>zk</code> by
- * {@link org.apache.bookkeeper.zookeeper.ZooKeeperClient}
- * <li> stats on zookeeper watched events are exposed under scope <code>watcher</code> by
- * {@link org.apache.bookkeeper.zookeeper.ZooKeeperWatcherBase}
- * <li> stats about {@link ZKWatcherManager} are exposed under scope <code>watcher_manager</code>
- * </ul>
- */
-public class ZooKeeperClient {
-
-    public static interface Credentials {
-
-        Credentials NONE = new Credentials() {
-            @Override
-            public void authenticate(ZooKeeper zooKeeper) {
-                // noop
-            }
-        };
-
-        void authenticate(ZooKeeper zooKeeper);
-    }
-
-    public static class DigestCredentials implements Credentials {
-
-        String username;
-        String password;
-
-        public DigestCredentials(String username, String password) {
-            this.username = username;
-            this.password = password;
-        }
-
-        @Override
-        public void authenticate(ZooKeeper zooKeeper) {
-            zooKeeper.addAuthInfo("digest", String.format("%s:%s", username, password).getBytes(UTF_8));
-        }
-    }
-
-    public interface ZooKeeperSessionExpireNotifier {
-        void notifySessionExpired();
-    }
-
-    /**
-     * Indicates an error connecting to a zookeeper cluster.
-     */
-    public static class ZooKeeperConnectionException extends IOException {
-        private static final long serialVersionUID = 6682391687004819361L;
-
-        public ZooKeeperConnectionException(String message) {
-            super(message);
-        }
-
-        public ZooKeeperConnectionException(String message, Throwable cause) {
-            super(message, cause);
-        }
-    }
-
-    private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperClient.class.getName());
-
-    private final String name;
-    private final int sessionTimeoutMs;
-    private final int defaultConnectionTimeoutMs;
-    private final String zooKeeperServers;
-    // GuardedBy "this", but still volatile for tests, where we want to be able to see writes
-    // made from within long synchronized blocks.
-    private volatile ZooKeeper zooKeeper = null;
-    private final RetryPolicy retryPolicy;
-    private final StatsLogger statsLogger;
-    private final int retryThreadCount;
-    private final double requestRateLimit;
-    private final Credentials credentials;
-    private volatile boolean authenticated = false;
-    private Stopwatch disconnectedStopwatch = null;
-
-    private boolean closed = false;
-
-    final Set<Watcher> watchers = new CopyOnWriteArraySet<Watcher>();
-
-    // watcher manager to manage watchers
-    private final ZKWatcherManager watcherManager;
-
-    /**
-     * Creates an unconnected client that will lazily attempt to connect on the first call to
-     * {@link #get}.  All successful connections will be authenticated with the given
-     * {@code credentials}.
-     *
-     * @param sessionTimeoutMs
-     *          ZK session timeout in milliseconds
-     * @param connectionTimeoutMs
-     *          ZK connection timeout in milliseconds
-     * @param zooKeeperServers
-     *          the set of servers forming the ZK cluster
-     */
-    ZooKeeperClient(int sessionTimeoutMs, int connectionTimeoutMs, String zooKeeperServers) {
-        this("default", sessionTimeoutMs, connectionTimeoutMs, zooKeeperServers, null, NullStatsLogger.INSTANCE, 1, 0,
-             Credentials.NONE);
-    }
-
-    ZooKeeperClient(String name,
-                    int sessionTimeoutMs,
-                    int connectionTimeoutMs,
-                    String zooKeeperServers,
-                    RetryPolicy retryPolicy,
-                    StatsLogger statsLogger,
-                    int retryThreadCount,
-                    double requestRateLimit,
-                    Credentials credentials) {
-        this.name = name;
-        this.sessionTimeoutMs = sessionTimeoutMs;
-        this.zooKeeperServers = zooKeeperServers;
-        this.defaultConnectionTimeoutMs = connectionTimeoutMs;
-        this.retryPolicy = retryPolicy;
-        this.statsLogger = statsLogger;
-        this.retryThreadCount = retryThreadCount;
-        this.requestRateLimit = requestRateLimit;
-        this.credentials = credentials;
-        this.watcherManager = ZKWatcherManager.newBuilder()
-                .name(name)
-                .zkc(this)
-                .statsLogger(statsLogger.scope("watcher_manager"))
-                .build();
-    }
-
-    public List<ACL> getDefaultACL() {
-        if (Credentials.NONE == credentials) {
-            return ZooDefs.Ids.OPEN_ACL_UNSAFE;
-        } else {
-            return DistributedLogConstants.EVERYONE_READ_CREATOR_ALL;
-        }
-    }
-
-    public ZKWatcherManager getWatcherManager() {
-        return watcherManager;
-    }
-
-    /**
-     * Returns the current active ZK connection or establishes a new one if none has yet been
-     * established or a previous connection was disconnected or had its session time out.
-     *
-     * @return a connected ZooKeeper client
-     * @throws ZooKeeperConnectionException if there was a problem connecting to the ZK cluster
-     * @throws InterruptedException if interrupted while waiting for a connection to be established
-     * @throws TimeoutException if a connection could not be established within the configured
-     * session timeout
-     */
-    public synchronized ZooKeeper get()
-        throws ZooKeeperConnectionException, InterruptedException {
-
-        try {
-            FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss);
-        } catch (IOException ioe) {
-            throw new ZooKeeperConnectionException("Client " + name + " failed on establishing zookeeper connection", ioe);
-        }
-
-        // This indicates that the client was explictly closed
-        if (closed) {
-            throw new ZooKeeperConnectionException("Client " + name + " has already been closed");
-        }
-
-        // the underneath zookeeper is retryable zookeeper
-        if (zooKeeper != null && retryPolicy != null) {
-            if (zooKeeper.getState().equals(ZooKeeper.States.CONNECTED)) {
-                // the zookeeper client is connected
-                disconnectedStopwatch = null;
-            } else {
-                if (disconnectedStopwatch == null) {
-                    disconnectedStopwatch = Stopwatch.createStarted();
-                } else {
-                    long disconnectedMs = disconnectedStopwatch.elapsed(TimeUnit.MILLISECONDS);
-                    if (disconnectedMs > defaultConnectionTimeoutMs) {
-                        closeInternal();
-                        authenticated = false;
-                    }
-                }
-            }
-        }
-
-        if (zooKeeper == null) {
-            zooKeeper = buildZooKeeper();
-            disconnectedStopwatch = null;
-        }
-
-        // In case authenticate throws an exception, the caller can try to recover the client by
-        // calling get again.
-        if (!authenticated) {
-            credentials.authenticate(zooKeeper);
-            authenticated = true;
-        }
-
-        return zooKeeper;
-    }
-
-    private ZooKeeper buildZooKeeper()
-        throws ZooKeeperConnectionException, InterruptedException {
-        Watcher watcher = new Watcher() {
-            @Override
-            public void process(WatchedEvent event) {
-                switch (event.getType()) {
-                    case None:
-                        switch (event.getState()) {
-                            case Expired:
-                                if (null == retryPolicy) {
-                                    LOG.info("ZooKeeper {}' session expired. Event: {}", name, event);
-                                    closeInternal();
-                                }
-                                authenticated = false;
-                                break;
-                            case Disconnected:
-                                if (null == retryPolicy) {
-                                    LOG.info("ZooKeeper {} is disconnected from zookeeper now," +
-                                            " but it is OK unless we received EXPIRED event.", name);
-                                }
-                                // Mark as not authenticated if expired or disconnected. In both cases
-                                // we lose any attached auth info. Relying on Expired/Disconnected is
-                                // sufficient since all Expired/Disconnected events are processed before
-                                // all SyncConnected events, and the underlying member is not updated until
-                                // SyncConnected is received.
-                                authenticated = false;
-                                break;
-                            default:
-                                break;
-                        }
-                }
-
-                try {
-                    for (Watcher watcher : watchers) {
-                        try {
-                            watcher.process(event);
-                        } catch (Throwable t) {
-                            LOG.warn("Encountered unexpected exception from watcher {} : ", watcher, t);
-                        }
-                    }
-                } catch (Throwable t) {
-                    LOG.warn("Encountered unexpected exception when firing watched event {} : ", event, t);
-                }
-            }
-        };
-
-        Set<Watcher> watchers = new HashSet<Watcher>();
-        watchers.add(watcher);
-
-        ZooKeeper zk;
-        try {
-            RetryPolicy opRetryPolicy = null == retryPolicy ?
-                    new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0) : retryPolicy;
-            RetryPolicy connectRetryPolicy = null == retryPolicy ?
-                    new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0) :
-                    new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, Integer.MAX_VALUE);
-            zk = org.apache.bookkeeper.zookeeper.ZooKeeperClient.newBuilder()
-                    .connectString(zooKeeperServers)
-                    .sessionTimeoutMs(sessionTimeoutMs)
-                    .watchers(watchers)
-                    .operationRetryPolicy(opRetryPolicy)
-                    .connectRetryPolicy(connectRetryPolicy)
-                    .statsLogger(statsLogger)
-                    .retryThreadCount(retryThreadCount)
-                    .requestRateLimit(requestRateLimit)
-                    .build();
-        } catch (KeeperException e) {
-            throw new ZooKeeperConnectionException("Problem connecting to servers: " + zooKeeperServers, e);
-        } catch (IOException e) {
-            throw new ZooKeeperConnectionException("Problem connecting to servers: " + zooKeeperServers, e);
-        }
-        return zk;
-    }
-
-    /**
-     * Clients that need to re-establish state after session expiration can register an
-     * {@code onExpired} command to execute.
-     *
-     * @param onExpired the {@code Command} to register
-     * @return the new {@link Watcher} which can later be passed to {@link #unregister} for
-     *         removal.
-     */
-    public Watcher registerExpirationHandler(final ZooKeeperSessionExpireNotifier onExpired) {
-        Watcher watcher = new Watcher() {
-            @Override
-            public void process(WatchedEvent event) {
-                if (event.getType() == EventType.None && event.getState() == KeeperState.Expired) {
-                    try {
-                        onExpired.notifySessionExpired();
-                    } catch (Exception exc) {
-                        // do nothing
-                    }
-                }
-            }
-        };
-        register(watcher);
-        return watcher;
-    }
-
-    /**
-     * Clients that need to register a top-level {@code Watcher} should do so using this method.  The
-     * registered {@code watcher} will remain registered across re-connects and session expiration
-     * events.
-     *
-     * @param watcher the {@code Watcher to register}
-     */
-    public void register(Watcher watcher) {
-        if (null != watcher) {
-            watchers.add(watcher);
-        }
-    }
-
-    /**
-     * Clients can attempt to unregister a top-level {@code Watcher} that has previously been
-     * registered.
-     *
-     * @param watcher the {@code Watcher} to unregister as a top-level, persistent watch
-     * @return whether the given {@code Watcher} was found and removed from the active set
-     */
-    public boolean unregister(Watcher watcher) {
-        return null != watcher && watchers.remove(watcher);
-    }
-
-    /**
-     * Closes the current connection if any expiring the current ZooKeeper session.  Any subsequent
-     * calls to this method will no-op until the next successful {@link #get}.
-     */
-    public synchronized void closeInternal() {
-        if (zooKeeper != null) {
-            try {
-                LOG.info("Closing zookeeper client {}.", name);
-                zooKeeper.close();
-                LOG.info("Closed zookeeper client {}.", name);
-            } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                LOG.warn("Interrupted trying to close zooKeeper {} : ", name, e);
-            } finally {
-                zooKeeper = null;
-            }
-        }
-    }
-
-    /**
-     * Closes the the underlying zookeeper instance.
-     * Subsequent attempts to {@link #get} will fail
-     */
-    public synchronized void close() {
-        if (closed) {
-            return;
-        }
-        LOG.info("Close zookeeper client {}.", name);
-        closeInternal();
-        // unregister gauges to prevent GC spiral
-        this.watcherManager.unregisterGauges();
-        closed = true;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java
deleted file mode 100644
index 15f1805..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java
+++ /dev/null
@@ -1,233 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Preconditions;
-import com.twitter.distributedlog.ZooKeeperClient.Credentials;
-import com.twitter.distributedlog.ZooKeeperClient.DigestCredentials;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-
-/**
- * Builder to build zookeeper client.
- */
-public class ZooKeeperClientBuilder {
-
-    static final Logger LOG = LoggerFactory.getLogger(ZooKeeperClientBuilder.class);
-
-    /**
-     * Create a zookeeper client builder to build zookeeper clients.
-     *
-     * @return zookeeper client builder.
-     */
-    public static ZooKeeperClientBuilder newBuilder() {
-        return new ZooKeeperClientBuilder();
-    }
-
-    // name
-    private String name = "default";
-    // sessionTimeoutMs
-    private int sessionTimeoutMs = -1;
-    // conectionTimeoutMs
-    private int conectionTimeoutMs = -1;
-    // zkServers
-    private String zkServers = null;
-    // retry policy
-    private RetryPolicy retryPolicy = null;
-    // stats logger
-    private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
-    // retry executor thread count
-    private int retryThreadCount = 1;
-    // zookeeper access requestRateLimit limit
-    private double requestRateLimit = 0;
-    // Did call the zkAclId setter on the builder, used to ensure the setter is set.
-    private boolean zkAclIdSet = false;
-    private String zkAclId;
-
-    // Cached ZooKeeper Client
-    private ZooKeeperClient cachedClient = null;
-
-    private ZooKeeperClientBuilder() {}
-
-    /**
-     * Set zookeeper client name
-     *
-     * @param name zookeeper client name
-     * @return zookeeper client builder
-     */
-    public synchronized ZooKeeperClientBuilder name(String name) {
-        this.name = name;
-        return this;
-    }
-
-    /**
-     * Set zookeeper session timeout in milliseconds.
-     *
-     * @param sessionTimeoutMs
-     *          session timeout in milliseconds.
-     * @return zookeeper client builder.
-     */
-    public synchronized ZooKeeperClientBuilder sessionTimeoutMs(int sessionTimeoutMs) {
-        this.sessionTimeoutMs = sessionTimeoutMs;
-        if (this.conectionTimeoutMs <= 0) {
-            this.conectionTimeoutMs = 2 * sessionTimeoutMs;
-        }
-        return this;
-    }
-
-    public synchronized ZooKeeperClientBuilder retryThreadCount(int retryThreadCount) {
-        this.retryThreadCount = retryThreadCount;
-        return this;
-    }
-
-    public synchronized ZooKeeperClientBuilder requestRateLimit(double requestRateLimit) {
-        this.requestRateLimit = requestRateLimit;
-        return this;
-    }
-
-    /**
-     * Set zookeeper connection timeout in milliseconds
-     *
-     * @param connectionTimeoutMs
-     *          connection timeout ms.
-     * @return builder
-     */
-    public synchronized ZooKeeperClientBuilder connectionTimeoutMs(int connectionTimeoutMs) {
-        this.conectionTimeoutMs = connectionTimeoutMs;
-        return this;
-    }
-
-    /**
-     * Set ZooKeeper Connect String.
-     *
-     * @param zkServers
-     *          zookeeper servers to connect.
-     * @return builder
-     */
-    public synchronized ZooKeeperClientBuilder zkServers(String zkServers) {
-        this.zkServers = zkServers;
-        return this;
-    }
-
-    /**
-     * Set DistributedLog URI.
-     *
-     * @param uri
-     *          distributedlog uri.
-     * @return builder.
-     */
-    public synchronized ZooKeeperClientBuilder uri(URI uri) {
-        this.zkServers = BKNamespaceDriver.getZKServersFromDLUri(uri);
-        return this;
-    }
-
-    /**
-     * Build zookeeper client using existing <i>zkc</i> client.
-     *
-     * @param zkc
-     *          zookeeper client.
-     * @return builder
-     */
-    public synchronized ZooKeeperClientBuilder zkc(ZooKeeperClient zkc) {
-        this.cachedClient = zkc;
-        return this;
-    }
-
-    /**
-     * Build zookeeper client with given retry policy <i>retryPolicy</i>.
-     *
-     * @param retryPolicy
-     *          retry policy
-     * @return builder
-     */
-    public synchronized ZooKeeperClientBuilder retryPolicy(RetryPolicy retryPolicy) {
-        this.retryPolicy = retryPolicy;
-        return this;
-    }
-
-    /**
-     * Build zookeeper client with given stats logger <i>statsLogger</i>.
-     *
-     * @param statsLogger
-     *          stats logger to expose zookeeper stats
-     * @return builder
-     */
-    public synchronized ZooKeeperClientBuilder statsLogger(StatsLogger statsLogger) {
-        this.statsLogger = statsLogger;
-        return this;
-    }
-
-    /**
-     * * Build zookeeper client with given zk acl digest id <i>zkAclId</i>.
-     */
-    public synchronized ZooKeeperClientBuilder zkAclId(String zkAclId) {
-        this.zkAclIdSet = true;
-        this.zkAclId = zkAclId;
-        return this;
-    }
-
-    private void validateParameters() {
-        Preconditions.checkNotNull(zkServers, "No zk servers provided.");
-        Preconditions.checkArgument(conectionTimeoutMs > 0,
-                "Invalid connection timeout : %d", conectionTimeoutMs);
-        Preconditions.checkArgument(sessionTimeoutMs > 0,
-                "Invalid session timeout : %d", sessionTimeoutMs);
-        Preconditions.checkNotNull(statsLogger, "No stats logger provided.");
-        Preconditions.checkArgument(zkAclIdSet, "Zookeeper acl id not set.");
-    }
-
-    /**
-     * Build a zookeeper client.
-     *
-     * @return zookeeper client.
-     */
-    public synchronized ZooKeeperClient build() {
-        if (null == cachedClient) {
-            cachedClient = buildClient();
-        }
-        return cachedClient;
-    }
-
-    private ZooKeeperClient buildClient() {
-        validateParameters();
-
-        Credentials credentials = Credentials.NONE;
-        if (null != zkAclId) {
-            credentials = new DigestCredentials(zkAclId, zkAclId);
-        }
-
-        return new ZooKeeperClient(
-                name,
-                sessionTimeoutMs,
-                conectionTimeoutMs,
-                zkServers,
-                retryPolicy,
-                statsLogger,
-                retryThreadCount,
-                requestRateLimit,
-                credentials
-        );
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/AccessControlManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/AccessControlManager.java
deleted file mode 100644
index 5fcc87e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/AccessControlManager.java
+++ /dev/null
@@ -1,74 +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 com.twitter.distributedlog.acl;
-
-/**
- * Access Control on stream operations
- */
-public interface AccessControlManager {
-
-    /**
-     * Whether allowing writing to a stream.
-     *
-     * @param stream
-     *          Stream to write
-     * @return true if allowing writing to the given stream, otherwise false.
-     */
-    boolean allowWrite(String stream);
-
-    /**
-     * Whether allowing truncating a given stream.
-     *
-     * @param stream
-     *          Stream to truncate
-     * @return true if allowing truncating a given stream.
-     */
-    boolean allowTruncate(String stream);
-
-    /**
-     * Whether allowing deleting a given stream.
-     *
-     * @param stream
-     *          Stream to delete
-     * @return true if allowing deleting a given stream.
-     */
-    boolean allowDelete(String stream);
-
-    /**
-     * Whether allowing proxies to acquire a given stream.
-     *
-     * @param stream
-     *          stream to acquire
-     * @return true if allowing proxies to acquire the given stream.
-     */
-    boolean allowAcquire(String stream);
-
-    /**
-     * Whether allowing proxies to release ownership for a given stream.
-     *
-     * @param stream
-     *          stream to release
-     * @return true if allowing proxies to release a given stream.
-     */
-    boolean allowRelease(String stream);
-
-    /**
-     * Close the access control manager.
-     */
-    void close();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/DefaultAccessControlManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/DefaultAccessControlManager.java
deleted file mode 100644
index e757595..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/DefaultAccessControlManager.java
+++ /dev/null
@@ -1,55 +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 com.twitter.distributedlog.acl;
-
-public class DefaultAccessControlManager implements AccessControlManager {
-
-    public static final DefaultAccessControlManager INSTANCE = new DefaultAccessControlManager();
-
-    private DefaultAccessControlManager() {
-    }
-
-    @Override
-    public boolean allowWrite(String stream) {
-        return true;
-    }
-
-    @Override
-    public boolean allowTruncate(String stream) {
-        return true;
-    }
-
-    @Override
-    public boolean allowDelete(String stream) {
-        return true;
-    }
-
-    @Override
-    public boolean allowAcquire(String stream) {
-        return true;
-    }
-
-    @Override
-    public boolean allowRelease(String stream) {
-        return true;
-    }
-
-    @Override
-    public void close() {
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/package-info.java
deleted file mode 100644
index 65109fc..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Access Control for distributedlog streams.
- */
-package com.twitter.distributedlog.acl;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/DistributedLogAdmin.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/DistributedLogAdmin.java
deleted file mode 100644
index 0512907..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/DistributedLogAdmin.java
+++ /dev/null
@@ -1,921 +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 com.twitter.distributedlog.admin;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.ReadUtils;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.impl.acl.ZKAccessControl;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.impl.federated.FederatedZKLogMetadataStore;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.DLMetadata;
-import com.twitter.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
-import com.twitter.distributedlog.metadata.MetadataUpdater;
-import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.NamespaceDriver;
-import com.twitter.distributedlog.thrift.AccessControlEntry;
-import com.twitter.distributedlog.tools.DistributedLogTool;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.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.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Admin Tool for DistributedLog.
- */
-public class DistributedLogAdmin extends DistributedLogTool {
-
-    static final Logger LOG = LoggerFactory.getLogger(DistributedLogAdmin.class);
-
-    /**
-     * Fix inprogress segment with lower ledger sequence number.
-     *
-     * @param namespace
-     *          dl namespace
-     * @param metadataUpdater
-     *          metadata updater.
-     * @param streamName
-     *          stream name.
-     * @param verbose
-     *          print verbose messages.
-     * @param interactive
-     *          is confirmation needed before executing actual action.
-     * @throws IOException
-     */
-    public static void fixInprogressSegmentWithLowerSequenceNumber(final DistributedLogNamespace namespace,
-                                                                   final MetadataUpdater metadataUpdater,
-                                                                   final String streamName,
-                                                                   final boolean verbose,
-                                                                   final boolean interactive) throws IOException {
-        DistributedLogManager dlm = namespace.openLog(streamName);
-        try {
-            List<LogSegmentMetadata> segments = dlm.getLogSegments();
-            if (verbose) {
-                System.out.println("LogSegments for " + streamName + " : ");
-                for (LogSegmentMetadata segment : segments) {
-                    System.out.println(segment.getLogSegmentSequenceNumber() + "\t: " + segment);
-                }
-            }
-            LOG.info("Get log segments for {} : {}", streamName, segments);
-            // validate log segments
-            long maxCompletedLogSegmentSequenceNumber = -1L;
-            LogSegmentMetadata inprogressSegment = null;
-            for (LogSegmentMetadata segment : segments) {
-                if (!segment.isInProgress()) {
-                    maxCompletedLogSegmentSequenceNumber = Math.max(maxCompletedLogSegmentSequenceNumber, segment.getLogSegmentSequenceNumber());
-                } else {
-                    // we already found an inprogress segment
-                    if (null != inprogressSegment) {
-                        throw new DLIllegalStateException("Multiple inprogress segments found for stream " + streamName + " : " + segments);
-                    }
-                    inprogressSegment = segment;
-                }
-            }
-            if (null == inprogressSegment || inprogressSegment.getLogSegmentSequenceNumber() > maxCompletedLogSegmentSequenceNumber) {
-                // nothing to fix
-                return;
-            }
-            final long newLogSegmentSequenceNumber = maxCompletedLogSegmentSequenceNumber + 1;
-            if (interactive && !IOUtils.confirmPrompt("Confirm to fix (Y/N), Ctrl+C to break : ")) {
-                return;
-            }
-            final LogSegmentMetadata newSegment =
-                    FutureUtils.result(metadataUpdater.changeSequenceNumber(inprogressSegment, newLogSegmentSequenceNumber));
-            LOG.info("Fixed {} : {} -> {} ",
-                     new Object[] { streamName, inprogressSegment, newSegment });
-            if (verbose) {
-                System.out.println("Fixed " + streamName + " : " + inprogressSegment.getZNodeName()
-                                   + " -> " + newSegment.getZNodeName());
-                System.out.println("\t old: " + inprogressSegment);
-                System.out.println("\t new: " + newSegment);
-                System.out.println();
-            }
-        } finally {
-            dlm.close();
-        }
-    }
-
-    private static class LogSegmentCandidate {
-        final LogSegmentMetadata metadata;
-        final LogRecordWithDLSN lastRecord;
-
-        LogSegmentCandidate(LogSegmentMetadata metadata, LogRecordWithDLSN lastRecord) {
-            this.metadata = metadata;
-            this.lastRecord = lastRecord;
-        }
-
-        @Override
-        public String toString() {
-            return "LogSegmentCandidate[ metadata = " + metadata + ", last record = " + lastRecord + " ]";
-        }
-
-    }
-
-    private static final Comparator<LogSegmentCandidate> LOG_SEGMENT_CANDIDATE_COMPARATOR =
-            new Comparator<LogSegmentCandidate>() {
-                @Override
-                public int compare(LogSegmentCandidate o1, LogSegmentCandidate o2) {
-                    return LogSegmentMetadata.COMPARATOR.compare(o1.metadata, o2.metadata);
-                }
-            };
-
-    private static class StreamCandidate {
-
-        final String streamName;
-        final SortedSet<LogSegmentCandidate> segmentCandidates =
-                new TreeSet<LogSegmentCandidate>(LOG_SEGMENT_CANDIDATE_COMPARATOR);
-
-        StreamCandidate(String streamName) {
-            this.streamName = streamName;
-        }
-
-        synchronized void addLogSegmentCandidate(LogSegmentCandidate segmentCandidate) {
-            segmentCandidates.add(segmentCandidate);
-        }
-
-        @Override
-        public String toString() {
-            return "StreamCandidate[ name = " + streamName + ", segments = " + segmentCandidates + " ]";
-        }
-    }
-
-    public static void checkAndRepairDLNamespace(final URI uri,
-                                                 final DistributedLogNamespace namespace,
-                                                 final MetadataUpdater metadataUpdater,
-                                                 final OrderedScheduler scheduler,
-                                                 final boolean verbose,
-                                                 final boolean interactive) throws IOException {
-        checkAndRepairDLNamespace(uri, namespace, metadataUpdater, scheduler, verbose, interactive, 1);
-    }
-
-    public static void checkAndRepairDLNamespace(final URI uri,
-                                                 final DistributedLogNamespace namespace,
-                                                 final MetadataUpdater metadataUpdater,
-                                                 final OrderedScheduler scheduler,
-                                                 final boolean verbose,
-                                                 final boolean interactive,
-                                                 final int concurrency) throws IOException {
-        Preconditions.checkArgument(concurrency > 0, "Invalid concurrency " + concurrency + " found.");
-        // 0. getting streams under a given uri.
-        Iterator<String> streamsIter = namespace.getLogs();
-        List<String> streams = Lists.newArrayList();
-        while (streamsIter.hasNext()) {
-            streams.add(streamsIter.next());
-        }
-        if (verbose) {
-            System.out.println("- 0. checking streams under " + uri);
-        }
-        if (streams.size() == 0) {
-            System.out.println("+ 0. nothing to check. quit.");
-            return;
-        }
-        Map<String, StreamCandidate> streamCandidates =
-                checkStreams(namespace, streams, scheduler, concurrency);
-        if (verbose) {
-            System.out.println("+ 0. " + streamCandidates.size() + " corrupted streams found.");
-        }
-        if (interactive && !IOUtils.confirmPrompt("Do you want to fix all " + streamCandidates.size() + " corrupted streams (Y/N) : ")) {
-            return;
-        }
-        if (verbose) {
-            System.out.println("- 1. repairing " + streamCandidates.size() + " corrupted streams.");
-        }
-        for (StreamCandidate candidate : streamCandidates.values()) {
-            if (!repairStream(metadataUpdater, candidate, verbose, interactive)) {
-                if (verbose) {
-                    System.out.println("* 1. aborted repairing corrupted streams.");
-                }
-                return;
-            }
-        }
-        if (verbose) {
-            System.out.println("+ 1. repaired " + streamCandidates.size() + " corrupted streams.");
-        }
-    }
-
-    private static Map<String, StreamCandidate> checkStreams(
-            final DistributedLogNamespace namespace,
-            final Collection<String> streams,
-            final OrderedScheduler scheduler,
-            final int concurrency) throws IOException {
-        final LinkedBlockingQueue<String> streamQueue =
-                new LinkedBlockingQueue<String>();
-        streamQueue.addAll(streams);
-        final Map<String, StreamCandidate> candidateMap =
-                new ConcurrentSkipListMap<String, StreamCandidate>();
-        final AtomicInteger numPendingStreams = new AtomicInteger(streams.size());
-        final CountDownLatch doneLatch = new CountDownLatch(1);
-        Runnable checkRunnable = new Runnable() {
-            @Override
-            public void run() {
-                while (!streamQueue.isEmpty()) {
-                    String stream;
-                    try {
-                        stream = streamQueue.take();
-                    } catch (InterruptedException e) {
-                        Thread.currentThread().interrupt();
-                        break;
-                    }
-                    StreamCandidate candidate;
-                    try {
-                        LOG.info("Checking stream {}.", stream);
-                        candidate = checkStream(namespace, stream, scheduler);
-                        LOG.info("Checked stream {} - {}.", stream, candidate);
-                    } catch (IOException e) {
-                        LOG.error("Error on checking stream {} : ", stream, e);
-                        doneLatch.countDown();
-                        break;
-                    }
-                    if (null != candidate) {
-                        candidateMap.put(stream, candidate);
-                    }
-                    if (numPendingStreams.decrementAndGet() == 0) {
-                        doneLatch.countDown();
-                    }
-                }
-            }
-        };
-        Thread[] threads = new Thread[concurrency];
-        for (int i = 0; i < concurrency; i++) {
-            threads[i] = new Thread(checkRunnable, "check-thread-" + i);
-            threads[i].start();
-        }
-        try {
-            doneLatch.await();
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-        }
-        if (numPendingStreams.get() != 0) {
-            throw new IOException(numPendingStreams.get() + " streams left w/o checked");
-        }
-        for (int i = 0; i < concurrency; i++) {
-            threads[i].interrupt();
-            try {
-                threads[i].join();
-            } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-            }
-        }
-        return candidateMap;
-    }
-
-    private static StreamCandidate checkStream(
-            final DistributedLogNamespace namespace,
-            final String streamName,
-            final OrderedScheduler scheduler) throws IOException {
-        DistributedLogManager dlm = namespace.openLog(streamName);
-        try {
-            List<LogSegmentMetadata> segments = dlm.getLogSegments();
-            if (segments.isEmpty()) {
-                return null;
-            }
-            List<Future<LogSegmentCandidate>> futures =
-                    new ArrayList<Future<LogSegmentCandidate>>(segments.size());
-            for (LogSegmentMetadata segment : segments) {
-                futures.add(checkLogSegment(namespace, streamName, segment, scheduler));
-            }
-            List<LogSegmentCandidate> segmentCandidates;
-            try {
-                segmentCandidates = Await.result(Future.collect(futures));
-            } catch (Exception e) {
-                throw new IOException("Failed on checking stream " + streamName, e);
-            }
-            StreamCandidate streamCandidate = new StreamCandidate(streamName);
-            for (LogSegmentCandidate segmentCandidate: segmentCandidates) {
-                if (null != segmentCandidate) {
-                    streamCandidate.addLogSegmentCandidate(segmentCandidate);
-                }
-            }
-            if (streamCandidate.segmentCandidates.isEmpty()) {
-                return null;
-            }
-            return streamCandidate;
-        } finally {
-            dlm.close();
-        }
-    }
-
-    private static Future<LogSegmentCandidate> checkLogSegment(
-            final DistributedLogNamespace namespace,
-            final String streamName,
-            final LogSegmentMetadata metadata,
-            final OrderedScheduler scheduler) {
-        if (metadata.isInProgress()) {
-            return Future.value(null);
-        }
-
-        final LogSegmentEntryStore entryStore = namespace.getNamespaceDriver()
-                .getLogSegmentEntryStore(NamespaceDriver.Role.READER);
-        return ReadUtils.asyncReadLastRecord(
-                streamName,
-                metadata,
-                true,
-                false,
-                true,
-                4,
-                16,
-                new AtomicInteger(0),
-                scheduler,
-                entryStore
-        ).map(new Function<LogRecordWithDLSN, LogSegmentCandidate>() {
-            @Override
-            public LogSegmentCandidate apply(LogRecordWithDLSN record) {
-                if (null != record &&
-                    (record.getDlsn().compareTo(metadata.getLastDLSN()) > 0 ||
-                     record.getTransactionId() > metadata.getLastTxId() ||
-                     !metadata.isRecordPositionWithinSegmentScope(record))) {
-                    return new LogSegmentCandidate(metadata, record);
-                } else {
-                    return null;
-                }
-            }
-        });
-    }
-
-    private static boolean repairStream(MetadataUpdater metadataUpdater,
-                                        StreamCandidate streamCandidate,
-                                        boolean verbose,
-                                        boolean interactive) throws IOException {
-        if (verbose) {
-            System.out.println("Stream " + streamCandidate.streamName + " : ");
-            for (LogSegmentCandidate segmentCandidate : streamCandidate.segmentCandidates) {
-                System.out.println("  " + segmentCandidate.metadata.getLogSegmentSequenceNumber()
-                        + " : metadata = " + segmentCandidate.metadata + ", last dlsn = "
-                        + segmentCandidate.lastRecord.getDlsn());
-            }
-            System.out.println("-------------------------------------------");
-        }
-        if (interactive && !IOUtils.confirmPrompt("Do you want to fix the stream " + streamCandidate.streamName + " (Y/N) : ")) {
-            return false;
-        }
-        for (LogSegmentCandidate segmentCandidate : streamCandidate.segmentCandidates) {
-            LogSegmentMetadata newMetadata = FutureUtils.result(
-                    metadataUpdater.updateLastRecord(segmentCandidate.metadata, segmentCandidate.lastRecord));
-            if (verbose) {
-                System.out.println("  Fixed segment " + segmentCandidate.metadata.getLogSegmentSequenceNumber() + " : ");
-                System.out.println("    old metadata : " + segmentCandidate.metadata);
-                System.out.println("    new metadata : " + newMetadata);
-            }
-        }
-        if (verbose) {
-            System.out.println("-------------------------------------------");
-        }
-        return true;
-    }
-
-    //
-    // Commands
-    //
-
-    /**
-     * Unbind the bookkeeper environment for a given distributedlog uri.
-     *
-     * TODO: move unbind operation to namespace driver
-     */
-    class UnbindCommand extends OptsCommand {
-
-        Options options = new Options();
-
-        UnbindCommand() {
-            super("unbind", "unbind the bookkeeper environment bound for a given distributedlog instance.");
-            options.addOption("f", "force", false, "Force unbinding without prompt.");
-        }
-
-        @Override
-        protected Options getOptions() {
-            return options;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "unbind [options] <distributedlog uri>";
-        }
-
-        @Override
-        protected int runCmd(CommandLine cmdline) throws Exception {
-            String[] args = cmdline.getArgs();
-            if (args.length <= 0) {
-                System.err.println("No distributedlog uri specified.");
-                printUsage();
-                return -1;
-            }
-            boolean force = cmdline.hasOption("f");
-            URI uri = URI.create(args[0]);
-            // resolving the uri to see if there is another bindings in this uri.
-            ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder().uri(uri)
-                    .sessionTimeoutMs(10000).build();
-            BKDLConfig bkdlConfig;
-            try {
-                bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
-            } catch (IOException ie) {
-                bkdlConfig = null;
-            }
-            if (null == bkdlConfig) {
-                System.out.println("No bookkeeper is bound to " + uri);
-                return 0;
-            } else {
-                System.out.println("There is bookkeeper bound to " + uri + " : ");
-                System.out.println("");
-                System.out.println(bkdlConfig.toString());
-                System.out.println("");
-                if (!force && !IOUtils.confirmPrompt("Do you want to unbind " + uri + " :\n")) {
-                    return 0;
-                }
-            }
-            DLMetadata.unbind(uri);
-            System.out.println("Unbound on " + uri + ".");
-            return 0;
-        }
-    }
-
-    /**
-     * Bind Command to bind bookkeeper environment for a given distributed uri.
-     *
-     * TODO: move bind to namespace driver
-     */
-    class BindCommand extends OptsCommand {
-
-        Options options = new Options();
-
-        BindCommand() {
-            super("bind", "bind the bookkeeper environment settings for a given distributedlog instance.");
-            options.addOption("l", "bkLedgers", true, "ZooKeeper ledgers path for bookkeeper instance.");
-            options.addOption("s", "bkZkServers", true, "ZooKeeper servers used for bookkeeper for writers.");
-            options.addOption("bkzr", "bkZkServersForReader", true, "ZooKeeper servers used for bookkeeper for readers.");
-            options.addOption("dlzw", "dlZkServersForWriter", true, "ZooKeeper servers used for distributedlog for writers.");
-            options.addOption("dlzr", "dlZkServersForReader", true, "ZooKeeper servers used for distributedlog for readers.");
-            options.addOption("i", "sanityCheckTxnID", true, "Flag to sanity check highest txn id.");
-            options.addOption("r", "encodeRegionID", true, "Flag to encode region id.");
-            options.addOption("seqno", "firstLogSegmentSeqNo", true, "The first log segment sequence number to use after upgrade");
-            options.addOption("fns", "federatedNamespace", false, "Flag to turn a namespace to federated namespace");
-            options.addOption("f", "force", false, "Force binding without prompt.");
-            options.addOption("c", "creation", false, "Whether is it a creation binding.");
-            options.addOption("q", "query", false, "Query the bookkeeper bindings");
-        }
-
-        @Override
-        protected Options getOptions() {
-            return options;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "bind [options] <distributedlog uri>";
-        }
-
-        @Override
-        protected int runCmd(CommandLine cmdline) throws Exception {
-            boolean isQuery = cmdline.hasOption("q");
-            if (!isQuery && (!cmdline.hasOption("l") || !cmdline.hasOption("s"))) {
-                System.err.println("Error: Neither zkServers nor ledgersPath specified for bookkeeper environment.");
-                printUsage();
-                return -1;
-            }
-            String[] args = cmdline.getArgs();
-            if (args.length <= 0) {
-                System.err.println("No distributedlog uri specified.");
-                printUsage();
-                return -1;
-            }
-            boolean force = cmdline.hasOption("f");
-            boolean creation = cmdline.hasOption("c");
-            String bkLedgersPath = cmdline.getOptionValue("l");
-            String bkZkServersForWriter = cmdline.getOptionValue("s");
-            boolean sanityCheckTxnID =
-                    !cmdline.hasOption("i") || Boolean.parseBoolean(cmdline.getOptionValue("i"));
-            boolean encodeRegionID =
-                    cmdline.hasOption("r") && Boolean.parseBoolean(cmdline.getOptionValue("r"));
-
-            String bkZkServersForReader;
-            if (cmdline.hasOption("bkzr")) {
-                bkZkServersForReader = cmdline.getOptionValue("bkzr");
-            } else {
-                bkZkServersForReader = bkZkServersForWriter;
-            }
-
-            URI uri = URI.create(args[0]);
-
-            String dlZkServersForWriter;
-            String dlZkServersForReader;
-            if (cmdline.hasOption("dlzw")) {
-                dlZkServersForWriter = cmdline.getOptionValue("dlzw");
-            } else {
-                dlZkServersForWriter = BKNamespaceDriver.getZKServersFromDLUri(uri);
-            }
-            if (cmdline.hasOption("dlzr")) {
-                dlZkServersForReader = cmdline.getOptionValue("dlzr");
-            } else {
-                dlZkServersForReader = dlZkServersForWriter;
-            }
-
-            // resolving the uri to see if there is another bindings in this uri.
-            ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder().uri(uri).zkAclId(null)
-                    .sessionTimeoutMs(10000).build();
-            try {
-                BKDLConfig newBKDLConfig =
-                        new BKDLConfig(dlZkServersForWriter, dlZkServersForReader,
-                                       bkZkServersForWriter, bkZkServersForReader, bkLedgersPath)
-                                .setSanityCheckTxnID(sanityCheckTxnID)
-                                .setEncodeRegionID(encodeRegionID);
-
-                if (cmdline.hasOption("seqno")) {
-                    newBKDLConfig = newBKDLConfig.setFirstLogSegmentSeqNo(Long.parseLong(cmdline.getOptionValue("seqno")));
-                }
-
-                if (cmdline.hasOption("fns")) {
-                    newBKDLConfig = newBKDLConfig.setFederatedNamespace(true);
-                }
-
-                BKDLConfig bkdlConfig;
-                try {
-                    bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
-                } catch (IOException ie) {
-                    bkdlConfig = null;
-                }
-                if (null == bkdlConfig) {
-                    System.out.println("No bookkeeper is bound to " + uri);
-                } else {
-                    System.out.println("There is bookkeeper bound to " + uri + " : ");
-                    System.out.println("");
-                    System.out.println(bkdlConfig.toString());
-                    System.out.println("");
-                    if (!isQuery) {
-                        if (newBKDLConfig.equals(bkdlConfig)) {
-                            System.out.println("No bookkeeper binding needs to be updated. Quit.");
-                            return 0;
-                        } else if(!newBKDLConfig.isFederatedNamespace() && bkdlConfig.isFederatedNamespace()) {
-                            System.out.println("You can't turn a federated namespace back to non-federated.");
-                            return 0;
-                        } else {
-                            if (!force && !IOUtils.confirmPrompt("Do you want to bind " + uri
-                                        + " with new bookkeeper instance :\n" + newBKDLConfig)) {
-                                return 0;
-                            }
-                        }
-                    }
-                }
-                if (isQuery) {
-                    System.out.println("Done.");
-                    return 0;
-                }
-                DLMetadata dlMetadata = DLMetadata.create(newBKDLConfig);
-                if (creation) {
-                    try {
-                        dlMetadata.create(uri);
-                        System.out.println("Created binding on " + uri + ".");
-                    } catch (IOException ie) {
-                        System.err.println("Failed to create binding on " + uri + " : " + ie.getMessage());
-                    }
-                } else {
-                    try {
-                        dlMetadata.update(uri);
-                        System.out.println("Updated binding on " + uri + " : ");
-                        System.out.println("");
-                        System.out.println(newBKDLConfig.toString());
-                        System.out.println("");
-                    } catch (IOException ie) {
-                        System.err.println("Failed to update binding on " + uri + " : " + ie.getMessage());
-                    }
-                }
-                if (newBKDLConfig.isFederatedNamespace()) {
-                    try {
-                        FederatedZKLogMetadataStore.createFederatedNamespace(uri, zkc);
-                    } catch (KeeperException.NodeExistsException nee) {
-                        // ignore node exists exception
-                    }
-                }
-                return 0;
-            } finally {
-                zkc.close();
-            }
-        }
-    }
-
-    static class RepairSeqNoCommand extends PerDLCommand {
-
-        boolean dryrun = false;
-        boolean verbose = false;
-        final List<String> streams = new ArrayList<String>();
-
-        RepairSeqNoCommand() {
-            super("repairseqno", "Repair a stream whose inprogress log segment has lower sequence number.");
-            options.addOption("d", "dryrun", false, "Dry run without repairing");
-            options.addOption("l", "list", true, "List of streams to repair, separated by comma");
-            options.addOption("v", "verbose", false, "Print verbose messages");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            dryrun = cmdline.hasOption("d");
-            verbose = cmdline.hasOption("v");
-            force = !dryrun && cmdline.hasOption("f");
-            if (!cmdline.hasOption("l")) {
-                throw new ParseException("No streams provided to repair");
-            }
-            String streamsList = cmdline.getOptionValue("l");
-            Collections.addAll(streams, streamsList.split(","));
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            MetadataUpdater metadataUpdater = dryrun ?
-                    new DryrunLogSegmentMetadataStoreUpdater(getConf(),
-                            getLogSegmentMetadataStore()) :
-                    LogSegmentMetadataStoreUpdater.createMetadataUpdater(getConf(),
-                            getLogSegmentMetadataStore());
-            System.out.println("List of streams : ");
-            System.out.println(streams);
-            if (!IOUtils.confirmPrompt("Do you want to repair all these streams (Y/N):")) {
-                return -1;
-            }
-            for (String stream : streams) {
-                fixInprogressSegmentWithLowerSequenceNumber(getNamespace(), metadataUpdater, stream, verbose, !getForce());
-            }
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "repairseqno [options]";
-        }
-    }
-
-    static class DLCKCommand extends PerDLCommand {
-
-        boolean dryrun = false;
-        boolean verbose = false;
-        int concurrency = 1;
-
-        DLCKCommand() {
-            super("dlck", "Check and repair a distributedlog namespace");
-            options.addOption("d", "dryrun", false, "Dry run without repairing");
-            options.addOption("v", "verbose", false, "Print verbose messages");
-            options.addOption("cy", "concurrency", true, "Concurrency on checking streams");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            dryrun = cmdline.hasOption("d");
-            verbose = cmdline.hasOption("v");
-            if (cmdline.hasOption("cy")) {
-                try {
-                    concurrency = Integer.parseInt(cmdline.getOptionValue("cy"));
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid concurrency value : " + cmdline.getOptionValue("cy"));
-                }
-            }
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            MetadataUpdater metadataUpdater = dryrun ?
-                    new DryrunLogSegmentMetadataStoreUpdater(getConf(),
-                            getLogSegmentMetadataStore()) :
-                    LogSegmentMetadataStoreUpdater.createMetadataUpdater(getConf(),
-                            getLogSegmentMetadataStore());
-            OrderedScheduler scheduler = OrderedScheduler.newBuilder()
-                    .name("dlck-scheduler")
-                    .corePoolSize(Runtime.getRuntime().availableProcessors())
-                    .build();
-            ExecutorService executorService = Executors.newCachedThreadPool();
-            try {
-                checkAndRepairDLNamespace(getUri(), getNamespace(), metadataUpdater, scheduler,
-                                          verbose, !getForce(), concurrency);
-            } finally {
-                SchedulerUtils.shutdownScheduler(executorService, 5, TimeUnit.MINUTES);
-            }
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "dlck [options]";
-        }
-    }
-
-    static class DeleteStreamACLCommand extends PerDLCommand {
-
-        String stream = null;
-
-        DeleteStreamACLCommand() {
-            super("delete_stream_acl", "Delete ACL for a given stream");
-            options.addOption("s", "stream", true, "Stream to set ACL");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("s")) {
-                throw new ParseException("No stream to set ACL");
-            }
-            stream = cmdline.getOptionValue("s");
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(getZooKeeperClient(), getUri());
-            if (null == bkdlConfig.getACLRootPath()) {
-                // acl isn't enabled for this namespace.
-                System.err.println("ACL isn't enabled for namespace " + getUri());
-                return -1;
-            }
-            String zkPath = getUri() + "/" + bkdlConfig.getACLRootPath() + "/" + stream;
-            ZKAccessControl.delete(getZooKeeperClient(), zkPath);
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return null;
-        }
-    }
-
-    static class SetStreamACLCommand extends SetACLCommand {
-
-        String stream = null;
-
-        SetStreamACLCommand() {
-            super("set_stream_acl", "Set Default ACL for a given stream");
-            options.addOption("s", "stream", true, "Stream to set ACL");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("s")) {
-                throw new ParseException("No stream to set ACL");
-            }
-            stream = cmdline.getOptionValue("s");
-        }
-
-        @Override
-        protected String getZKPath(String zkRootPath) {
-            return zkRootPath + "/" + stream;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "set_stream_acl [options]";
-        }
-    }
-
-    static class SetDefaultACLCommand extends SetACLCommand {
-
-        SetDefaultACLCommand() {
-            super("set_default_acl", "Set Default ACL for a namespace");
-        }
-
-        @Override
-        protected String getZKPath(String zkRootPath) {
-            return zkRootPath;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "set_default_acl [options]";
-        }
-    }
-
-    static abstract class SetACLCommand extends PerDLCommand {
-
-        boolean denyWrite = false;
-        boolean denyTruncate = false;
-        boolean denyDelete = false;
-        boolean denyAcquire = false;
-        boolean denyRelease = false;
-
-        protected SetACLCommand(String name, String description) {
-            super(name, description);
-            options.addOption("dw", "deny-write", false, "Deny write/bulkWrite requests");
-            options.addOption("dt", "deny-truncate", false, "Deny truncate requests");
-            options.addOption("dd", "deny-delete", false, "Deny delete requests");
-            options.addOption("da", "deny-acquire", false, "Deny acquire requests");
-            options.addOption("dr", "deny-release", false, "Deny release requests");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            denyWrite = cmdline.hasOption("dw");
-            denyTruncate = cmdline.hasOption("dt");
-            denyDelete = cmdline.hasOption("dd");
-            denyAcquire = cmdline.hasOption("da");
-            denyRelease = cmdline.hasOption("dr");
-        }
-
-        protected abstract String getZKPath(String zkRootPath);
-
-        protected ZKAccessControl getZKAccessControl(ZooKeeperClient zkc, String zkPath) throws Exception {
-            ZKAccessControl accessControl;
-            try {
-                accessControl = Await.result(ZKAccessControl.read(zkc, zkPath, null));
-            } catch (KeeperException.NoNodeException nne) {
-                accessControl = new ZKAccessControl(new AccessControlEntry(), zkPath);
-            }
-            return accessControl;
-        }
-
-        protected void setZKAccessControl(ZooKeeperClient zkc, ZKAccessControl accessControl) throws Exception {
-            String zkPath = accessControl.getZKPath();
-            if (null == zkc.get().exists(zkPath, false)) {
-                accessControl.create(zkc);
-            } else {
-                accessControl.update(zkc);
-            }
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(getZooKeeperClient(), getUri());
-            if (null == bkdlConfig.getACLRootPath()) {
-                // acl isn't enabled for this namespace.
-                System.err.println("ACL isn't enabled for namespace " + getUri());
-                return -1;
-            }
-            String zkPath = getZKPath(getUri().getPath() + "/" + bkdlConfig.getACLRootPath());
-            ZKAccessControl accessControl = getZKAccessControl(getZooKeeperClient(), zkPath);
-            AccessControlEntry acl = accessControl.getAccessControlEntry();
-            acl.setDenyWrite(denyWrite);
-            acl.setDenyTruncate(denyTruncate);
-            acl.setDenyDelete(denyDelete);
-            acl.setDenyAcquire(denyAcquire);
-            acl.setDenyRelease(denyRelease);
-            setZKAccessControl(getZooKeeperClient(), accessControl);
-            return 0;
-        }
-
-    }
-
-    public DistributedLogAdmin() {
-        super();
-        commands.clear();
-        addCommand(new HelpCommand());
-        addCommand(new BindCommand());
-        addCommand(new UnbindCommand());
-        addCommand(new RepairSeqNoCommand());
-        addCommand(new DLCKCommand());
-        addCommand(new SetDefaultACLCommand());
-        addCommand(new SetStreamACLCommand());
-        addCommand(new DeleteStreamACLCommand());
-    }
-
-    @Override
-    protected String getName() {
-        return "dlog_admin";
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/package-info.java
deleted file mode 100644
index a7d6adb..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Admin Tools for DistributedLog
- */
-package com.twitter.distributedlog.admin;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/auditor/DLAuditor.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/auditor/DLAuditor.java
deleted file mode 100644
index a081606..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/auditor/DLAuditor.java
+++ /dev/null
@@ -1,630 +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 com.twitter.distributedlog.auditor;
-
-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 com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.BookKeeperClientBuilder;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.namespace.NamespaceDriver;
-import com.twitter.distributedlog.util.DLUtils;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.BookKeeperAccessor;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.meta.LedgerManager;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * DL Auditor will audit DL namespace, e.g. find leaked ledger, report disk usage by streams.
- */
-public class DLAuditor {
-
-    private static final Logger logger = LoggerFactory.getLogger(DLAuditor.class);
-
-    private final DistributedLogConfiguration conf;
-
-    public DLAuditor(DistributedLogConfiguration conf) {
-        this.conf = conf;
-    }
-
-    private ZooKeeperClient getZooKeeperClient(DistributedLogNamespace namespace) {
-        NamespaceDriver driver = namespace.getNamespaceDriver();
-        assert(driver instanceof BKNamespaceDriver);
-        return ((BKNamespaceDriver) driver).getWriterZKC();
-    }
-
-    private BookKeeperClient getBookKeeperClient(DistributedLogNamespace namespace) {
-        NamespaceDriver driver = namespace.getNamespaceDriver();
-        assert(driver instanceof BKNamespaceDriver);
-        return ((BKNamespaceDriver) driver).getReaderBKC();
-    }
-
-    private String validateAndGetZKServers(List<URI> uris) {
-        URI firstURI = uris.get(0);
-        String zkServers = BKNamespaceDriver.getZKServersFromDLUri(firstURI);
-        for (URI uri : uris) {
-            if (!zkServers.equalsIgnoreCase(BKNamespaceDriver.getZKServersFromDLUri(uri))) {
-                throw new IllegalArgumentException("Uris don't belong to same zookeeper cluster");
-            }
-        }
-        return zkServers;
-    }
-
-    private BKDLConfig resolveBKDLConfig(ZooKeeperClient zkc, List<URI> uris) throws IOException {
-        URI firstURI = uris.get(0);
-        BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, firstURI);
-        for (URI uri : uris) {
-            BKDLConfig anotherConfig = BKDLConfig.resolveDLConfig(zkc, uri);
-            if (!(Objects.equal(bkdlConfig.getBkLedgersPath(), anotherConfig.getBkLedgersPath())
-                    && Objects.equal(bkdlConfig.getBkZkServersForWriter(), anotherConfig.getBkZkServersForWriter()))) {
-                throw new IllegalArgumentException("Uris don't use same bookkeeper cluster");
-            }
-        }
-        return bkdlConfig;
-    }
-
-    public Pair<Set<Long>, Set<Long>> collectLedgers(List<URI> uris, List<List<String>> allocationPaths)
-            throws IOException {
-        Preconditions.checkArgument(uris.size() > 0, "No uri provided to audit");
-
-        String zkServers = validateAndGetZKServers(uris);
-        RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
-                conf.getZKRetryBackoffStartMillis(),
-                conf.getZKRetryBackoffMaxMillis(),
-                Integer.MAX_VALUE);
-        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
-                .name("DLAuditor-ZK")
-                .zkServers(zkServers)
-                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                .retryPolicy(retryPolicy)
-                .zkAclId(conf.getZkAclId())
-                .build();
-        ExecutorService executorService = Executors.newCachedThreadPool();
-        try {
-            BKDLConfig bkdlConfig = resolveBKDLConfig(zkc, uris);
-            logger.info("Resolved bookkeeper config : {}", bkdlConfig);
-
-            BookKeeperClient bkc = BookKeeperClientBuilder.newBuilder()
-                    .name("DLAuditor-BK")
-                    .dlConfig(conf)
-                    .zkServers(bkdlConfig.getBkZkServersForWriter())
-                    .ledgersPath(bkdlConfig.getBkLedgersPath())
-                    .build();
-            try {
-                Set<Long> bkLedgers = collectLedgersFromBK(bkc, executorService);
-                Set<Long> dlLedgers = collectLedgersFromDL(uris, allocationPaths);
-                return Pair.of(bkLedgers, dlLedgers);
-            } finally {
-                bkc.close();
-            }
-        } finally {
-            zkc.close();
-            executorService.shutdown();
-        }
-    }
-
-    /**
-     * Find leak ledgers phase 1: collect ledgers set.
-     */
-    private Set<Long> collectLedgersFromBK(BookKeeperClient bkc,
-                                           final ExecutorService executorService)
-            throws IOException {
-        LedgerManager lm = BookKeeperAccessor.getLedgerManager(bkc.get());
-
-        final Set<Long> ledgers = new HashSet<Long>();
-        final SettableFuture<Void> doneFuture = SettableFuture.create();
-
-        BookkeeperInternalCallbacks.Processor<Long> collector =
-                new BookkeeperInternalCallbacks.Processor<Long>() {
-            @Override
-            public void process(Long lid,
-                                final AsyncCallback.VoidCallback cb) {
-                synchronized (ledgers) {
-                    ledgers.add(lid);
-                    if (0 == ledgers.size() % 1000) {
-                        logger.info("Collected {} ledgers", ledgers.size());
-                    }
-                }
-                executorService.submit(new Runnable() {
-                    @Override
-                    public void run() {
-                        cb.processResult(BKException.Code.OK, null, null);
-                    }
-                });
-
-            }
-        };
-        AsyncCallback.VoidCallback finalCb = new AsyncCallback.VoidCallback() {
-            @Override
-            public void processResult(int rc, String path, Object ctx) {
-                if (BKException.Code.OK == rc) {
-                    doneFuture.set(null);
-                } else {
-                    doneFuture.setException(BKException.create(rc));
-                }
-            }
-        };
-        lm.asyncProcessLedgers(collector, finalCb, null, BKException.Code.OK,
-                BKException.Code.ZKException);
-        try {
-            doneFuture.get();
-            logger.info("Collected total {} ledgers", ledgers.size());
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw new DLInterruptedException("Interrupted on collecting ledgers : ", e);
-        } catch (ExecutionException e) {
-            if (e.getCause() instanceof IOException) {
-                throw (IOException)(e.getCause());
-            } else {
-                throw new IOException("Failed to collect ledgers : ", e.getCause());
-            }
-        }
-        return ledgers;
-    }
-
-    /**
-     * Find leak ledgers phase 2: collect ledgers from uris.
-     */
-    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());
-        try {
-            for (URI uri : uris) {
-                namespaces.add(
-                        DistributedLogNamespaceBuilder.newBuilder()
-                                .conf(conf)
-                                .uri(uri)
-                                .build());
-            }
-            final CountDownLatch doneLatch = new CountDownLatch(uris.size());
-            final AtomicInteger numFailures = new AtomicInteger(0);
-            ExecutorService executor = Executors.newFixedThreadPool(uris.size());
-            try {
-                int i = 0;
-                for (final DistributedLogNamespace namespace : namespaces) {
-                    final DistributedLogNamespace dlNamespace = namespace;
-                    final URI uri = uris.get(i);
-                    final List<String> aps = allocationPaths.get(i);
-                    i++;
-                    executor.submit(new Runnable() {
-                        @Override
-                        public void run() {
-                            try {
-                                logger.info("Collecting ledgers from {} : {}", uri, aps);
-                                collectLedgersFromAllocator(uri, namespace, aps, ledgers);
-                                synchronized (ledgers) {
-                                    logger.info("Collected {} ledgers from allocators for {} : {} ",
-                                            new Object[]{ledgers.size(), uri, ledgers});
-                                }
-                                collectLedgersFromDL(uri, namespace, ledgers);
-                            } catch (IOException e) {
-                                numFailures.incrementAndGet();
-                                logger.info("Error to collect ledgers from DL : ", e);
-                            }
-                            doneLatch.countDown();
-                        }
-                    });
-                }
-                try {
-                    doneLatch.await();
-                    if (numFailures.get() > 0) {
-                        throw new IOException(numFailures.get() + " errors to collect ledgers from DL");
-                    }
-                } catch (InterruptedException e) {
-                    Thread.currentThread().interrupt();
-                    logger.warn("Interrupted on collecting ledgers from DL : ", e);
-                    throw new DLInterruptedException("Interrupted on collecting ledgers from DL : ", e);
-                }
-            } finally {
-                executor.shutdown();
-            }
-        } finally {
-            for (DistributedLogNamespace namespace : namespaces) {
-                namespace.close();
-            }
-        }
-        return ledgers;
-    }
-
-    private void collectLedgersFromAllocator(final URI uri,
-                                             final DistributedLogNamespace namespace,
-                                             final List<String> allocationPaths,
-                                             final Set<Long> ledgers) throws IOException {
-        final LinkedBlockingQueue<String> poolQueue =
-                new LinkedBlockingQueue<String>();
-        for (String allocationPath : allocationPaths) {
-            String rootPath = uri.getPath() + "/" + allocationPath;
-            try {
-                List<String> pools = getZooKeeperClient(namespace).get().getChildren(rootPath, false);
-                for (String pool : pools) {
-                    poolQueue.add(rootPath + "/" + pool);
-                }
-            } catch (KeeperException e) {
-                throw new ZKException("Failed to get list of pools from " + rootPath, e);
-            } catch (InterruptedException e) {
-                Thread.currentThread().interrupt();
-                throw new DLInterruptedException("Interrupted on getting list of pools from " + rootPath, e);
-            }
-        }
-
-
-        logger.info("Collecting ledgers from allocators for {} : {}", uri, poolQueue);
-
-        executeAction(poolQueue, 10, new Action<String>() {
-            @Override
-            public void execute(String poolPath) throws IOException {
-                try {
-                    collectLedgersFromPool(poolPath);
-                } catch (InterruptedException e) {
-                    throw new DLInterruptedException("Interrupted on collecting ledgers from allocation pool " + poolPath, e);
-                } catch (KeeperException e) {
-                    throw new ZKException("Failed to collect ledgers from allocation pool " + poolPath, e.code());
-                }
-            }
-
-            private void collectLedgersFromPool(String poolPath)
-                    throws InterruptedException, ZooKeeperClient.ZooKeeperConnectionException, KeeperException {
-                List<String> allocators = getZooKeeperClient(namespace).get()
-                                        .getChildren(poolPath, false);
-                for (String allocator : allocators) {
-                    String allocatorPath = poolPath + "/" + allocator;
-                    byte[] data = getZooKeeperClient(namespace).get().getData(allocatorPath, false, new Stat());
-                    if (null != data && data.length > 0) {
-                        try {
-                            long ledgerId = DLUtils.bytes2LogSegmentId(data);
-                            synchronized (ledgers) {
-                                ledgers.add(ledgerId);
-                            }
-                        } catch (NumberFormatException nfe) {
-                            logger.warn("Invalid ledger found in allocator path {} : ", allocatorPath, nfe);
-                        }
-                    }
-                }
-            }
-        });
-
-        logger.info("Collected ledgers from allocators for {}.", uri);
-    }
-
-    private void collectLedgersFromDL(final URI uri,
-                                      final DistributedLogNamespace namespace,
-                                      final Set<Long> ledgers) throws IOException {
-        logger.info("Enumerating {} to collect streams.", uri);
-        Iterator<String> streams = namespace.getLogs();
-        final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
-        while (streams.hasNext()) {
-            streamQueue.add(streams.next());
-        }
-
-        logger.info("Collected {} streams from uri {} : {}",
-                    new Object[] { streamQueue.size(), uri, streams });
-
-        executeAction(streamQueue, 10, new Action<String>() {
-            @Override
-            public void execute(String stream) throws IOException {
-                collectLedgersFromStream(namespace, stream, ledgers);
-            }
-        });
-    }
-
-    private List<Long> collectLedgersFromStream(DistributedLogNamespace namespace,
-                                                String stream,
-                                                Set<Long> ledgers)
-            throws IOException {
-        DistributedLogManager dlm = namespace.openLog(stream);
-        try {
-            List<LogSegmentMetadata> segments = dlm.getLogSegments();
-            List<Long> sLedgers = new ArrayList<Long>();
-            for (LogSegmentMetadata segment : segments) {
-                synchronized (ledgers) {
-                    ledgers.add(segment.getLogSegmentId());
-                }
-                sLedgers.add(segment.getLogSegmentId());
-            }
-            return sLedgers;
-        } finally {
-            dlm.close();
-        }
-    }
-
-    /**
-     * Calculating stream space usage from given <i>uri</i>.
-     *
-     * @param uri dl uri
-     * @throws IOException
-     */
-    public Map<String, Long> calculateStreamSpaceUsage(final URI uri) throws IOException {
-        logger.info("Collecting stream space usage for {}.", uri);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(uri)
-                .build();
-        try {
-            return calculateStreamSpaceUsage(uri, namespace);
-        } finally {
-            namespace.close();
-        }
-    }
-
-    private Map<String, Long> calculateStreamSpaceUsage(
-            final URI uri, final DistributedLogNamespace namespace)
-        throws IOException {
-        Iterator<String> streams = namespace.getLogs();
-        final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
-        while (streams.hasNext()) {
-            streamQueue.add(streams.next());
-        }
-
-        final Map<String, Long> streamSpaceUsageMap =
-                new ConcurrentSkipListMap<String, Long>();
-        final AtomicInteger numStreamsCollected = new AtomicInteger(0);
-
-        executeAction(streamQueue, 10, new Action<String>() {
-            @Override
-            public void execute(String stream) throws IOException {
-                streamSpaceUsageMap.put(stream,
-                        calculateStreamSpaceUsage(namespace, stream));
-                if (numStreamsCollected.incrementAndGet() % 1000 == 0) {
-                    logger.info("Calculated {} streams from uri {}.", numStreamsCollected.get(), uri);
-                }
-            }
-        });
-
-        return streamSpaceUsageMap;
-    }
-
-    private long calculateStreamSpaceUsage(final DistributedLogNamespace namespace,
-                                           final String stream) throws IOException {
-        DistributedLogManager dlm = namespace.openLog(stream);
-        long totalBytes = 0;
-        try {
-            List<LogSegmentMetadata> segments = dlm.getLogSegments();
-            for (LogSegmentMetadata segment : segments) {
-                try {
-                    LedgerHandle lh = getBookKeeperClient(namespace).get().openLedgerNoRecovery(segment.getLogSegmentId(),
-                            BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
-                    totalBytes += lh.getLength();
-                    lh.close();
-                } catch (BKException e) {
-                    logger.error("Failed to open ledger {} : ", segment.getLogSegmentId(), e);
-                    throw new IOException("Failed to open ledger " + segment.getLogSegmentId(), e);
-                } catch (InterruptedException e) {
-                    logger.warn("Interrupted on opening ledger {} : ", segment.getLogSegmentId(), e);
-                    Thread.currentThread().interrupt();
-                    throw new DLInterruptedException("Interrupted on opening ledger " + segment.getLogSegmentId(), e);
-                }
-            }
-        } finally {
-            dlm.close();
-        }
-        return totalBytes;
-    }
-
-    public long calculateLedgerSpaceUsage(URI uri) throws IOException {
-        List<URI> uris = Lists.newArrayList(uri);
-        String zkServers = validateAndGetZKServers(uris);
-        RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
-                conf.getZKRetryBackoffStartMillis(),
-                conf.getZKRetryBackoffMaxMillis(),
-                Integer.MAX_VALUE);
-        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
-                .name("DLAuditor-ZK")
-                .zkServers(zkServers)
-                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                .retryPolicy(retryPolicy)
-                .zkAclId(conf.getZkAclId())
-                .build();
-        ExecutorService executorService = Executors.newCachedThreadPool();
-        try {
-            BKDLConfig bkdlConfig = resolveBKDLConfig(zkc, uris);
-            logger.info("Resolved bookkeeper config : {}", bkdlConfig);
-
-            BookKeeperClient bkc = BookKeeperClientBuilder.newBuilder()
-                    .name("DLAuditor-BK")
-                    .dlConfig(conf)
-                    .zkServers(bkdlConfig.getBkZkServersForWriter())
-                    .ledgersPath(bkdlConfig.getBkLedgersPath())
-                    .build();
-            try {
-                return calculateLedgerSpaceUsage(bkc, executorService);
-            } finally {
-                bkc.close();
-            }
-        } finally {
-            zkc.close();
-            executorService.shutdown();
-        }
-    }
-
-    private long calculateLedgerSpaceUsage(BookKeeperClient bkc,
-                                           final ExecutorService executorService)
-        throws IOException {
-        final AtomicLong totalBytes = new AtomicLong(0);
-        final AtomicLong totalEntries = new AtomicLong(0);
-        final AtomicLong numLedgers = new AtomicLong(0);
-
-        LedgerManager lm = BookKeeperAccessor.getLedgerManager(bkc.get());
-
-        final SettableFuture<Void> doneFuture = SettableFuture.create();
-        final BookKeeper bk = bkc.get();
-
-        BookkeeperInternalCallbacks.Processor<Long> collector =
-                new BookkeeperInternalCallbacks.Processor<Long>() {
-            @Override
-            public void process(final Long lid,
-                                final AsyncCallback.VoidCallback cb) {
-                numLedgers.incrementAndGet();
-                executorService.submit(new Runnable() {
-                    @Override
-                    public void run() {
-                        bk.asyncOpenLedgerNoRecovery(lid, BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8),
-                                new org.apache.bookkeeper.client.AsyncCallback.OpenCallback() {
-                            @Override
-                            public void openComplete(int rc, LedgerHandle lh, Object ctx) {
-                                final int cbRc;
-                                if (BKException.Code.OK == rc) {
-                                    totalBytes.addAndGet(lh.getLength());
-                                    totalEntries.addAndGet(lh.getLastAddConfirmed() + 1);
-                                    cbRc = rc;
-                                } else {
-                                    cbRc = BKException.Code.ZKException;
-                                }
-                                executorService.submit(new Runnable() {
-                                    @Override
-                                    public void run() {
-                                        cb.processResult(cbRc, null, null);
-                                    }
-                                });
-                            }
-                        }, null);
-                    }
-                });
-            }
-        };
-        AsyncCallback.VoidCallback finalCb = new AsyncCallback.VoidCallback() {
-            @Override
-            public void processResult(int rc, String path, Object ctx) {
-                if (BKException.Code.OK == rc) {
-                    doneFuture.set(null);
-                } else {
-                    doneFuture.setException(BKException.create(rc));
-                }
-            }
-        };
-        lm.asyncProcessLedgers(collector, finalCb, null, BKException.Code.OK, BKException.Code.ZKException);
-        try {
-            doneFuture.get();
-            logger.info("calculated {} ledgers\n\ttotal bytes = {}\n\ttotal entries = {}",
-                    new Object[] { numLedgers.get(), totalBytes.get(), totalEntries.get() });
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw new DLInterruptedException("Interrupted on calculating ledger space : ", e);
-        } catch (ExecutionException e) {
-            if (e.getCause() instanceof IOException) {
-                throw (IOException)(e.getCause());
-            } else {
-                throw new IOException("Failed to calculate ledger space : ", e.getCause());
-            }
-        }
-        return totalBytes.get();
-    }
-
-    public void close() {
-        // no-op
-    }
-
-    static interface Action<T> {
-        void execute(T item) throws IOException ;
-    }
-
-    static <T> void executeAction(final LinkedBlockingQueue<T> queue,
-                                  final int numThreads,
-                                  final Action<T> action) throws IOException {
-        final CountDownLatch failureLatch = new CountDownLatch(1);
-        final CountDownLatch doneLatch = new CountDownLatch(queue.size());
-        final AtomicInteger numFailures = new AtomicInteger(0);
-        final AtomicInteger completedThreads = new AtomicInteger(0);
-
-        ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
-        try {
-            for (int i = 0 ; i < numThreads; i++) {
-                executorService.submit(new Runnable() {
-                    @Override
-                    public void run() {
-                        while (true) {
-                            T item = queue.poll();
-                            if (null == item) {
-                                break;
-                            }
-                            try {
-                                action.execute(item);
-                            } catch (IOException ioe) {
-                                logger.error("Failed to execute action on item '{}'", item, ioe);
-                                numFailures.incrementAndGet();
-                                failureLatch.countDown();
-                                break;
-                            }
-                            doneLatch.countDown();
-                        }
-                        if (numFailures.get() == 0 && completedThreads.incrementAndGet() == numThreads) {
-                            failureLatch.countDown();
-                        }
-                    }
-                });
-            }
-            try {
-                failureLatch.await();
-                if (numFailures.get() > 0) {
-                    throw new IOException("Encountered " + numFailures.get() + " failures on executing action.");
-                }
-                doneLatch.await();
-            } catch (InterruptedException ie) {
-                Thread.currentThread().interrupt();
-                logger.warn("Interrupted on executing action", ie);
-                throw new DLInterruptedException("Interrupted on executing action", ie);
-            }
-        } finally {
-            executorService.shutdown();
-        }
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/DynamicQuorumConfigProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/DynamicQuorumConfigProvider.java
deleted file mode 100644
index 871997f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/DynamicQuorumConfigProvider.java
+++ /dev/null
@@ -1,37 +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 com.twitter.distributedlog.bk;
-
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-
-/**
- * Provider returns quorum configs based on dynamic configuration.
- */
-public class DynamicQuorumConfigProvider implements QuorumConfigProvider {
-
-    private final DynamicDistributedLogConfiguration conf;
-
-    public DynamicQuorumConfigProvider(DynamicDistributedLogConfiguration conf) {
-        this.conf = conf;
-    }
-
-    @Override
-    public QuorumConfig getQuorumConfig() {
-        return conf.getQuorumConfig();
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/ImmutableQuorumConfigProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/ImmutableQuorumConfigProvider.java
deleted file mode 100644
index 6c3f06e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/ImmutableQuorumConfigProvider.java
+++ /dev/null
@@ -1,35 +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 com.twitter.distributedlog.bk;
-
-/**
- * Provider that returns an immutable quorum config.
- */
-public class ImmutableQuorumConfigProvider implements QuorumConfigProvider {
-
-    private final QuorumConfig quorumConfig;
-
-    public ImmutableQuorumConfigProvider(QuorumConfig quorumConfig) {
-        this.quorumConfig = quorumConfig;
-    }
-
-    @Override
-    public QuorumConfig getQuorumConfig() {
-        return quorumConfig;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocator.java
deleted file mode 100644
index c14f374..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocator.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.bk;
-
-import com.twitter.distributedlog.util.Allocator;
-import org.apache.bookkeeper.client.LedgerHandle;
-
-import java.io.IOException;
-
-public interface LedgerAllocator extends Allocator<LedgerHandle, Object> {
-
-    /**
-     * Start the ledger allocator. The implementaion should not be blocking call.
-     */
-    void start() throws IOException;
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorDelegator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorDelegator.java
deleted file mode 100644
index b76d03a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorDelegator.java
+++ /dev/null
@@ -1,83 +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 com.twitter.distributedlog.bk;
-
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.client.LedgerHandle;
-
-import java.io.IOException;
-
-/**
- * Delegator of the underlying allocator. If it owns the allocator, it takes
- * the responsibility of start the allocator and close the allocator.
- */
-public class LedgerAllocatorDelegator implements LedgerAllocator {
-
-    private final LedgerAllocator allocator;
-    private final boolean ownAllocator;
-
-    /**
-     * Create an allocator's delegator.
-     *
-     * @param allocator
-     *          the underlying allocator
-     * @param ownAllocator
-     *          whether to own the allocator
-     */
-    public LedgerAllocatorDelegator(LedgerAllocator allocator,
-                                    boolean ownAllocator)
-            throws IOException {
-        this.allocator = allocator;
-        this.ownAllocator = ownAllocator;
-        if (this.ownAllocator) {
-            this.allocator.start();
-        }
-    }
-
-    @Override
-    public void start() throws IOException {
-        // no-op
-    }
-
-    @Override
-    public Future<Void> delete() {
-        return Future.exception(new UnsupportedOperationException("Can't delete an allocator by delegator"));
-    }
-
-    @Override
-    public void allocate() throws IOException {
-        this.allocator.allocate();
-    }
-
-    @Override
-    public Future<LedgerHandle> tryObtain(Transaction<Object> txn,
-                                          OpListener<LedgerHandle> listener) {
-        return this.allocator.tryObtain(txn, listener);
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        if (ownAllocator) {
-            return this.allocator.asyncClose();
-        } else {
-            return Future.value(null);
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorPool.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorPool.java
deleted file mode 100644
index dd0894e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorPool.java
+++ /dev/null
@@ -1,458 +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 com.twitter.distributedlog.bk;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.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;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-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;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-public class LedgerAllocatorPool implements LedgerAllocator {
-
-    static final Logger logger = LoggerFactory.getLogger(LedgerAllocatorPool.class);
-
-    private final DistributedLogConfiguration conf;
-    private final QuorumConfigProvider quorumConfigProvider;
-    private final BookKeeperClient bkc;
-    private final ZooKeeperClient zkc;
-    private final ScheduledExecutorService scheduledExecutorService;
-    private final String poolPath;
-    private final int corePoolSize;
-
-    private final LinkedList<SimpleLedgerAllocator> pendingList =
-            new LinkedList<SimpleLedgerAllocator>();
-    private final LinkedList<SimpleLedgerAllocator> allocatingList =
-            new LinkedList<SimpleLedgerAllocator>();
-    private final Map<String, SimpleLedgerAllocator> rescueMap =
-            new HashMap<String, SimpleLedgerAllocator>();
-    private final Map<LedgerHandle, SimpleLedgerAllocator> obtainMap =
-            new HashMap<LedgerHandle, SimpleLedgerAllocator>();
-    private final Map<SimpleLedgerAllocator, LedgerHandle> reverseObtainMap =
-            new HashMap<SimpleLedgerAllocator, LedgerHandle>();
-
-    public LedgerAllocatorPool(String poolPath, int corePoolSize,
-                               DistributedLogConfiguration conf,
-                               ZooKeeperClient zkc,
-                               BookKeeperClient bkc,
-                               ScheduledExecutorService scheduledExecutorService) throws IOException {
-        this.poolPath = poolPath;
-        this.corePoolSize = corePoolSize;
-        this.conf = conf;
-        this.quorumConfigProvider =
-                new ImmutableQuorumConfigProvider(conf.getQuorumConfig());
-        this.zkc = zkc;
-        this.bkc = bkc;
-        this.scheduledExecutorService = scheduledExecutorService;
-        initializePool();
-    }
-
-    @Override
-    public void start() throws IOException {
-        for (LedgerAllocator allocator : pendingList) {
-            // issue allocating requests during initialize
-            allocator.allocate();
-        }
-    }
-
-    @VisibleForTesting
-    synchronized int pendingListSize() {
-        return pendingList.size();
-    }
-
-    @VisibleForTesting
-    synchronized int allocatingListSize() {
-        return allocatingList.size();
-    }
-
-    @VisibleForTesting
-    public synchronized int obtainMapSize() {
-        return obtainMap.size();
-    }
-
-    @VisibleForTesting
-    synchronized int rescueSize() {
-        return rescueMap.size();
-    }
-
-    @VisibleForTesting
-    synchronized SimpleLedgerAllocator getLedgerAllocator(LedgerHandle lh) {
-        return obtainMap.get(lh);
-    }
-
-    private void initializePool() throws IOException {
-        try {
-            List<String> allocators;
-            try {
-                allocators = zkc.get().getChildren(poolPath, false);
-            } catch (KeeperException.NoNodeException e) {
-                logger.info("Allocator Pool {} doesn't exist. Creating it.", poolPath);
-                ZkUtils.createFullPathOptimistic(zkc.get(), poolPath, new byte[0], zkc.getDefaultACL(),
-                        CreateMode.PERSISTENT);
-                allocators = zkc.get().getChildren(poolPath, false);
-            }
-            if (null == allocators) {
-                allocators = new ArrayList<String>();
-            }
-            if (allocators.size() < corePoolSize) {
-                createAllocators(corePoolSize - allocators.size());
-                allocators = zkc.get().getChildren(poolPath, false);
-            }
-            initializeAllocators(allocators);
-        } catch (InterruptedException ie) {
-            throw new DLInterruptedException("Interrupted when ensuring " + poolPath + " created : ", ie);
-        } catch (KeeperException ke) {
-            throw new IOException("Encountered zookeeper exception when initializing pool " + poolPath + " : ", ke);
-        }
-    }
-
-    private void createAllocators(int numAllocators) throws InterruptedException, IOException {
-        final AtomicInteger numPendings = new AtomicInteger(numAllocators);
-        final AtomicInteger numFailures = new AtomicInteger(0);
-        final CountDownLatch latch = new CountDownLatch(1);
-        AsyncCallback.StringCallback createCallback = new AsyncCallback.StringCallback() {
-            @Override
-            public void processResult(int rc, String path, Object ctx, String name) {
-                if (KeeperException.Code.OK.intValue() != rc) {
-                    numFailures.incrementAndGet();
-                    latch.countDown();
-                    return;
-                }
-                if (numPendings.decrementAndGet() == 0 && numFailures.get() == 0) {
-                    latch.countDown();
-                }
-            }
-        };
-        for (int i = 0; i < numAllocators; i++) {
-            zkc.get().create(poolPath + "/A", new byte[0],
-                             zkc.getDefaultACL(),
-                             CreateMode.PERSISTENT_SEQUENTIAL,
-                             createCallback, null);
-        }
-        latch.await();
-        if (numFailures.get() > 0) {
-            throw new IOException("Failed to create " + numAllocators + " allocators.");
-        }
-    }
-
-    /**
-     * Initialize simple allocators with given list of allocator names <i>allocators</i>.
-     * It initializes a simple allocator with its simple allocator path.
-     */
-    private void initializeAllocators(List<String> allocators) throws IOException, InterruptedException {
-        final AtomicInteger numPendings = new AtomicInteger(allocators.size());
-        final AtomicInteger numFailures = new AtomicInteger(0);
-        final CountDownLatch latch = new CountDownLatch(numPendings.get() > 0 ? 1 : 0);
-        AsyncCallback.DataCallback dataCallback = new AsyncCallback.DataCallback() {
-            @Override
-            public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
-                if (KeeperException.Code.OK.intValue() != rc) {
-                    numFailures.incrementAndGet();
-                    latch.countDown();
-                    return;
-                }
-                Versioned<byte[]> allocatorData =
-                        new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
-                SimpleLedgerAllocator allocator =
-                        new SimpleLedgerAllocator(path, allocatorData, quorumConfigProvider, zkc, bkc);
-                allocator.start();
-                pendingList.add(allocator);
-                if (numPendings.decrementAndGet() == 0 && numFailures.get() == 0) {
-                    latch.countDown();
-                }
-            }
-        };
-        for (String name : allocators) {
-            String path = poolPath + "/" + name;
-            zkc.get().getData(path, false, dataCallback, null);
-        }
-        latch.await();
-        if (numFailures.get() > 0) {
-            throw new IOException("Failed to initialize allocators : " + allocators);
-        }
-    }
-
-    private void scheduleAllocatorRescue(final SimpleLedgerAllocator ledgerAllocator) {
-        try {
-            scheduledExecutorService.schedule(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        rescueAllocator(ledgerAllocator);
-                    } catch (DLInterruptedException dle) {
-                        Thread.currentThread().interrupt();
-                    }
-                }
-            }, conf.getZKRetryBackoffStartMillis(), TimeUnit.MILLISECONDS);
-        } catch (RejectedExecutionException ree) {
-            logger.warn("Failed to schedule rescuing ledger allocator {} : ", ledgerAllocator.allocatePath, ree);
-        }
-    }
-
-    /**
-     * Rescue a ledger allocator from an ERROR state
-     * @param ledgerAllocator
-     *          ledger allocator to rescue
-     */
-    private void rescueAllocator(final SimpleLedgerAllocator ledgerAllocator) throws DLInterruptedException {
-        SimpleLedgerAllocator oldAllocator;
-        synchronized (this) {
-            oldAllocator = rescueMap.put(ledgerAllocator.allocatePath, ledgerAllocator);
-        }
-        if (oldAllocator != null) {
-            logger.info("ledger allocator {} is being rescued.", ledgerAllocator.allocatePath);
-            return;
-        }
-        try {
-            zkc.get().getData(ledgerAllocator.allocatePath, false, new AsyncCallback.DataCallback() {
-                @Override
-                public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
-                    boolean retry = false;
-                    SimpleLedgerAllocator newAllocator = null;
-                    if (KeeperException.Code.OK.intValue() == rc) {
-                        Versioned<byte[]> allocatorData =
-                                new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
-                        logger.info("Rescuing ledger allocator {}.", path);
-                        newAllocator = new SimpleLedgerAllocator(path, allocatorData, quorumConfigProvider, zkc, bkc);
-                        newAllocator.start();
-                        logger.info("Rescued ledger allocator {}.", path);
-                    } else if (KeeperException.Code.NONODE.intValue() == rc) {
-                        logger.info("Ledger allocator {} doesn't exist, skip rescuing it.", path);
-                    } else {
-                        retry = true;
-                    }
-                    synchronized (LedgerAllocatorPool.this) {
-                        rescueMap.remove(ledgerAllocator.allocatePath);
-                        if (null != newAllocator) {
-                            pendingList.addLast(newAllocator);
-                        }
-                    }
-                    if (retry) {
-                        scheduleAllocatorRescue(ledgerAllocator);
-                    }
-                }
-            }, null);
-        } catch (InterruptedException ie) {
-            logger.warn("Interrupted on rescuing ledger allocator {} : ", ledgerAllocator.allocatePath, ie);
-            synchronized (LedgerAllocatorPool.this) {
-                rescueMap.remove(ledgerAllocator.allocatePath);
-            }
-            throw new DLInterruptedException("Interrupted on rescuing ledger allocator " + ledgerAllocator.allocatePath, ie);
-        } catch (IOException ioe) {
-            logger.warn("Failed to rescue ledger allocator {}, retry rescuing it later : ", ledgerAllocator.allocatePath, ioe);
-            synchronized (LedgerAllocatorPool.this) {
-                rescueMap.remove(ledgerAllocator.allocatePath);
-            }
-            scheduleAllocatorRescue(ledgerAllocator);
-        }
-    }
-
-    @Override
-    public void allocate() throws IOException {
-        SimpleLedgerAllocator allocator;
-        synchronized (this) {
-            if (pendingList.isEmpty()) {
-                // if no ledger allocator available, we should fail it immediately, which the request will be redirected to other
-                // proxies
-                throw new IOException("No ledger allocator available under " + poolPath + ".");
-            } else {
-                allocator = pendingList.removeFirst();
-            }
-        }
-        boolean success = false;
-        try {
-            allocator.allocate();
-            synchronized (this) {
-                allocatingList.addLast(allocator);
-            }
-            success = true;
-        } finally {
-            if (!success) {
-                rescueAllocator(allocator);
-            }
-        }
-    }
-
-    @Override
-    public Future<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 + "."));
-            } else {
-                allocator = allocatingList.removeFirst();
-            }
-        }
-
-        final Promise<LedgerHandle> tryObtainPromise = new Promise<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);
-                }
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                try {
-                    rescueAllocator(allocator);
-                } catch (IOException ioe) {
-                    logger.info("Failed to rescue allocator {}", allocator.allocatePath, ioe);
-                }
-                tryObtainPromise.setException(cause);
-            }
-        };
-
-        allocator.tryObtain(txn, new Transaction.OpListener<LedgerHandle>() {
-            @Override
-            public void onCommit(LedgerHandle lh) {
-                confirmObtain(allocator);
-                listener.onCommit(lh);
-            }
-
-            @Override
-            public void onAbort(Throwable t) {
-                abortObtain(allocator);
-                listener.onAbort(t);
-            }
-        }).addEventListener(tryObtainListener);
-        return tryObtainPromise;
-    }
-
-    void confirmObtain(SimpleLedgerAllocator allocator) {
-        synchronized (this) {
-            LedgerHandle lh = reverseObtainMap.remove(allocator);
-            if (null != lh) {
-                obtainMap.remove(lh);
-            }
-        }
-        synchronized (this) {
-            pendingList.addLast(allocator);
-        }
-    }
-
-    void abortObtain(SimpleLedgerAllocator allocator) {
-        synchronized (this) {
-            LedgerHandle lh = reverseObtainMap.remove(allocator);
-            if (null != lh) {
-                obtainMap.remove(lh);
-            }
-        }
-        // if a ledger allocator is aborted, it is better to rescue it. since the ledger allocator might
-        // already encounter BadVersion exception.
-        try {
-            rescueAllocator(allocator);
-        } catch (DLInterruptedException e) {
-            logger.warn("Interrupted on rescuing ledger allocator pool {} : ", poolPath, e);
-            Thread.currentThread().interrupt();
-        }
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        List<LedgerAllocator> allocatorsToClose;
-        synchronized (this) {
-            allocatorsToClose = Lists.newArrayListWithExpectedSize(
-                    pendingList.size() + allocatingList.size() + obtainMap.size());
-            for (LedgerAllocator allocator : pendingList) {
-                allocatorsToClose.add(allocator);
-            }
-            for (LedgerAllocator allocator : allocatingList) {
-                allocatorsToClose.add(allocator);
-            }
-            for (LedgerAllocator allocator : obtainMap.values()) {
-                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;
-            }
-        });
-    }
-
-    @Override
-    public Future<Void> delete() {
-        List<LedgerAllocator> allocatorsToDelete;
-        synchronized (this) {
-            allocatorsToDelete = Lists.newArrayListWithExpectedSize(
-                    pendingList.size() + allocatingList.size() + obtainMap.size());
-            for (LedgerAllocator allocator : pendingList) {
-                allocatorsToDelete.add(allocator);
-            }
-            for (LedgerAllocator allocator : allocatingList) {
-                allocatorsToDelete.add(allocator);
-            }
-            for (LedgerAllocator allocator : obtainMap.values()) {
-                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));
-            }
-        });
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorUtils.java
deleted file mode 100644
index 0db6d74..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorUtils.java
+++ /dev/null
@@ -1,54 +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 com.twitter.distributedlog.bk;
-
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.ZooKeeperClient;
-
-import java.io.IOException;
-import java.util.concurrent.ScheduledExecutorService;
-
-public class LedgerAllocatorUtils {
-
-    /**
-     * Create ledger allocator pool.
-     *
-     * @param poolPath
-     *          ledger allocator pool path.
-     * @param corePoolSize
-     *          ledger allocator pool core size.
-     * @param conf
-     *          distributedlog configuration.
-     * @param zkc
-     *          zookeeper client
-     * @param bkc
-     *          bookkeeper client
-     * @return ledger allocator
-     * @throws IOException
-     */
-    public static LedgerAllocator createLedgerAllocatorPool(
-            String poolPath,
-            int corePoolSize,
-            DistributedLogConfiguration conf,
-            ZooKeeperClient zkc,
-            BookKeeperClient bkc,
-            ScheduledExecutorService scheduledExecutorService) throws IOException {
-        return new LedgerAllocatorPool(poolPath, corePoolSize, conf, zkc, bkc, scheduledExecutorService);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfig.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfig.java
deleted file mode 100644
index a9cc16c..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfig.java
+++ /dev/null
@@ -1,92 +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 com.twitter.distributedlog.bk;
-
-import com.google.common.base.Objects;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Configuration for quorums
- */
-public class QuorumConfig {
-
-    private static final Logger logger = LoggerFactory.getLogger(QuorumConfig.class);
-
-    private final int ensembleSize;
-    private final int writeQuorumSize;
-    private final int ackQuorumSize;
-
-    public QuorumConfig(int ensembleSize,
-                        int writeQuorumSize,
-                        int ackQuorumSize) {
-        this.ensembleSize = ensembleSize;
-        if (this.ensembleSize < writeQuorumSize) {
-            this.writeQuorumSize = this.ensembleSize;
-            logger.warn("Setting write quorum size {} greater than ensemble size {}",
-                    writeQuorumSize, this.ensembleSize);
-        } else {
-            this.writeQuorumSize = writeQuorumSize;
-        }
-        if (this.writeQuorumSize < ackQuorumSize) {
-            this.ackQuorumSize = this.writeQuorumSize;
-            logger.warn("Setting write ack quorum size {} greater than write quorum size {}",
-                    ackQuorumSize, this.writeQuorumSize);
-        } else {
-            this.ackQuorumSize = ackQuorumSize;
-        }
-    }
-
-    public int getEnsembleSize() {
-        return ensembleSize;
-    }
-
-    public int getWriteQuorumSize() {
-        return writeQuorumSize;
-    }
-
-    public int getAckQuorumSize() {
-        return ackQuorumSize;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hashCode(ensembleSize, writeQuorumSize, ackQuorumSize);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (!(obj instanceof QuorumConfig)) {
-            return false;
-        }
-        QuorumConfig other = (QuorumConfig) obj;
-        return ensembleSize == other.ensembleSize
-                && writeQuorumSize == other.writeQuorumSize
-                && ackQuorumSize == other.ackQuorumSize;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("QuorumConfig[ensemble=")
-          .append(ensembleSize).append(", write quorum=")
-          .append(writeQuorumSize).append(", ack quorum=")
-          .append(ackQuorumSize).append("]");
-        return sb.toString();
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfigProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfigProvider.java
deleted file mode 100644
index 2f65427..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfigProvider.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.bk;
-
-/**
- * Provider to provide quorum config
- */
-public interface QuorumConfigProvider {
-
-    /**
-     * Get the quorum config for a given log stream.
-     *
-     * @return quorum config
-     */
-    QuorumConfig getQuorumConfig();
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/SimpleLedgerAllocator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/SimpleLedgerAllocator.java
deleted file mode 100644
index ab5976e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/SimpleLedgerAllocator.java
+++ /dev/null
@@ -1,536 +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 com.twitter.distributedlog.bk;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.distributedlog.zk.ZKTransaction;
-import com.twitter.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;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-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;
-import java.util.List;
-
-/**
- * Allocator to allocate ledgers.
- */
-public class SimpleLedgerAllocator implements LedgerAllocator, FutureEventListener<LedgerHandle>, OpListener<Version> {
-
-    static final Logger LOG = LoggerFactory.getLogger(SimpleLedgerAllocator.class);
-
-    static enum Phase {
-        ALLOCATING, ALLOCATED, HANDING_OVER, HANDED_OVER, ERROR
-    }
-
-    static class AllocationException extends IOException {
-
-        private static final long serialVersionUID = -1111397872059426882L;
-
-        private final Phase phase;
-
-        public AllocationException(Phase phase, String msg) {
-            super(msg);
-            this.phase = phase;
-        }
-
-        public Phase getPhase() {
-            return this.phase;
-        }
-
-    }
-
-    static class ConcurrentObtainException extends AllocationException {
-
-        private static final long serialVersionUID = -8532471098537176913L;
-
-        public ConcurrentObtainException(Phase phase, String msg) {
-            super(phase, msg);
-        }
-    }
-
-    // zookeeper client
-    final ZooKeeperClient zkc;
-    // bookkeeper client
-    final BookKeeperClient bkc;
-    // znode path
-    final String allocatePath;
-    // allocation phase
-    Phase phase = Phase.HANDED_OVER;
-    // version
-    ZkVersion version = new ZkVersion(-1);
-    // outstanding allocation
-    Promise<LedgerHandle> allocatePromise;
-    // outstanding tryObtain transaction
-    Transaction<Object> tryObtainTxn = null;
-    OpListener<LedgerHandle> tryObtainListener = null;
-    // ledger id left from previous allocation
-    Long ledgerIdLeftFromPrevAllocation = null;
-    // Allocated Ledger
-    LedgerHandle allocatedLh = null;
-
-    Future<Void> closeFuture = null;
-    final LinkedList<Future<Void>> ledgerDeletions =
-            new LinkedList<Future<Void>>();
-
-    // Ledger configuration
-    private final QuorumConfigProvider quorumConfigProvider;
-
-    static Future<Versioned<byte[]>> getAndCreateAllocationData(final String allocatePath,
-                                                                final ZooKeeperClient zkc) {
-        return Utils.zkGetData(zkc, allocatePath, false)
-                .flatMap(new AbstractFunction1<Versioned<byte[]>, Future<Versioned<byte[]>>>() {
-            @Override
-            public Future<Versioned<byte[]>> apply(Versioned<byte[]> result) {
-                if (null != result && null != result.getVersion() && null != result.getValue()) {
-                    return Future.value(result);
-                }
-                return createAllocationData(allocatePath, zkc);
-            }
-        });
-    }
-
-    private static Future<Versioned<byte[]>> createAllocationData(final String allocatePath,
-                                                                  final ZooKeeperClient zkc) {
-        try {
-            final Promise<Versioned<byte[]>> promise = new Promise<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,
-                                        new ZkVersion(stat.getVersion())));
-                            } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
-                                Utils.zkGetData(zkc, allocatePath, false).proxyTo(promise);
-                            } else {
-                                promise.setException(FutureUtils.zkException(
-                                        KeeperException.create(KeeperException.Code.get(rc)), allocatePath));
-                            }
-                        }
-                    }, null);
-            return promise;
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            return Future.exception(FutureUtils.zkException(e, allocatePath));
-        } catch (InterruptedException e) {
-            return Future.exception(FutureUtils.zkException(e, allocatePath));
-        }
-    }
-
-    public static Future<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,
-                    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);
-            }
-        });
-    }
-
-    /**
-     * Construct a ledger allocator.
-     *
-     * @param allocatePath
-     *          znode path to store the allocated ledger.
-     * @param allocationData
-     *          allocation data.
-     * @param quorumConfigProvider
-     *          Quorum configuration provider.
-     * @param zkc
-     *          zookeeper client.
-     * @param bkc
-     *          bookkeeper client.
-     */
-    public SimpleLedgerAllocator(String allocatePath,
-                                 Versioned<byte[]> allocationData,
-                                 QuorumConfigProvider quorumConfigProvider,
-                                 ZooKeeperClient zkc,
-                                 BookKeeperClient bkc) {
-        this.zkc = zkc;
-        this.bkc = bkc;
-        this.allocatePath = allocatePath;
-        this.quorumConfigProvider = quorumConfigProvider;
-        initialize(allocationData);
-    }
-
-    /**
-     * Initialize the allocator.
-     *
-     * @param allocationData
-     *          Allocation Data.
-     */
-    private void initialize(Versioned<byte[]> allocationData) {
-        setVersion((ZkVersion) allocationData.getVersion());
-        byte[] data = allocationData.getValue();
-        if (null != data && data.length > 0) {
-            // delete the allocated ledger since this is left by last allocation.
-            try {
-                ledgerIdLeftFromPrevAllocation = DLUtils.bytes2LogSegmentId(data);
-            } catch (NumberFormatException nfe) {
-                LOG.warn("Invalid data found in allocator path {} : ", allocatePath, nfe);
-            }
-        }
-
-    }
-
-    private synchronized void deleteLedgerLeftFromPreviousAllocationIfNecessary() {
-        if (null != ledgerIdLeftFromPrevAllocation) {
-            LOG.info("Deleting allocated-but-unused ledger left from previous allocation {}.", ledgerIdLeftFromPrevAllocation);
-            deleteLedger(ledgerIdLeftFromPrevAllocation);
-            ledgerIdLeftFromPrevAllocation = null;
-        }
-    }
-
-    @Override
-    public synchronized void allocate() throws IOException {
-        if (Phase.ERROR == phase) {
-            throw new AllocationException(Phase.ERROR, "Error on ledger allocator for " + allocatePath);
-        }
-        if (Phase.HANDED_OVER == phase) {
-            // issue an allocate request when ledger is already handed over.
-            allocateLedger();
-        }
-    }
-
-    @Override
-    public synchronized Future<LedgerHandle> tryObtain(final Transaction<Object> txn,
-                                                       final OpListener<LedgerHandle> listener) {
-        if (Phase.ERROR == phase) {
-            return Future.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,
-                    "Ledger handle is handling over to another thread : " + phase));
-        }
-        tryObtainTxn = txn;
-        tryObtainListener = listener;
-        if (null != allocatedLh) {
-            completeAllocation(allocatedLh);
-        }
-        return allocatePromise;
-    }
-
-    @Override
-    public void onCommit(Version r) {
-        confirmObtain((ZkVersion) r);
-    }
-
-    private void confirmObtain(ZkVersion zkVersion) {
-        boolean shouldAllocate = false;
-        OpListener<LedgerHandle> listenerToNotify = null;
-        LedgerHandle lhToNotify = null;
-        synchronized (this) {
-            if (Phase.HANDING_OVER == phase) {
-                setPhase(Phase.HANDED_OVER);
-                setVersion(zkVersion);
-                listenerToNotify = tryObtainListener;
-                lhToNotify = allocatedLh;
-                // reset the state
-                allocatedLh = null;
-                allocatePromise = null;
-                tryObtainTxn = null;
-                tryObtainListener = null;
-                // mark flag to issue an allocation request
-                shouldAllocate = true;
-            }
-        }
-        if (null != listenerToNotify && null != lhToNotify) {
-            // notify the listener
-            listenerToNotify.onCommit(lhToNotify);
-        }
-        if (shouldAllocate) {
-            // issue an allocation request
-            allocateLedger();
-        }
-    }
-
-    @Override
-    public void onAbort(Throwable t) {
-        OpListener<LedgerHandle> listenerToNotify;
-        synchronized (this) {
-            listenerToNotify = tryObtainListener;
-            if (t instanceof KeeperException &&
-                    ((KeeperException) t).code() == KeeperException.Code.BADVERSION) {
-                LOG.info("Set ledger allocator {} to ERROR state after hit bad version : version = {}",
-                        allocatePath, getVersion());
-                setPhase(Phase.ERROR);
-            } else {
-                if (Phase.HANDING_OVER == phase) {
-                    setPhase(Phase.ALLOCATED);
-                    tryObtainTxn = null;
-                    tryObtainListener = null;
-                }
-            }
-        }
-        if (null != listenerToNotify) {
-            listenerToNotify.onAbort(t);
-        }
-    }
-
-    private synchronized void setPhase(Phase phase) {
-        this.phase = phase;
-        LOG.info("Ledger allocator {} moved to phase {} : version = {}.",
-                new Object[] { allocatePath, phase, version });
-    }
-
-    private synchronized void allocateLedger() {
-        // make sure previous allocation is already handed over.
-        if (Phase.HANDED_OVER != phase) {
-            LOG.error("Trying allocate ledger for {} in phase {}, giving up.", allocatePath, phase);
-            return;
-        }
-        setPhase(Phase.ALLOCATING);
-        allocatePromise = new Promise<LedgerHandle>();
-        QuorumConfig quorumConfig = quorumConfigProvider.getQuorumConfig();
-        bkc.createLedger(
-                quorumConfig.getEnsembleSize(),
-                quorumConfig.getWriteQuorumSize(),
-                quorumConfig.getAckQuorumSize()
-        ).addEventListener(this);
-    }
-
-    private synchronized void completeAllocation(LedgerHandle lh) {
-        allocatedLh = lh;
-        if (null == tryObtainTxn) {
-            return;
-        }
-        org.apache.zookeeper.Op zkSetDataOp = org.apache.zookeeper.Op.setData(
-                allocatePath, DistributedLogConstants.EMPTY_BYTES, version.getZnodeVersion());
-        ZKVersionedSetOp commitOp = new ZKVersionedSetOp(zkSetDataOp, this);
-        tryObtainTxn.addOp(commitOp);
-        setPhase(Phase.HANDING_OVER);
-        FutureUtils.setValue(allocatePromise, lh);
-    }
-
-    private synchronized void failAllocation(Throwable cause) {
-        FutureUtils.setException(allocatePromise, cause);
-    }
-
-    @Override
-    public void onSuccess(LedgerHandle lh) {
-        // a ledger is created, update the ledger to allocation path before handling it over for usage.
-        markAsAllocated(lh);
-    }
-
-    @Override
-    public void onFailure(Throwable cause) {
-        LOG.error("Error creating ledger for allocating {} : ", allocatePath, cause);
-        setPhase(Phase.ERROR);
-        failAllocation(cause);
-    }
-
-    private synchronized ZkVersion getVersion() {
-        return version;
-    }
-
-    private synchronized void setVersion(ZkVersion newVersion) {
-        Version.Occurred occurred = newVersion.compare(version);
-        if (occurred == Version.Occurred.AFTER) {
-            LOG.info("Ledger allocator for {} moved version from {} to {}.",
-                    new Object[] { allocatePath, version, newVersion });
-            version = newVersion;
-        } else {
-            LOG.warn("Ledger allocator for {} received an old version {}, current version is {}.",
-                    new Object[] { allocatePath, newVersion , version });
-        }
-    }
-
-    private void markAsAllocated(final LedgerHandle lh) {
-        byte[] data = DLUtils.logSegmentId2Bytes(lh.getId());
-        Utils.zkSetData(zkc, allocatePath, data, getVersion())
-            .addEventListener(new FutureEventListener<ZkVersion>() {
-                @Override
-                public void onSuccess(ZkVersion version) {
-                    // we only issue deleting ledger left from previous allocation when we could allocate first ledger
-                    // as zookeeper version could prevent us doing stupid things.
-                    deleteLedgerLeftFromPreviousAllocationIfNecessary();
-                    setVersion(version);
-                    setPhase(Phase.ALLOCATED);
-                    // complete the allocation after it is marked as allocated
-                    completeAllocation(lh);
-                }
-
-                @Override
-                public void onFailure(Throwable cause) {
-                    setPhase(Phase.ERROR);
-                    deleteLedger(lh.getId());
-                    LOG.error("Fail mark ledger {} as allocated under {} : ",
-                            new Object[] { lh.getId(), allocatePath, cause });
-                    // fail the allocation since failed to mark it as allocated
-                    failAllocation(cause);
-                }
-            });
-    }
-
-    void deleteLedger(final long ledgerId) {
-        final Future<Void> deleteFuture = bkc.deleteLedger(ledgerId, true);
-        synchronized (ledgerDeletions) {
-            ledgerDeletions.add(deleteFuture);
-        }
-        deleteFuture.onFailure(new AbstractFunction1<Throwable, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Throwable 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;
-            }
-        });
-    }
-
-    private synchronized boolean isClosing() {
-        return closeFuture != null;
-    }
-
-    private Future<Void> closeInternal(boolean cleanup) {
-        Promise<Void> closePromise;
-        synchronized (this) {
-            if (null != closeFuture) {
-                return closeFuture;
-            }
-            closePromise = new Promise<Void>();
-            closeFuture = closePromise;
-        }
-        if (!cleanup) {
-            LOG.info("Abort ledger allocator without cleaning up on {}.", allocatePath);
-            FutureUtils.setValue(closePromise, null);
-            return closePromise;
-        }
-        cleanupAndClose(closePromise);
-        return closePromise;
-    }
-
-    private void cleanupAndClose(final Promise<Void> closePromise) {
-        LOG.info("Closing ledger allocator on {}.", allocatePath);
-        final ZKTransaction txn = new ZKTransaction(zkc);
-        // try obtain ledger handle
-        tryObtain(txn, new OpListener<LedgerHandle>() {
-            @Override
-            public void onCommit(LedgerHandle r) {
-                // no-op
-                complete();
-            }
-
-            @Override
-            public void onAbort(Throwable t) {
-                // no-op
-                complete();
-            }
-
-            private void complete() {
-                FutureUtils.setValue(closePromise, null);
-                LOG.info("Closed ledger allocator on {}.", allocatePath);
-            }
-        }).addEventListener(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;
-                synchronized (ledgerDeletions) {
-                    outstandingDeletions = Lists.newArrayList(ledgerDeletions);
-                }
-                Future.collect(outstandingDeletions).addEventListener(new FutureEventListener<List<Void>>() {
-                    @Override
-                    public void onSuccess(List<Void> values) {
-                        txn.execute();
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        LOG.debug("Fail to obtain the allocated ledger handle when closing the allocator : ", cause);
-                        FutureUtils.setValue(closePromise, null);
-                    }
-                });
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                LOG.debug("Fail to obtain the allocated ledger handle when closing the allocator : ", cause);
-                FutureUtils.setValue(closePromise, null);
-            }
-        });
-
-    }
-
-    @Override
-    public void start() {
-        // nop
-    }
-
-    @Override
-    public Future<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());
-            }
-        });
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/package-info.java
deleted file mode 100644
index 697d2e9..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/package-info.java
+++ /dev/null
@@ -1,24 +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.
- */
-/**
- * BookKeeper related util functions.
- * <p>
- * <h2>Ledger Allocator</h2>
- *
- */
-package com.twitter.distributedlog.bk;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentListener.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentListener.java
deleted file mode 100644
index 2196245..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentListener.java
+++ /dev/null
@@ -1,42 +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 com.twitter.distributedlog.callback;
-
-import com.twitter.distributedlog.LogSegmentMetadata;
-
-import java.util.List;
-
-/**
- * Listener on log segments changes for a given stream used by {@link com.twitter.distributedlog.BKLogReadHandler}
- */
-public interface LogSegmentListener {
-
-    /**
-     * Notified when <i>segments</i> updated. The new sorted log segments
-     * list is returned in this method.
-     *
-     * @param segments
-     *          updated list of segments.
-     */
-    void onSegmentsUpdated(List<LogSegmentMetadata> segments);
-
-    /**
-     * Notified when the log stream is deleted.
-     */
-    void onLogStreamDeleted();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentNamesListener.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentNamesListener.java
deleted file mode 100644
index e38f305..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentNamesListener.java
+++ /dev/null
@@ -1,42 +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 com.twitter.distributedlog.callback;
-
-import org.apache.bookkeeper.versioning.Versioned;
-
-import java.util.List;
-
-/**
- * Listener on list of log segments changes for a given stream used by
- * {@link com.twitter.distributedlog.logsegment.LogSegmentMetadataStore}.
- */
-public interface LogSegmentNamesListener {
-    /**
-     * Notified when <i>segments</i> updated. The new log segments
-     * list is returned in this method.
-     *
-     * @param segments
-     *          updated list of segments.
-     */
-    void onSegmentsUpdated(Versioned<List<String>> segments);
-
-    /**
-     * Notified when the log stream is deleted.
-     */
-    void onLogStreamDeleted();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/NamespaceListener.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/NamespaceListener.java
deleted file mode 100644
index fc63ff5..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/NamespaceListener.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 com.twitter.distributedlog.callback;
-
-import com.google.common.annotations.Beta;
-
-import java.util.Iterator;
-
-@Beta
-public interface NamespaceListener {
-
-    /**
-     * Updated with latest streams.
-     *
-     * @param streams
-     *          latest list of streams under a given namespace.
-     */
-    void onStreamsChanged(Iterator<String> streams);
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/ReadAheadCallback.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/ReadAheadCallback.java
deleted file mode 100644
index 7c46a1a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/ReadAheadCallback.java
+++ /dev/null
@@ -1,25 +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 com.twitter.distributedlog.callback;
-
-/**
- * ReadAhead Callback
- */
-public interface ReadAheadCallback {
-    void resumeReadAhead();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/package-info.java
deleted file mode 100644
index 2724d43..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Callbacks for distributedlog operations.
- */
-package com.twitter.distributedlog.callback;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentBaseConfiguration.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentBaseConfiguration.java
deleted file mode 100644
index 91603c1..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentBaseConfiguration.java
+++ /dev/null
@@ -1,76 +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 com.twitter.distributedlog.config;
-
-import com.google.common.base.Preconditions;
-
-import org.apache.commons.configuration.AbstractConfiguration;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.Iterator;
-import java.util.concurrent.ConcurrentHashMap;
-
-/**
- * Configuration view built on concurrent hash map for fast thread-safe access.
- * Notes:
- * 1. Multi-property list aggregation will not work in this class. I.e. commons config
- * normally combines all properties with the same key into one list property automatically.
- * 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;
-
-    public ConcurrentBaseConfiguration() {
-        this.map = new ConcurrentHashMap<String, Object>();
-    }
-
-    @Override
-    protected void addPropertyDirect(String key, Object value) {
-        Preconditions.checkNotNull(value);
-        map.put(key, value);
-    }
-
-    @Override
-    public Object getProperty(String key) {
-        return map.get(key);
-    }
-
-    @Override
-    public Iterator getKeys() {
-        return map.keySet().iterator();
-    }
-
-    @Override
-    public boolean containsKey(String key) {
-        return map.containsKey(key);
-    }
-
-    @Override
-    public boolean isEmpty() {
-        return map.isEmpty();
-    }
-
-    @Override
-    protected void clearPropertyDirect(String key) {
-        map.remove(key);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentConstConfiguration.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentConstConfiguration.java
deleted file mode 100644
index a044a13..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentConstConfiguration.java
+++ /dev/null
@@ -1,31 +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 com.twitter.distributedlog.config;
-
-import com.google.common.base.Preconditions;
-import org.apache.commons.configuration.Configuration;
-
-/**
- * Invariant thread-safe view of some configuration.
- */
-public class ConcurrentConstConfiguration extends ConcurrentBaseConfiguration {
-    public ConcurrentConstConfiguration(Configuration conf) {
-        Preconditions.checkNotNull(conf);
-        copy(conf);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationListener.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationListener.java
deleted file mode 100644
index d4c44b7..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationListener.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.config;
-
-/**
- * Configuration listener triggered when reloading configuration settings.
- */
-public interface ConfigurationListener {
-
-    /**
-     * Reload the configuration.
-     *
-     * @param conf configuration to reload
-     */
-    void onReload(ConcurrentBaseConfiguration conf);
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationSubscription.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationSubscription.java
deleted file mode 100644
index dadfe81..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationSubscription.java
+++ /dev/null
@@ -1,186 +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 com.twitter.distributedlog.config;
-
-import java.io.FileNotFoundException;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArraySet;
-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;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * ConfigurationSubscription publishes a reloading, thread-safe view of file configuration. The class
- * periodically calls FileConfiguration.reload on the underlying conf, and propagates changes to the
- * concurrent config. The configured FileChangedReloadingStrategy ensures that file config will only
- * be reloaded if something changed.
- * Notes:
- * 1. Reload schedule is never terminated. The assumption is a finite number of these are started
- * at the calling layer, and terminated only once the executor service is shut down.
- * 2. The underlying FileConfiguration is not at all thread-safe, so its important to ensure access
- * to this object is always single threaded.
- */
-public class ConfigurationSubscription {
-    static final Logger LOG = LoggerFactory.getLogger(ConfigurationSubscription.class);
-
-    private final ConcurrentBaseConfiguration viewConfig;
-    private final ScheduledExecutorService executorService;
-    private final int reloadPeriod;
-    private final TimeUnit reloadUnit;
-    private final List<FileConfigurationBuilder> fileConfigBuilders;
-    private final List<FileConfiguration> fileConfigs;
-    private final CopyOnWriteArraySet<ConfigurationListener> confListeners;
-
-    public ConfigurationSubscription(ConcurrentBaseConfiguration viewConfig,
-                                     List<FileConfigurationBuilder> fileConfigBuilders,
-                                     ScheduledExecutorService executorService,
-                                     int reloadPeriod,
-                                     TimeUnit reloadUnit)
-            throws ConfigurationException {
-        Preconditions.checkNotNull(fileConfigBuilders);
-        Preconditions.checkArgument(!fileConfigBuilders.isEmpty());
-        Preconditions.checkNotNull(executorService);
-        Preconditions.checkNotNull(viewConfig);
-        this.viewConfig = viewConfig;
-        this.executorService = executorService;
-        this.reloadPeriod = reloadPeriod;
-        this.reloadUnit = reloadUnit;
-        this.fileConfigBuilders = fileConfigBuilders;
-        this.fileConfigs = Lists.newArrayListWithExpectedSize(this.fileConfigBuilders.size());
-        this.confListeners = new CopyOnWriteArraySet<ConfigurationListener>();
-        reload();
-        scheduleReload();
-    }
-
-    public void registerListener(ConfigurationListener listener) {
-        this.confListeners.add(listener);
-    }
-
-    public void unregisterListener(ConfigurationListener listener) {
-        this.confListeners.remove(listener);
-    }
-
-    private boolean initConfig() {
-        if (fileConfigs.isEmpty()) {
-            try {
-                for (FileConfigurationBuilder fileConfigBuilder : fileConfigBuilders) {
-                    FileConfiguration fileConfig = fileConfigBuilder.getConfiguration();
-                    FileChangedReloadingStrategy reloadingStrategy = new FileChangedReloadingStrategy();
-                    reloadingStrategy.setRefreshDelay(0);
-                    fileConfig.setReloadingStrategy(reloadingStrategy);
-                    fileConfigs.add(fileConfig);
-                }
-            } catch (ConfigurationException ex) {
-                if (!fileNotFound(ex)) {
-                    LOG.error("Config init failed {}", ex);
-                }
-            }
-        }
-        return !fileConfigs.isEmpty();
-    }
-
-    private void scheduleReload() {
-        executorService.scheduleAtFixedRate(new Runnable() {
-            @Override
-            public void run() {
-                reload();
-            }
-        }, 0, reloadPeriod, reloadUnit);
-    }
-
-    @VisibleForTesting
-    void reload() {
-        // No-op if already loaded.
-        if (!initConfig()) {
-            return;
-        }
-        // Reload if config exists.
-        Set<String> confKeys = Sets.newHashSet();
-        for (FileConfiguration fileConfig : fileConfigs) {
-            LOG.debug("Check and reload config, file={}, lastModified={}", fileConfig.getFile(),
-                    fileConfig.getFile().lastModified());
-            fileConfig.reload();
-            // load keys
-            Iterator keyIter = fileConfig.getKeys();
-            while (keyIter.hasNext()) {
-                String key = (String) keyIter.next();
-                confKeys.add(key);
-            }
-        }
-        // clear unexisted keys
-        Iterator viewIter = viewConfig.getKeys();
-        while (viewIter.hasNext()) {
-            String key = (String) viewIter.next();
-            if (!confKeys.contains(key)) {
-                clearViewProperty(key);
-            }
-        }
-        LOG.info("Reload features : {}", confKeys);
-        // load keys from files
-        for (FileConfiguration fileConfig : fileConfigs) {
-            try {
-                loadView(fileConfig);
-            } catch (Exception ex) {
-                if (!fileNotFound(ex)) {
-                    LOG.error("Config reload failed for file {}", fileConfig.getFileName(), ex);
-                }
-            }
-        }
-        for (ConfigurationListener listener : confListeners) {
-            listener.onReload(viewConfig);
-        }
-    }
-
-    private boolean fileNotFound(Exception ex) {
-        return ex instanceof FileNotFoundException ||
-                ex.getCause() != null && ex.getCause() instanceof FileNotFoundException;
-    }
-
-    private void loadView(FileConfiguration fileConfig) {
-        Iterator fileIter = fileConfig.getKeys();
-        while (fileIter.hasNext()) {
-            String key = (String) fileIter.next();
-            setViewProperty(fileConfig, key, fileConfig.getProperty(key));
-        }
-    }
-
-    private void clearViewProperty(String key) {
-        LOG.debug("Removing property, key={}", key);
-        viewConfig.clearProperty(key);
-    }
-
-    private void setViewProperty(FileConfiguration fileConfig,
-                                 String key,
-                                 Object value) {
-        if (!viewConfig.containsKey(key) || !viewConfig.getProperty(key).equals(value)) {
-            LOG.debug("Setting property, key={} value={}", key, fileConfig.getProperty(key));
-            viewConfig.setProperty(key, fileConfig.getProperty(key));
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicConfigurationFactory.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicConfigurationFactory.java
deleted file mode 100644
index 2510f74..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicConfigurationFactory.java
+++ /dev/null
@@ -1,91 +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 com.twitter.distributedlog.config;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-
-import java.io.File;
-import java.io.FileNotFoundException;
-import java.net.MalformedURLException;
-import java.util.HashMap;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import org.apache.commons.configuration.ConfigurationException;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Encapsulates creation of DynamicDistributedLogConfiguration instances. Ensures one instance per
- * factory.
- * Notes:
- * Once loaded, stays loaded until shutdown. Caller ensures small finite number of configs are created.
- */
-public class DynamicConfigurationFactory {
-    private static final Logger LOG = LoggerFactory.getLogger(DynamicConfigurationFactory.class);
-
-    private final Map<String, DynamicDistributedLogConfiguration> dynamicConfigs;
-    private final List<ConfigurationSubscription> subscriptions;
-    private final ScheduledExecutorService executorService;
-    private final int reloadPeriod;
-    private final TimeUnit reloadUnit;
-
-    public DynamicConfigurationFactory(ScheduledExecutorService executorService, int reloadPeriod, TimeUnit reloadUnit) {
-        this.executorService = executorService;
-        this.reloadPeriod = reloadPeriod;
-        this.reloadUnit = reloadUnit;
-        this.dynamicConfigs = new HashMap<String, DynamicDistributedLogConfiguration>();
-        this.subscriptions = new LinkedList<ConfigurationSubscription>();
-    }
-
-    public synchronized Optional<DynamicDistributedLogConfiguration> getDynamicConfiguration(
-            String configPath,
-            ConcurrentBaseConfiguration defaultConf) throws ConfigurationException {
-        Preconditions.checkNotNull(configPath);
-        try {
-            if (!dynamicConfigs.containsKey(configPath)) {
-                File configFile = new File(configPath);
-                FileConfigurationBuilder properties =
-                        new PropertiesConfigurationBuilder(configFile.toURI().toURL());
-                DynamicDistributedLogConfiguration dynConf =
-                        new DynamicDistributedLogConfiguration(defaultConf);
-                List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(properties);
-                ConfigurationSubscription subscription = new ConfigurationSubscription(
-                        dynConf, fileConfigBuilders, executorService, reloadPeriod, reloadUnit);
-                subscriptions.add(subscription);
-                dynamicConfigs.put(configPath, dynConf);
-                LOG.info("Loaded dynamic configuration at {}", configPath);
-            }
-            return Optional.of(dynamicConfigs.get(configPath));
-        } catch (MalformedURLException ex) {
-            throw new ConfigurationException(ex);
-        }
-    }
-
-    public synchronized Optional<DynamicDistributedLogConfiguration> getDynamicConfiguration(String configPath) throws ConfigurationException {
-        return getDynamicConfiguration(configPath, new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicDistributedLogConfiguration.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicDistributedLogConfiguration.java
deleted file mode 100644
index ca43cfa..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicDistributedLogConfiguration.java
+++ /dev/null
@@ -1,356 +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 com.twitter.distributedlog.config;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.bk.QuorumConfig;
-
-import static com.twitter.distributedlog.DistributedLogConfiguration.*;
-
-/**
- * Whitelist dynamic configuration by adding an accessor to this class.
- */
-public class DynamicDistributedLogConfiguration extends ConcurrentBaseConfiguration {
-
-    private final ConcurrentBaseConfiguration defaultConfig;
-
-    public DynamicDistributedLogConfiguration(ConcurrentBaseConfiguration defaultConfig) {
-        this.defaultConfig = defaultConfig;
-    }
-
-    private static int getInt(ConcurrentBaseConfiguration configuration,
-                              String newKey,
-                              String oldKey,
-                              int defaultValue) {
-        return configuration.getInt(newKey, configuration.getInt(oldKey, defaultValue));
-    }
-
-    /**
-     * Get retention period in hours
-     *
-     * @return retention period in hours
-     */
-    public int getRetentionPeriodHours() {
-        return getInt(
-                this,
-                BKDL_RETENTION_PERIOD_IN_HOURS,
-                BKDL_RETENTION_PERIOD_IN_HOURS_OLD,
-                getInt(defaultConfig,
-                        BKDL_RETENTION_PERIOD_IN_HOURS,
-                        BKDL_RETENTION_PERIOD_IN_HOURS_OLD,
-                        BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT)
-        );
-    }
-
-    /**
-     * A lower threshold bytes per second limit on writes to the distributedlog proxy.
-     *
-     * @return Bytes per second write limit
-     */
-    public int getBpsSoftWriteLimit() {
-        return getInt(DistributedLogConfiguration.BKDL_BPS_SOFT_WRITE_LIMIT,
-            defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_SOFT_WRITE_LIMIT,
-                DistributedLogConfiguration.BKDL_BPS_SOFT_WRITE_LIMIT_DEFAULT));
-    }
-
-    /**
-     * An upper threshold bytes per second limit on writes to the distributedlog proxy.
-     *
-     * @return Bytes per second write limit
-     */
-    public int getBpsHardWriteLimit() {
-        return getInt(DistributedLogConfiguration.BKDL_BPS_HARD_WRITE_LIMIT,
-            defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_HARD_WRITE_LIMIT,
-                DistributedLogConfiguration.BKDL_BPS_HARD_WRITE_LIMIT_DEFAULT));
-    }
-
-    /**
-     * A lower threshold requests per second limit on writes to the distributedlog proxy.
-     *
-     * @return Requests per second write limit
-     */
-    public int getRpsSoftWriteLimit() {
-        return getInt(DistributedLogConfiguration.BKDL_RPS_SOFT_WRITE_LIMIT,
-            defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_SOFT_WRITE_LIMIT,
-                DistributedLogConfiguration.BKDL_RPS_SOFT_WRITE_LIMIT_DEFAULT));
-    }
-
-    /**
-     * An upper threshold requests per second limit on writes to the distributedlog proxy.
-     *
-     * @return Requests per second write limit
-     */
-    public int getRpsHardWriteLimit() {
-        return getInt(DistributedLogConfiguration.BKDL_RPS_HARD_WRITE_LIMIT,
-            defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_HARD_WRITE_LIMIT,
-                DistributedLogConfiguration.BKDL_RPS_HARD_WRITE_LIMIT_DEFAULT));
-    }
-
-    /**
-     * A lower threshold requests per second limit on writes to the distributedlog proxy globally.
-     *
-     * @return Requests per second write limit
-     */
-    public int getRpsSoftServiceLimit() {
-        return getInt(DistributedLogConfiguration.BKDL_RPS_SOFT_SERVICE_LIMIT,
-            defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_SOFT_SERVICE_LIMIT,
-                DistributedLogConfiguration.BKDL_RPS_SOFT_SERVICE_LIMIT_DEFAULT));
-    }
-
-    /**
-     * An upper threshold requests per second limit on writes to the distributedlog proxy globally.
-     *
-     * @return Requests per second write limit
-     */
-    public int getRpsHardServiceLimit() {
-        return getInt(DistributedLogConfiguration.BKDL_RPS_HARD_SERVICE_LIMIT,
-            defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_HARD_SERVICE_LIMIT,
-                DistributedLogConfiguration.BKDL_RPS_HARD_SERVICE_LIMIT_DEFAULT));
-    }
-
-    /**
-     * When 60min average rps for the entire service instance hits this value, new streams will be
-     * rejected.
-     *
-     * @return Requests per second limit
-     */
-    public int getRpsStreamAcquireServiceLimit() {
-        return getInt(DistributedLogConfiguration.BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT,
-            defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT,
-                DistributedLogConfiguration.BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT));
-    }
-
-    /**
-     * A lower threshold bytes per second limit on writes to the distributedlog proxy globally.
-     *
-     * @return Bytes per second write limit
-     */
-    public int getBpsSoftServiceLimit() {
-        return getInt(DistributedLogConfiguration.BKDL_BPS_SOFT_SERVICE_LIMIT,
-            defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_SOFT_SERVICE_LIMIT,
-                DistributedLogConfiguration.BKDL_BPS_SOFT_SERVICE_LIMIT_DEFAULT));
-    }
-
-    /**
-     * An upper threshold bytes per second limit on writes to the distributedlog proxy globally.
-     *
-     * @return Bytes per second write limit
-     */
-    public int getBpsHardServiceLimit() {
-        return getInt(DistributedLogConfiguration.BKDL_BPS_HARD_SERVICE_LIMIT,
-            defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_HARD_SERVICE_LIMIT,
-                DistributedLogConfiguration.BKDL_BPS_HARD_SERVICE_LIMIT_DEFAULT));
-    }
-
-    /**
-     * When 60min average bps for the entire service instance hits this value, new streams will be
-     * rejected.
-     *
-     * @return Bytes per second limit
-     */
-    public int getBpsStreamAcquireServiceLimit() {
-        return getInt(DistributedLogConfiguration.BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT,
-            defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT,
-                DistributedLogConfiguration.BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT));
-    }
-
-    /**
-     * Get percent of write bytes which should be delayed by BKDL_EI_INJECTED_WRITE_DELAY_MS.
-     *
-     * @return percent of writes to delay.
-     */
-    public double getEIInjectedWriteDelayPercent() {
-        return getDouble(DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_PERCENT,
-            defaultConfig.getDouble(DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_PERCENT,
-                DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_PERCENT_DEFAULT));
-    }
-
-    /**
-     * Get amount of time to delay writes for in writer failure injection.
-     *
-     * @return millis to delay writes for.
-     */
-    public int getEIInjectedWriteDelayMs() {
-        return getInt(DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_MS,
-            defaultConfig.getInt(DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_MS,
-                DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_MS_DEFAULT));
-    }
-
-    /**
-     * Get output buffer size
-     *
-     * @return buffer size
-     */
-    public int getOutputBufferSize() {
-        return getInt(
-                this,
-                BKDL_OUTPUT_BUFFER_SIZE,
-                BKDL_OUTPUT_BUFFER_SIZE_OLD,
-                getInt(defaultConfig,
-                        BKDL_OUTPUT_BUFFER_SIZE,
-                        BKDL_OUTPUT_BUFFER_SIZE_OLD,
-                        BKDL_OUTPUT_BUFFER_SIZE_DEFAULT)
-        );
-    }
-
-    /**
-     * Get Periodic Log Flush Frequency in seconds
-     *
-     * @return periodic flush frequency
-     */
-    public int getPeriodicFlushFrequencyMilliSeconds() {
-        return getInt(DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
-            defaultConfig.getInt(DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
-                DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT));
-    }
-
-    /**
-     * Get the number of entries that readahead worker reads as a batch from bookkeeper
-     *
-     * @return the batch size
-     */
-    public int getReadAheadBatchSize() {
-        return getInt(
-                this,
-                BKDL_READAHEAD_BATCHSIZE,
-                BKDL_READAHEAD_BATCHSIZE_OLD,
-                getInt(defaultConfig,
-                        BKDL_READAHEAD_BATCHSIZE,
-                        BKDL_READAHEAD_BATCHSIZE_OLD,
-                        BKDL_READAHEAD_BATCHSIZE_DEFAULT)
-        );
-    }
-
-    /**
-     * Get the maximum number of {@link com.twitter.distributedlog.LogRecord } that readahead worker will cache.
-     *
-     * @return the maximum number
-     */
-    public int getReadAheadMaxRecords() {
-        return getInt(
-                this,
-                BKDL_READAHEAD_MAX_RECORDS,
-                BKDL_READAHEAD_MAX_RECORDS_OLD,
-                getInt(defaultConfig,
-                        BKDL_READAHEAD_MAX_RECORDS,
-                        BKDL_READAHEAD_MAX_RECORDS_OLD,
-                        BKDL_READAHEAD_MAX_RECORDS_DEFAULT)
-        );
-    }
-
-    /**
-     * Whether to enable ledger allocator pool or not.
-     * It is disabled by default.
-     *
-     * @return whether using ledger allocator pool or not.
-     */
-    public boolean getEnableLedgerAllocatorPool() {
-        return getBoolean(BKDL_ENABLE_LEDGER_ALLOCATOR_POOL,
-                defaultConfig.getBoolean(
-                        BKDL_ENABLE_LEDGER_ALLOCATOR_POOL,
-                        BKDL_ENABLE_LEDGER_ALLOCATOR_POOL_DEFAULT));
-    }
-
-    /**
-     * Get the quorum config.
-     *
-     * @return quorum config
-     */
-    public QuorumConfig getQuorumConfig() {
-        int ensembleSize = getInt(
-                this,
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE,
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD,
-                getInt(defaultConfig,
-                        BKDL_BOOKKEEPER_ENSEMBLE_SIZE,
-                        BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD,
-                        BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT));
-        int writeQuorumSize = getInt(
-                this,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
-                getInt(defaultConfig,
-                        BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
-                        BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
-                        BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT));
-        int ackQuorumSize = getInt(
-                this,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
-                getInt(defaultConfig,
-                        BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
-                        BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
-                        BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT));
-        return new QuorumConfig(ensembleSize, writeQuorumSize, ackQuorumSize);
-    }
-
-    /**
-     * Get the maximum number of partitions of each stream allowed to be acquired per proxy.
-     *
-     * @return maximum number of partitions of each stream allowed to be acquired
-     * @see DistributedLogConfiguration#getMaxAcquiredPartitionsPerProxy()
-     */
-    public int getMaxAcquiredPartitionsPerProxy() {
-        return getInt(
-                BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY,
-                defaultConfig.getInt(
-                        BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY,
-                        BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY_DEFAULT)
-        );
-    }
-
-    /**
-     * Get the maximum number of partitions of each stream allowed to cache per proxy.
-     *
-     * @return maximum number of partitions of each stream allowed to cache
-     * @see DistributedLogConfiguration#getMaxAcquiredPartitionsPerProxy()
-     */
-    public int getMaxCachedPartitionsPerProxy() {
-        return getInt(
-                BKDL_MAX_CACHED_PARTITIONS_PER_PROXY,
-                defaultConfig.getInt(
-                        BKDL_MAX_CACHED_PARTITIONS_PER_PROXY,
-                        BKDL_MAX_CACHED_PARTITIONS_PER_PROXY_DEFAULT)
-        );
-    }
-
-    /**
-     * Check whether the durable write is enabled.
-     *
-     * @return true if durable write is enabled. otherwise, false.
-     */
-    public boolean isDurableWriteEnabled() {
-        return getBoolean(BKDL_IS_DURABLE_WRITE_ENABLED,
-                defaultConfig.getBoolean(
-                        BKDL_IS_DURABLE_WRITE_ENABLED,
-                        BKDL_IS_DURABLE_WRITE_ENABLED_DEFAULT));
-    }
-
-    /**
-     * Get the flag whether to deserialize recordset on reads.
-     *
-     * @return flag whether to deserialize recordset on reads.
-     */
-    public boolean getDeserializeRecordSetOnReads() {
-        return getBoolean(BKDL_DESERIALIZE_RECORDSET_ON_READS,
-                defaultConfig.getBoolean(
-                        BKDL_DESERIALIZE_RECORDSET_ON_READS,
-                        BKDL_DESERIALIZE_RECORDSET_ON_READS_DEFAULT));
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/FileConfigurationBuilder.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/FileConfigurationBuilder.java
deleted file mode 100644
index b3c4e6c..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/FileConfigurationBuilder.java
+++ /dev/null
@@ -1,28 +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 com.twitter.distributedlog.config;
-
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.FileConfiguration;
-
-/**
- * Abstract out FileConfiguration subclass construction.
- */
-public interface FileConfigurationBuilder {
-    FileConfiguration getConfiguration() throws ConfigurationException;
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/PropertiesConfigurationBuilder.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/PropertiesConfigurationBuilder.java
deleted file mode 100644
index 6efaa20..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/PropertiesConfigurationBuilder.java
+++ /dev/null
@@ -1,40 +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 com.twitter.distributedlog.config;
-
-import java.net.URL;
-
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.FileConfiguration;
-import org.apache.commons.configuration.PropertiesConfiguration;
-
-/**
- * Hide PropertiesConfiguration dependency.
- */
-public class PropertiesConfigurationBuilder implements FileConfigurationBuilder {
-    private URL url;
-
-    public PropertiesConfigurationBuilder(URL url) {
-        this.url = url;
-    }
-
-    @Override
-    public FileConfiguration getConfiguration() throws ConfigurationException {
-        return new PropertiesConfiguration(url);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/config/package-info.java
deleted file mode 100644
index b4f77b4..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/config/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Configuration
- */
-package com.twitter.distributedlog.config;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/exceptions/ZKException.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/exceptions/ZKException.java
deleted file mode 100644
index 8ed1610..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/exceptions/ZKException.java
+++ /dev/null
@@ -1,54 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.KeeperException.Code;
-
-/**
- * TODO: move ZKException to distributedlog-protocol
- */
-public class ZKException extends DLException {
-
-    private static final long serialVersionUID = 7542748595054923600L;
-
-    final KeeperException.Code code;
-
-    public ZKException(String msg, Code code) {
-        super(StatusCode.ZOOKEEPER_ERROR, msg + " : " + code);
-        this.code = code;
-    }
-
-    public ZKException(String msg, KeeperException exception) {
-        super(StatusCode.ZOOKEEPER_ERROR, msg, exception);
-        this.code = exception.code();
-    }
-
-    public Code getKeeperExceptionCode() {
-        return this.code;
-    }
-
-    public static boolean isRetryableZKException(ZKException zke) {
-        KeeperException.Code code = zke.getKeeperExceptionCode();
-        return KeeperException.Code.CONNECTIONLOSS == code ||
-                KeeperException.Code.OPERATIONTIMEOUT == code ||
-                KeeperException.Code.SESSIONEXPIRED == code ||
-                KeeperException.Code.SESSIONMOVED == code;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/AbstractFeatureProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/AbstractFeatureProvider.java
deleted file mode 100644
index f484307..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/AbstractFeatureProvider.java
+++ /dev/null
@@ -1,103 +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 com.twitter.distributedlog.feature;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import org.apache.bookkeeper.feature.CacheableFeatureProvider;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.commons.configuration.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-
-/**
- * Decider based feature provider
- */
-public abstract class AbstractFeatureProvider<T extends Feature> extends CacheableFeatureProvider<T> {
-
-    protected static final Logger logger = LoggerFactory.getLogger(AbstractFeatureProvider.class);
-
-    public static FeatureProvider getFeatureProvider(String rootScope,
-                                                     DistributedLogConfiguration conf,
-                                                     StatsLogger statsLogger)
-            throws IOException {
-        Class<? extends FeatureProvider> featureProviderClass;
-        try {
-            featureProviderClass = conf.getFeatureProviderClass();
-        } catch (ConfigurationException e) {
-            throw new IOException("Can't initialize the feature provider : ", e);
-        }
-        // create feature provider
-        Constructor<? extends FeatureProvider> constructor;
-        try {
-            constructor = featureProviderClass.getDeclaredConstructor(
-                    String.class,
-                    DistributedLogConfiguration.class,
-                    StatsLogger.class);
-        } catch (NoSuchMethodException e) {
-            throw new IOException("No constructor found for feature provider class " + featureProviderClass + " : ", e);
-        }
-        try {
-            return constructor.newInstance(rootScope, conf, statsLogger);
-        } catch (InstantiationException e) {
-            throw new IOException("Failed to instantiate feature provider : ", e);
-        } catch (IllegalAccessException e) {
-            throw new IOException("Encountered illegal access when instantiating feature provider : ", e);
-        } catch (InvocationTargetException e) {
-            Throwable targetException = e.getTargetException();
-            if (targetException instanceof IOException) {
-                throw (IOException) targetException;
-            } else {
-                throw new IOException("Encountered invocation target exception while instantiating feature provider : ", e);
-            }
-        }
-    }
-
-    protected final DistributedLogConfiguration conf;
-    protected final StatsLogger statsLogger;
-
-    protected AbstractFeatureProvider(String rootScope,
-                                      DistributedLogConfiguration conf,
-                                      StatsLogger statsLogger) {
-        super(rootScope);
-        this.conf = conf;
-        this.statsLogger = statsLogger;
-    }
-
-    /**
-     * Start the feature provider.
-     *
-     * @throws IOException when failed to start the feature provider.
-     */
-    public void start() throws IOException {
-        // no-op
-    }
-
-    /**
-     * Stop the feature provider.
-     */
-    public void stop() {
-        // no-op
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/ConfigurationFeatureProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/ConfigurationFeatureProvider.java
deleted file mode 100644
index 02a4d79..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/ConfigurationFeatureProvider.java
+++ /dev/null
@@ -1,76 +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 com.twitter.distributedlog.feature;
-
-import com.twitter.distributedlog.config.ConcurrentBaseConfiguration;
-import org.apache.bookkeeper.feature.CacheableFeatureProvider;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Feature Provider that load features from configuration
- */
-class ConfigurationFeatureProvider extends CacheableFeatureProvider {
-
-    private static final Logger logger = LoggerFactory.getLogger(ConfigurationFeatureProvider.class);
-
-    static SettableFeature makeFeature(ConcurrentBaseConfiguration featuresConf,
-                                       ConcurrentMap<String, SettableFeature> features,
-                                       String featureName) {
-        SettableFeature feature = features.get(featureName);
-        if (null == feature) {
-            int availability = featuresConf.getInt(featureName, 0);
-            feature = new SettableFeature(featureName, availability);
-            SettableFeature oldFeature =
-                    features.putIfAbsent(featureName, feature);
-            if (null != oldFeature) {
-                feature = oldFeature;
-            } else {
-                logger.info("Load feature {}={}", featureName, availability);
-            }
-        }
-        return feature;
-    }
-
-    private final ConcurrentBaseConfiguration featuresConf;
-    private final ConcurrentMap<String, SettableFeature> features;
-
-    ConfigurationFeatureProvider(String rootScope,
-                                 ConcurrentBaseConfiguration featuresConf,
-                                 ConcurrentMap<String, SettableFeature> features) {
-        super(rootScope);
-        this.featuresConf = featuresConf;
-        this.features = features;
-    }
-
-    @Override
-    protected Feature makeFeature(String featureName) {
-        return makeFeature(featuresConf, features, featureName);
-    }
-
-    @Override
-    protected FeatureProvider makeProvider(String fullScopeName) {
-        return new ConfigurationFeatureProvider(
-                fullScopeName, featuresConf, features);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/CoreFeatureKeys.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/CoreFeatureKeys.java
deleted file mode 100644
index 49b3354..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/CoreFeatureKeys.java
+++ /dev/null
@@ -1,29 +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 com.twitter.distributedlog.feature;
-
-/**
- * List of feature keys used by distributedlog core
- */
-public enum CoreFeatureKeys {
-    // @Deprecated: bkc features are managed by bookkeeper prefixed with a scope
-    DISABLE_DURABILITY_ENFORCEMENT,
-    // disabling logsegment rolling
-    DISABLE_LOGSEGMENT_ROLLING,
-    DISABLE_WRITE_LIMIT,
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DefaultFeatureProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DefaultFeatureProvider.java
deleted file mode 100644
index 6554eaa..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DefaultFeatureProvider.java
+++ /dev/null
@@ -1,47 +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 com.twitter.distributedlog.feature;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.apache.bookkeeper.feature.SettableFeatureProvider;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Default feature provider which disable all features by default.
- */
-public class DefaultFeatureProvider extends AbstractFeatureProvider {
-
-    public DefaultFeatureProvider(String rootScope,
-                                  DistributedLogConfiguration conf,
-                                  StatsLogger statsLogger) {
-        super(rootScope, conf, statsLogger);
-    }
-
-    @Override
-    protected Feature makeFeature(String featureName) {
-        return new SettableFeature(featureName, 0);
-    }
-
-    @Override
-    protected FeatureProvider makeProvider(String fullScopeName) {
-        return new SettableFeatureProvider(fullScopeName, 0);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DynamicConfigurationFeatureProvider.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DynamicConfigurationFeatureProvider.java
deleted file mode 100644
index 1eeb155..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DynamicConfigurationFeatureProvider.java
+++ /dev/null
@@ -1,132 +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 com.twitter.distributedlog.feature;
-
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.config.ConcurrentBaseConfiguration;
-import com.twitter.distributedlog.config.ConfigurationListener;
-import com.twitter.distributedlog.config.ConfigurationSubscription;
-import com.twitter.distributedlog.config.FileConfigurationBuilder;
-import com.twitter.distributedlog.config.PropertiesConfigurationBuilder;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.commons.configuration.ConfigurationException;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Feature Provider based dynamic configuration.
- */
-public class DynamicConfigurationFeatureProvider extends AbstractFeatureProvider
-        implements ConfigurationListener {
-
-    private final ConcurrentBaseConfiguration featuresConf;
-    private ConfigurationSubscription featuresConfSubscription;
-    private final ConcurrentMap<String, SettableFeature> features;
-    private final ScheduledExecutorService executorService;
-
-    public DynamicConfigurationFeatureProvider(String rootScope,
-                                               DistributedLogConfiguration conf,
-                                               StatsLogger statsLogger) {
-        super(rootScope, conf, statsLogger);
-        this.features = new ConcurrentHashMap<String, SettableFeature>();
-        this.featuresConf = new ConcurrentBaseConfiguration();
-        this.executorService = Executors.newSingleThreadScheduledExecutor(
-                new ThreadFactoryBuilder().setNameFormat("DynamicConfigurationFeatureProvider-%d").build());
-    }
-
-    ConcurrentBaseConfiguration getFeatureConf() {
-        return featuresConf;
-    }
-
-    ConfigurationSubscription getFeatureConfSubscription() {
-        return featuresConfSubscription;
-    }
-
-    @Override
-    public void start() throws IOException {
-        List<FileConfigurationBuilder> fileConfigBuilders =
-                Lists.newArrayListWithExpectedSize(2);
-        String baseConfigPath = conf.getFileFeatureProviderBaseConfigPath();
-        Preconditions.checkNotNull(baseConfigPath);
-        File baseConfigFile = new File(baseConfigPath);
-        FileConfigurationBuilder baseProperties =
-                new PropertiesConfigurationBuilder(baseConfigFile.toURI().toURL());
-        fileConfigBuilders.add(baseProperties);
-        String overlayConfigPath = conf.getFileFeatureProviderOverlayConfigPath();
-        if (null != overlayConfigPath) {
-            File overlayConfigFile = new File(overlayConfigPath);
-            FileConfigurationBuilder overlayProperties =
-                    new PropertiesConfigurationBuilder(overlayConfigFile.toURI().toURL());
-            fileConfigBuilders.add(overlayProperties);
-        }
-        try {
-            this.featuresConfSubscription = new ConfigurationSubscription(
-                    this.featuresConf,
-                    fileConfigBuilders,
-                    executorService,
-                    conf.getDynamicConfigReloadIntervalSec(),
-                    TimeUnit.SECONDS);
-        } catch (ConfigurationException e) {
-            throw new IOException("Failed to register subscription on features configuration");
-        }
-        this.featuresConfSubscription.registerListener(this);
-    }
-
-    @Override
-    public void stop() {
-        this.executorService.shutdown();
-    }
-
-    @Override
-    public void onReload(ConcurrentBaseConfiguration conf) {
-        for (Map.Entry<String, SettableFeature> feature : features.entrySet()) {
-            String featureName = feature.getKey();
-            int availability = conf.getInt(featureName, 0);
-            if (availability != feature.getValue().availability()) {
-                feature.getValue().set(availability);
-                logger.info("Reload feature {}={}", featureName, availability);
-            }
-        }
-    }
-
-    @Override
-    protected Feature makeFeature(String featureName) {
-        return ConfigurationFeatureProvider.makeFeature(
-                featuresConf, features, featureName);
-    }
-
-    @Override
-    protected FeatureProvider makeProvider(String fullScopeName) {
-        return new ConfigurationFeatureProvider(
-                fullScopeName, featuresConf, features);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/package-info.java
deleted file mode 100644
index e8d8134..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Feature & FeatureProvider for distributedlog
- */
-package com.twitter.distributedlog.feature;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/CloseAsyncCloseableFunction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/function/CloseAsyncCloseableFunction.java
deleted file mode 100644
index 698a088..0000000
--- a/distributedlog-core/src/main/java/com/twitter/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 com.twitter.distributedlog.function;
-
-import com.twitter.distributedlog.io.AsyncCloseable;
-import scala.Function0;
-import scala.runtime.AbstractFunction0;
-import scala.runtime.BoxedUnit;
-
-/**
- * Function to close {@link com.twitter.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/com/twitter/distributedlog/function/DefaultValueMapFunction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/function/DefaultValueMapFunction.java
deleted file mode 100644
index f08cd0f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/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 com.twitter.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/com/twitter/distributedlog/function/GetLastTxIdFunction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetLastTxIdFunction.java
deleted file mode 100644
index bc77d6a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetLastTxIdFunction.java
+++ /dev/null
@@ -1,43 +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 com.twitter.distributedlog.function;
-
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.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 static final GetLastTxIdFunction INSTANCE = new GetLastTxIdFunction();
-
-    private GetLastTxIdFunction() {}
-
-    @Override
-    public Long apply(List<LogSegmentMetadata> segmentList) {
-        long lastTxId = DistributedLogConstants.INVALID_TXID;
-        for (LogSegmentMetadata l : segmentList) {
-            lastTxId = Math.max(lastTxId, l.getLastTxId());
-        }
-        return lastTxId;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetVersionedValueFunction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetVersionedValueFunction.java
deleted file mode 100644
index 4e7844c..0000000
--- a/distributedlog-core/src/main/java/com/twitter/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 com.twitter.distributedlog.function;
-
-import com.twitter.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/com/twitter/distributedlog/function/VoidFunctions.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/function/VoidFunctions.java
deleted file mode 100644
index e260482..0000000
--- a/distributedlog-core/src/main/java/com/twitter/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 com.twitter.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/com/twitter/distributedlog/function/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/function/package-info.java
deleted file mode 100644
index 2da98dc..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/function/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Common Functions for DistributedLog
- */
-package com.twitter.distributedlog.function;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKNamespaceDriver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKNamespaceDriver.java
deleted file mode 100644
index 5921233..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKNamespaceDriver.java
+++ /dev/null
@@ -1,631 +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 com.twitter.distributedlog.impl;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.BookKeeperClientBuilder;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.MetadataAccessor;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.acl.DefaultAccessControlManager;
-import com.twitter.distributedlog.impl.acl.ZKAccessControlManager;
-import com.twitter.distributedlog.bk.LedgerAllocator;
-import com.twitter.distributedlog.bk.LedgerAllocatorUtils;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.AlreadyClosedException;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-import com.twitter.distributedlog.impl.federated.FederatedZKLogMetadataStore;
-import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
-import com.twitter.distributedlog.impl.metadata.ZKLogStreamMetadataStore;
-import com.twitter.distributedlog.impl.subscription.ZKSubscriptionsStore;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.LogMetadataForReader;
-import com.twitter.distributedlog.metadata.LogMetadataStore;
-import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
-import com.twitter.distributedlog.namespace.NamespaceDriver;
-import com.twitter.distributedlog.namespace.NamespaceDriverManager;
-import com.twitter.distributedlog.subscription.SubscriptionsStore;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Utils;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.common.PathUtils;
-import org.apache.zookeeper.data.Stat;
-import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
-import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static com.twitter.distributedlog.util.DLUtils.isReservedStreamName;
-import static com.twitter.distributedlog.util.DLUtils.validateName;
-
-/**
- * Manager for ZooKeeper/BookKeeper based namespace
- */
-public class BKNamespaceDriver implements NamespaceDriver {
-
-    private static Logger LOG = LoggerFactory.getLogger(BKNamespaceDriver.class);
-
-    // register itself
-    static {
-        NamespaceDriverManager.registerDriver(DistributedLogConstants.BACKEND_BK, BKNamespaceDriver.class);
-    }
-
-    /**
-     * Extract zk servers fro dl <i>namespace</i>.
-     *
-     * @param uri dl namespace
-     * @return zk servers
-     */
-    public static String getZKServersFromDLUri(URI uri) {
-        return uri.getAuthority().replace(";", ",");
-    }
-
-    // resources (passed from initialization)
-    private DistributedLogConfiguration conf;
-    private DynamicDistributedLogConfiguration dynConf;
-    private URI namespace;
-    private OrderedScheduler scheduler;
-    private FeatureProvider featureProvider;
-    private AsyncFailureInjector failureInjector;
-    private StatsLogger statsLogger;
-    private StatsLogger perLogStatsLogger;
-    private String clientId;
-    private int regionId;
-
-    //
-    // resources (created internally and initialized at #initialize())
-    //
-
-    // namespace binding
-    private BKDLConfig bkdlConfig;
-
-    // zookeeper clients
-    // NOTE: The actual zookeeper client is initialized lazily when it is referenced by
-    //       {@link com.twitter.distributedlog.ZooKeeperClient#get()}. So it is safe to
-    //       keep builders and their client wrappers here, as they will be used when
-    //       instantiating readers or writers.
-    private ZooKeeperClientBuilder sharedWriterZKCBuilder;
-    private ZooKeeperClient writerZKC;
-    private ZooKeeperClientBuilder sharedReaderZKCBuilder;
-    private ZooKeeperClient readerZKC;
-    // NOTE: The actual bookkeeper client is initialized lazily when it is referenced by
-    //       {@link com.twitter.distributedlog.BookKeeperClient#get()}. So it is safe to
-    //       keep builders and their client wrappers here, as they will be used when
-    //       instantiating readers or writers.
-    private ClientSocketChannelFactory channelFactory;
-    private HashedWheelTimer requestTimer;
-    private BookKeeperClientBuilder sharedWriterBKCBuilder;
-    private BookKeeperClient writerBKC;
-    private BookKeeperClientBuilder sharedReaderBKCBuilder;
-    private BookKeeperClient readerBKC;
-
-    // log stream metadata store
-    private LogMetadataStore metadataStore;
-    private LogStreamMetadataStore writerStreamMetadataStore;
-    private LogStreamMetadataStore readerStreamMetadataStore;
-
-    //
-    // resources (lazily initialized)
-    //
-
-    // ledger allocator
-    private LedgerAllocator allocator;
-
-    // log segment entry stores
-    private LogSegmentEntryStore writerEntryStore;
-    private LogSegmentEntryStore readerEntryStore;
-
-    // access control manager
-    private AccessControlManager accessControlManager;
-
-    //
-    // states
-    //
-    protected boolean initialized = false;
-    protected AtomicBoolean closed = new AtomicBoolean(false);
-
-    /**
-     * Public constructor for reflection.
-     */
-    public BKNamespaceDriver() {
-    }
-
-    @Override
-    public synchronized NamespaceDriver initialize(DistributedLogConfiguration conf,
-                                                   DynamicDistributedLogConfiguration dynConf,
-                                                   URI namespace,
-                                                   OrderedScheduler scheduler,
-                                                   FeatureProvider featureProvider,
-                                                   AsyncFailureInjector failureInjector,
-                                                   StatsLogger statsLogger,
-                                                   StatsLogger perLogStatsLogger,
-                                                   String clientId,
-                                                   int regionId) throws IOException {
-        if (initialized) {
-            return this;
-        }
-        // validate the namespace
-        if ((null == namespace) || (null == namespace.getAuthority()) || (null == namespace.getPath())) {
-            throw new IOException("Incorrect distributedlog namespace : " + namespace);
-        }
-
-        // initialize the resources
-        this.conf = conf;
-        this.dynConf = dynConf;
-        this.namespace = namespace;
-        this.scheduler = scheduler;
-        this.featureProvider = featureProvider;
-        this.failureInjector = failureInjector;
-        this.statsLogger = statsLogger;
-        this.perLogStatsLogger = perLogStatsLogger;
-        this.clientId = clientId;
-        this.regionId = regionId;
-
-        // initialize the zookeeper clients
-        initializeZooKeeperClients();
-
-        // initialize the bookkeeper clients
-        initializeBookKeeperClients();
-
-        // propagate bkdlConfig to configuration
-        BKDLConfig.propagateConfiguration(bkdlConfig, conf);
-
-        // initialize the log metadata & stream metadata store
-        initializeLogStreamMetadataStores();
-
-        // initialize other resources
-        initializeOtherResources();
-
-        initialized = true;
-
-        LOG.info("Initialized BK namespace driver: clientId = {}, regionId = {}, federated = {}.",
-                new Object[]{clientId, regionId, bkdlConfig.isFederatedNamespace()});
-        return this;
-    }
-
-    private void initializeZooKeeperClients() throws IOException {
-        // Build the namespace zookeeper client
-        this.sharedWriterZKCBuilder = createZKClientBuilder(
-                String.format("dlzk:%s:factory_writer_shared", namespace),
-                conf,
-                getZKServersFromDLUri(namespace),
-                statsLogger.scope("dlzk_factory_writer_shared"));
-        this.writerZKC = sharedWriterZKCBuilder.build();
-
-        // Resolve namespace binding
-        this.bkdlConfig = BKDLConfig.resolveDLConfig(writerZKC, namespace);
-
-        // Build zookeeper client for readers
-        if (bkdlConfig.getDlZkServersForWriter().equals(bkdlConfig.getDlZkServersForReader())) {
-            this.sharedReaderZKCBuilder = this.sharedWriterZKCBuilder;
-        } else {
-            this.sharedReaderZKCBuilder = createZKClientBuilder(
-                    String.format("dlzk:%s:factory_reader_shared", namespace),
-                    conf,
-                    bkdlConfig.getDlZkServersForReader(),
-                    statsLogger.scope("dlzk_factory_reader_shared"));
-        }
-        this.readerZKC = this.sharedReaderZKCBuilder.build();
-    }
-
-    private synchronized BKDLConfig getBkdlConfig() {
-        return bkdlConfig;
-    }
-
-    private void initializeBookKeeperClients() throws IOException {
-        this.channelFactory = new NioClientSocketChannelFactory(
-                Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat("DL-netty-boss-%d").build()),
-                Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat("DL-netty-worker-%d").build()),
-                conf.getBKClientNumberIOThreads());
-        this.requestTimer = new HashedWheelTimer(
-                new ThreadFactoryBuilder().setNameFormat("DLFactoryTimer-%d").build(),
-                conf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS,
-                conf.getTimeoutTimerNumTicks());
-        // Build bookkeeper client for writers
-        this.sharedWriterBKCBuilder = createBKCBuilder(
-                String.format("bk:%s:factory_writer_shared", namespace),
-                conf,
-                bkdlConfig.getBkZkServersForWriter(),
-                bkdlConfig.getBkLedgersPath(),
-                channelFactory,
-                requestTimer,
-                Optional.of(featureProvider.scope("bkc")),
-                statsLogger);
-        this.writerBKC = this.sharedWriterBKCBuilder.build();
-
-        // Build bookkeeper client for readers
-        if (bkdlConfig.getBkZkServersForWriter().equals(bkdlConfig.getBkZkServersForReader())) {
-            this.sharedReaderBKCBuilder = this.sharedWriterBKCBuilder;
-        } else {
-            this.sharedReaderBKCBuilder = createBKCBuilder(
-                    String.format("bk:%s:factory_reader_shared", namespace),
-                    conf,
-                    bkdlConfig.getBkZkServersForReader(),
-                    bkdlConfig.getBkLedgersPath(),
-                    channelFactory,
-                    requestTimer,
-                    Optional.<FeatureProvider>absent(),
-                    statsLogger);
-        }
-        this.readerBKC = this.sharedReaderBKCBuilder.build();
-    }
-
-    private void initializeLogStreamMetadataStores() throws IOException {
-        // log metadata store
-        if (bkdlConfig.isFederatedNamespace() || conf.isFederatedNamespaceEnabled()) {
-            this.metadataStore = new FederatedZKLogMetadataStore(conf, namespace, readerZKC, scheduler);
-        } else {
-            this.metadataStore = new ZKLogMetadataStore(conf, namespace, readerZKC, scheduler);
-        }
-
-        // create log stream metadata store
-        this.writerStreamMetadataStore =
-                new ZKLogStreamMetadataStore(
-                        clientId,
-                        conf,
-                        writerZKC,
-                        scheduler,
-                        statsLogger);
-        this.readerStreamMetadataStore =
-                new ZKLogStreamMetadataStore(
-                        clientId,
-                        conf,
-                        readerZKC,
-                        scheduler,
-                        statsLogger);
-    }
-
-    @VisibleForTesting
-    public static String validateAndGetFullLedgerAllocatorPoolPath(DistributedLogConfiguration conf, URI uri) throws IOException {
-        String poolPath = conf.getLedgerAllocatorPoolPath();
-        LOG.info("PoolPath is {}", poolPath);
-        if (null == poolPath || !poolPath.startsWith(".") || poolPath.endsWith("/")) {
-            LOG.error("Invalid ledger allocator pool path specified when enabling ledger allocator pool : {}", poolPath);
-            throw new IOException("Invalid ledger allocator pool path specified : " + poolPath);
-        }
-        String poolName = conf.getLedgerAllocatorPoolName();
-        if (null == poolName) {
-            LOG.error("No ledger allocator pool name specified when enabling ledger allocator pool.");
-            throw new IOException("No ledger allocator name specified when enabling ledger allocator pool.");
-        }
-        String rootPath = uri.getPath() + "/" + poolPath + "/" + poolName;
-        try {
-            PathUtils.validatePath(rootPath);
-        } catch (IllegalArgumentException iae) {
-            LOG.error("Invalid ledger allocator pool path specified when enabling ledger allocator pool : {}", poolPath);
-            throw new IOException("Invalid ledger allocator pool path specified : " + poolPath);
-        }
-        return rootPath;
-    }
-
-    private void initializeOtherResources() throws IOException {
-        // Ledger allocator
-        if (conf.getEnableLedgerAllocatorPool()) {
-            String allocatorPoolPath = validateAndGetFullLedgerAllocatorPoolPath(conf, namespace);
-            allocator = LedgerAllocatorUtils.createLedgerAllocatorPool(
-                    allocatorPoolPath,
-                    conf.getLedgerAllocatorPoolCoreSize(),
-                    conf,
-                    writerZKC,
-                    writerBKC,
-                    scheduler);
-            if (null != allocator) {
-                allocator.start();
-            }
-            LOG.info("Created ledger allocator pool under {} with size {}.", allocatorPoolPath, conf.getLedgerAllocatorPoolCoreSize());
-        } else {
-            allocator = null;
-        }
-
-    }
-
-    private void checkState() throws IOException {
-        if (closed.get()) {
-            LOG.error("BK namespace driver {} is already closed", namespace);
-            throw new AlreadyClosedException("BK namespace driver " + namespace + " is already closed");
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        if (!closed.compareAndSet(false, true)) {
-            return;
-        }
-        doClose();
-    }
-
-    private void doClose() {
-        if (null != accessControlManager) {
-            accessControlManager.close();
-            LOG.info("Access Control Manager Stopped.");
-        }
-
-        // Close the allocator
-        if (null != allocator) {
-            Utils.closeQuietly(allocator);
-            LOG.info("Ledger Allocator stopped.");
-        }
-
-        // Shutdown log segment metadata stores
-        Utils.close(writerStreamMetadataStore);
-        Utils.close(readerStreamMetadataStore);
-
-        writerBKC.close();
-        readerBKC.close();
-        writerZKC.close();
-        readerZKC.close();
-        // release bookkeeper resources
-        channelFactory.releaseExternalResources();
-        LOG.info("Release external resources used by channel factory.");
-        requestTimer.stop();
-        LOG.info("Stopped request timer");
-    }
-
-    @Override
-    public URI getUri() {
-        return namespace;
-    }
-
-    @Override
-    public String getScheme() {
-        return DistributedLogConstants.BACKEND_BK;
-    }
-
-    @Override
-    public LogMetadataStore getLogMetadataStore() {
-        return metadataStore;
-    }
-
-    @Override
-    public LogStreamMetadataStore getLogStreamMetadataStore(Role role) {
-        if (Role.WRITER == role) {
-            return writerStreamMetadataStore;
-        } else {
-            return readerStreamMetadataStore;
-        }
-    }
-
-    @Override
-    public LogSegmentEntryStore getLogSegmentEntryStore(Role role) {
-        if (Role.WRITER == role) {
-            return getWriterEntryStore();
-        } else {
-            return getReaderEntryStore();
-        }
-    }
-
-    private LogSegmentEntryStore getWriterEntryStore() {
-        if (null == writerEntryStore) {
-            writerEntryStore = new BKLogSegmentEntryStore(
-                    conf,
-                    dynConf,
-                    writerZKC,
-                    writerBKC,
-                    scheduler,
-                    allocator,
-                    statsLogger,
-                    failureInjector);
-        }
-        return writerEntryStore;
-    }
-
-    private LogSegmentEntryStore getReaderEntryStore() {
-        if (null == readerEntryStore) {
-            readerEntryStore = new BKLogSegmentEntryStore(
-                    conf,
-                    dynConf,
-                    writerZKC,
-                    readerBKC,
-                    scheduler,
-                    allocator,
-                    statsLogger,
-                    failureInjector);
-        }
-        return readerEntryStore;
-    }
-
-    @Override
-    public AccessControlManager getAccessControlManager() throws IOException {
-        if (null == accessControlManager) {
-            String aclRootPath = getBkdlConfig().getACLRootPath();
-            // Build the access control manager
-            if (aclRootPath == null) {
-                accessControlManager = DefaultAccessControlManager.INSTANCE;
-                LOG.info("Created default access control manager for {}", namespace);
-            } else {
-                if (!isReservedStreamName(aclRootPath)) {
-                    throw new IOException("Invalid Access Control List Root Path : " + aclRootPath);
-                }
-                String zkRootPath = namespace.getPath() + "/" + aclRootPath;
-                LOG.info("Creating zk based access control manager @ {} for {}",
-                        zkRootPath, namespace);
-                accessControlManager = new ZKAccessControlManager(conf, readerZKC,
-                        zkRootPath, scheduler);
-                LOG.info("Created zk based access control manager @ {} for {}",
-                        zkRootPath, namespace);
-            }
-        }
-        return accessControlManager;
-    }
-
-    @Override
-    public SubscriptionsStore getSubscriptionsStore(String streamName) {
-        return new ZKSubscriptionsStore(
-                writerZKC,
-                LogMetadataForReader.getSubscribersPath(namespace, streamName, conf.getUnpartitionedStreamName()));
-    }
-
-    //
-    // Legacy Intefaces
-    //
-
-    @Override
-    public MetadataAccessor getMetadataAccessor(String streamName)
-            throws InvalidStreamNameException, IOException {
-        if (getBkdlConfig().isFederatedNamespace()) {
-            throw new UnsupportedOperationException();
-        }
-        checkState();
-        validateName(streamName);
-        return new ZKMetadataAccessor(
-                streamName,
-                conf,
-                namespace,
-                sharedWriterZKCBuilder,
-                sharedReaderZKCBuilder,
-                statsLogger);
-    }
-
-    public Map<String, byte[]> enumerateLogsWithMetadataInNamespace()
-        throws IOException, IllegalArgumentException {
-        String namespaceRootPath = namespace.getPath();
-        HashMap<String, byte[]> result = new HashMap<String, byte[]>();
-        ZooKeeperClient zkc = writerZKC;
-        try {
-            ZooKeeper zk = Utils.sync(zkc, namespaceRootPath);
-            Stat currentStat = zk.exists(namespaceRootPath, false);
-            if (currentStat == null) {
-                return result;
-            }
-            List<String> children = zk.getChildren(namespaceRootPath, false);
-            for(String child: children) {
-                if (isReservedStreamName(child)) {
-                    continue;
-                }
-                String zkPath = String.format("%s/%s", namespaceRootPath, child);
-                currentStat = zk.exists(zkPath, false);
-                if (currentStat == null) {
-                    result.put(child, new byte[0]);
-                } else {
-                    result.put(child, zk.getData(zkPath, false, currentStat));
-                }
-            }
-        } catch (InterruptedException ie) {
-            LOG.error("Interrupted while deleting " + namespaceRootPath, ie);
-            throw new IOException("Interrupted while reading " + namespaceRootPath, ie);
-        } catch (KeeperException ke) {
-            LOG.error("Error reading" + namespaceRootPath + "entry in zookeeper", ke);
-            throw new IOException("Error reading" + namespaceRootPath + "entry in zookeeper", ke);
-        }
-        return result;
-    }
-
-    //
-    // Zk & Bk Utils
-    //
-
-    public static ZooKeeperClientBuilder createZKClientBuilder(String zkcName,
-                                                               DistributedLogConfiguration conf,
-                                                               String zkServers,
-                                                               StatsLogger statsLogger) {
-        RetryPolicy retryPolicy = null;
-        if (conf.getZKNumRetries() > 0) {
-            retryPolicy = new BoundExponentialBackoffRetryPolicy(
-                conf.getZKRetryBackoffStartMillis(),
-                conf.getZKRetryBackoffMaxMillis(), conf.getZKNumRetries());
-        }
-        ZooKeeperClientBuilder builder = ZooKeeperClientBuilder.newBuilder()
-            .name(zkcName)
-            .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-            .retryThreadCount(conf.getZKClientNumberRetryThreads())
-            .requestRateLimit(conf.getZKRequestRateLimit())
-            .zkServers(zkServers)
-            .retryPolicy(retryPolicy)
-            .statsLogger(statsLogger)
-            .zkAclId(conf.getZkAclId());
-        LOG.info("Created shared zooKeeper client builder {}: zkServers = {}, numRetries = {}, sessionTimeout = {}, retryBackoff = {},"
-                + " maxRetryBackoff = {}, zkAclId = {}.", new Object[] { zkcName, zkServers, conf.getZKNumRetries(),
-                conf.getZKSessionTimeoutMilliseconds(), conf.getZKRetryBackoffStartMillis(),
-                conf.getZKRetryBackoffMaxMillis(), conf.getZkAclId() });
-        return builder;
-    }
-
-    private BookKeeperClientBuilder createBKCBuilder(String bkcName,
-                                                     DistributedLogConfiguration conf,
-                                                     String zkServers,
-                                                     String ledgersPath,
-                                                     ClientSocketChannelFactory channelFactory,
-                                                     HashedWheelTimer requestTimer,
-                                                     Optional<FeatureProvider> featureProviderOptional,
-                                                     StatsLogger statsLogger) {
-        BookKeeperClientBuilder builder = BookKeeperClientBuilder.newBuilder()
-                .name(bkcName)
-                .dlConfig(conf)
-                .zkServers(zkServers)
-                .ledgersPath(ledgersPath)
-                .channelFactory(channelFactory)
-                .requestTimer(requestTimer)
-                .featureProvider(featureProviderOptional)
-                .statsLogger(statsLogger);
-        LOG.info("Created shared client builder {} : zkServers = {}, ledgersPath = {}, numIOThreads = {}",
-                new Object[] { bkcName, zkServers, ledgersPath, conf.getBKClientNumberIOThreads() });
-        return builder;
-    }
-
-    //
-    // Test Methods
-    //
-
-    @VisibleForTesting
-    public ZooKeeperClient getWriterZKC() {
-        return writerZKC;
-    }
-
-    @VisibleForTesting
-    public BookKeeperClient getReaderBKC() {
-        return readerBKC;
-    }
-
-    @VisibleForTesting
-    public AsyncFailureInjector getFailureInjector() {
-        return this.failureInjector;
-    }
-
-    @VisibleForTesting
-    public LogStreamMetadataStore getWriterStreamMetadataStore() {
-        return writerStreamMetadataStore;
-    }
-
-    @VisibleForTesting
-    public LedgerAllocator getLedgerAllocator() {
-        return allocator;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java
deleted file mode 100644
index 50b1405..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java
+++ /dev/null
@@ -1,124 +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 com.twitter.distributedlog.impl;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.metadata.LogMetadataStore;
-import com.twitter.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 com.twitter.distributedlog.util.DLUtils.*;
-
-/**
- * ZooKeeper based log metadata store
- */
-public class ZKLogMetadataStore implements LogMetadataStore {
-
-    final URI namespace;
-    final Optional<URI> nsOptional;
-    final ZooKeeperClient zkc;
-    final ZKNamespaceWatcher nsWatcher;
-
-    public ZKLogMetadataStore(
-            DistributedLogConfiguration conf,
-            URI namespace,
-            ZooKeeperClient zkc,
-            OrderedScheduler scheduler) {
-        this.namespace = namespace;
-        this.nsOptional = Optional.of(this.namespace);
-        this.zkc = zkc;
-        this.nsWatcher = new ZKNamespaceWatcher(conf, namespace, zkc, scheduler);
-    }
-
-    @Override
-    public Future<URI> createLog(String logName) {
-        return Future.value(namespace);
-    }
-
-    @Override
-    public Future<Optional<URI>> getLogLocation(String logName) {
-        return Future.value(nsOptional);
-    }
-
-    @Override
-    public Future<Iterator<String>> getLogs() {
-        final Promise<Iterator<String>> promise = new Promise<Iterator<String>>();
-        final String nsRootPath = namespace.getPath();
-        try {
-            final ZooKeeper zk = zkc.get();
-            zk.sync(nsRootPath, new AsyncCallback.VoidCallback() {
-                @Override
-                public void processResult(int syncRc, String syncPath, Object ctx) {
-                    if (KeeperException.Code.OK.intValue() == syncRc) {
-                        zk.getChildren(nsRootPath, false, new AsyncCallback.Children2Callback() {
-                            @Override
-                            public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
-                                if (KeeperException.Code.OK.intValue() == rc) {
-                                    List<String> results = Lists.newArrayListWithExpectedSize(children.size());
-                                    for (String child : children) {
-                                        if (!isReservedStreamName(child)) {
-                                            results.add(child);
-                                        }
-                                    }
-                                    promise.setValue(results.iterator());
-                                } else if (KeeperException.Code.NONODE.intValue() == rc) {
-                                    List<String> streams = Lists.newLinkedList();
-                                    promise.setValue(streams.iterator());
-                                } else {
-                                    promise.setException(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());
-                    } else {
-                        promise.setException(new ZKException("Error reading namespace " + nsRootPath,
-                                KeeperException.Code.get(syncRc)));
-                    }
-                }
-            }, null);
-            zkc.get();
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
-        } catch (InterruptedException e) {
-            promise.setException(e);
-        }
-        return promise;
-    }
-
-    @Override
-    public void registerNamespaceListener(NamespaceListener listener) {
-        this.nsWatcher.registerListener(listener);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentFilters.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentFilters.java
deleted file mode 100644
index e55b2f2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentFilters.java
+++ /dev/null
@@ -1,89 +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 com.twitter.distributedlog.impl;
-
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.List;
-
-/**
- * Filters based on current zookeeper log segments.
- */
-public class ZKLogSegmentFilters {
-
-    static final Logger LOG = LoggerFactory.getLogger(ZKLogSegmentFilters.class);
-
-    /**
-     * Write handler filter should return all inprogress log segments and the last completed log segment.
-     * Because sequence id & ledger sequence number assignment rely on previous log segments.
-     */
-    public static final LogSegmentFilter WRITE_HANDLE_FILTER = new LogSegmentFilter() {
-        @Override
-        public Collection<String> filter(Collection<String> fullList) {
-            List<String> result = new ArrayList<String>(fullList.size());
-            String lastCompletedLogSegmentName = null;
-            long lastLogSegmentSequenceNumber = -1L;
-            for (String s : fullList) {
-                if (s.startsWith(DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX)) {
-                    result.add(s);
-                } else if (s.startsWith(DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX)) {
-                    String[] parts = s.split("_");
-                    try {
-                        if (2 == parts.length) {
-                            // name: logrecs_<logsegment_sequence_number>
-                            long logSegmentSequenceNumber = Long.parseLong(parts[1]);
-                            if (logSegmentSequenceNumber > lastLogSegmentSequenceNumber) {
-                                lastLogSegmentSequenceNumber = logSegmentSequenceNumber;
-                                lastCompletedLogSegmentName = s;
-                            }
-                        } else if (6 == parts.length) {
-                            // name: logrecs_<start_tx_id>_<end_tx_id>_<logsegment_sequence_number>_<ledger_id>_<region_id>
-                            long logSegmentSequenceNumber = Long.parseLong(parts[3]);
-                            if (logSegmentSequenceNumber > lastLogSegmentSequenceNumber) {
-                                lastLogSegmentSequenceNumber = logSegmentSequenceNumber;
-                                lastCompletedLogSegmentName = s;
-                            }
-                        } else {
-                            // name: logrecs_<start_tx_id>_<end_tx_id> or any unknown names
-                            // we don't know the ledger sequence from the name, so add it to the list
-                            result.add(s);
-                        }
-                    } catch (NumberFormatException nfe) {
-                        LOG.warn("Unexpected sequence number in log segment {} :", s, nfe);
-                        result.add(s);
-                    }
-                } else {
-                    LOG.error("Unknown log segment name : {}", s);
-                }
-            }
-            if (null != lastCompletedLogSegmentName) {
-                result.add(lastCompletedLogSegmentName);
-            }
-            if (LOG.isTraceEnabled()) {
-                LOG.trace("Filtered log segments {} from {}.", result, fullList);
-            }
-            return result;
-        }
-    };
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentMetadataStore.java
deleted file mode 100644
index 2076dd8..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentMetadataStore.java
+++ /dev/null
@@ -1,503 +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 com.twitter.distributedlog.impl;
-
-import com.google.common.collect.ImmutableList;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.callback.LogSegmentNamesListener;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.LogSegmentNotFoundException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.metadata.LogMetadata;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import com.twitter.distributedlog.zk.DefaultZKOp;
-import com.twitter.distributedlog.zk.ZKOp;
-import com.twitter.distributedlog.zk.ZKTransaction;
-import com.twitter.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;
-import org.apache.zookeeper.AsyncCallback.Children2Callback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * ZooKeeper based log segment metadata store.
- */
-public class ZKLogSegmentMetadataStore implements LogSegmentMetadataStore, Watcher, Children2Callback {
-
-    private static final Logger logger = LoggerFactory.getLogger(ZKLogSegmentMetadataStore.class);
-
-    private static final List<String> EMPTY_LIST = ImmutableList.of();
-
-    private static class ReadLogSegmentsTask implements Runnable, FutureEventListener<Versioned<List<String>>> {
-
-        private final String logSegmentsPath;
-        private final ZKLogSegmentMetadataStore store;
-        private int currentZKBackOffMs;
-
-        ReadLogSegmentsTask(String logSegmentsPath,
-                            ZKLogSegmentMetadataStore metadataStore) {
-            this.logSegmentsPath = logSegmentsPath;
-            this.store = metadataStore;
-            this.currentZKBackOffMs = store.minZKBackoffMs;
-        }
-
-        @Override
-        public void onSuccess(final Versioned<List<String>> segments) {
-            // reset the back off after a successful operation
-            currentZKBackOffMs = store.minZKBackoffMs;
-            store.notifyLogSegmentsUpdated(
-                    logSegmentsPath,
-                    store.listeners.get(logSegmentsPath),
-                    segments);
-        }
-
-        @Override
-        public void onFailure(Throwable cause) {
-            int backoffMs;
-            if (cause instanceof LogNotFoundException) {
-                // the log segment has been deleted, remove all the registered listeners
-                store.notifyLogStreamDeleted(logSegmentsPath,
-                        store.listeners.remove(logSegmentsPath));
-                return;
-            } else {
-                backoffMs = currentZKBackOffMs;
-                currentZKBackOffMs = Math.min(2 * currentZKBackOffMs, store.maxZKBackoffMs);
-            }
-            store.scheduleTask(logSegmentsPath, this, backoffMs);
-        }
-
-        @Override
-        public void run() {
-            if (null != store.listeners.get(logSegmentsPath)) {
-                store.zkGetLogSegmentNames(logSegmentsPath, store).addEventListener(this);
-            } else {
-                logger.debug("Log segments listener for {} has been removed.", logSegmentsPath);
-            }
-        }
-    }
-
-    /**
-     * A log segment names listener that keeps tracking the version of list of log segments that it has been notified.
-     * It only notify the newer log segments.
-     */
-    static class VersionedLogSegmentNamesListener {
-
-        private final LogSegmentNamesListener listener;
-        private Versioned<List<String>> lastNotifiedLogSegments;
-
-        VersionedLogSegmentNamesListener(LogSegmentNamesListener listener) {
-            this.listener = listener;
-            this.lastNotifiedLogSegments = new Versioned<List<String>>(EMPTY_LIST, Version.NEW);
-        }
-
-        synchronized void onSegmentsUpdated(Versioned<List<String>> logSegments) {
-            if (lastNotifiedLogSegments.getVersion() == Version.NEW ||
-                    lastNotifiedLogSegments.getVersion().compare(logSegments.getVersion()) == Version.Occurred.BEFORE) {
-                lastNotifiedLogSegments = logSegments;
-                listener.onSegmentsUpdated(logSegments);
-            }
-        }
-
-        @Override
-        public int hashCode() {
-            return listener.hashCode();
-        }
-
-        @Override
-        public boolean equals(Object obj) {
-            if (!(obj instanceof VersionedLogSegmentNamesListener)) {
-                return false;
-            }
-            VersionedLogSegmentNamesListener other = (VersionedLogSegmentNamesListener) obj;
-            return listener.equals(other.listener);
-        }
-
-        @Override
-        public String toString() {
-            return listener.toString();
-        }
-    }
-
-    final DistributedLogConfiguration conf;
-    // settings
-    final int minZKBackoffMs;
-    final int maxZKBackoffMs;
-    final boolean skipMinVersionCheck;
-
-    final ZooKeeperClient zkc;
-    // log segment listeners
-    final ConcurrentMap<String, Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener>> listeners;
-    // scheduler
-    final OrderedScheduler scheduler;
-    final ReentrantReadWriteLock closeLock;
-    boolean closed = false;
-
-    public ZKLogSegmentMetadataStore(DistributedLogConfiguration conf,
-                                     ZooKeeperClient zkc,
-                                     OrderedScheduler scheduler) {
-        this.conf = conf;
-        this.zkc = zkc;
-        this.listeners =
-                new ConcurrentHashMap<String, Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener>>();
-        this.scheduler = scheduler;
-        this.closeLock = new ReentrantReadWriteLock();
-        // settings
-        this.minZKBackoffMs = conf.getZKRetryBackoffStartMillis();
-        this.maxZKBackoffMs = conf.getZKRetryBackoffMaxMillis();
-        this.skipMinVersionCheck = conf.getDLLedgerMetadataSkipMinVersionCheck();
-    }
-
-    protected void scheduleTask(Object key, Runnable r, long delayMs) {
-        closeLock.readLock().lock();
-        try {
-            if (closed) {
-                return;
-            }
-            scheduler.schedule(key, r, delayMs, TimeUnit.MILLISECONDS);
-        } finally {
-            closeLock.readLock().unlock();
-        }
-    }
-
-    protected void submitTask(Object key, Runnable r) {
-        closeLock.readLock().lock();
-        try {
-            if (closed) {
-                return;
-            }
-            scheduler.submit(key, r);
-        } finally {
-            closeLock.readLock().unlock();
-        }
-    }
-
-    // max sequence number and max transaction id
-
-    @Override
-    public void storeMaxLogSegmentSequenceNumber(Transaction<Object> txn,
-                                                 LogMetadata logMetadata,
-                                                 Versioned<Long> lssn,
-                                                 Transaction.OpListener<Version> listener) {
-        Version version = lssn.getVersion();
-        assert(version instanceof ZkVersion);
-        ZkVersion zkVersion = (ZkVersion) version;
-        byte[] data = DLUtils.serializeLogSegmentSequenceNumber(lssn.getValue());
-        Op setDataOp = Op.setData(logMetadata.getLogSegmentsPath(), data, zkVersion.getZnodeVersion());
-        ZKOp zkOp = new ZKVersionedSetOp(setDataOp, listener);
-        txn.addOp(zkOp);
-    }
-
-    @Override
-    public void storeMaxTxnId(Transaction<Object> txn,
-                              LogMetadataForWriter logMetadata,
-                              Versioned<Long> transactionId,
-                              Transaction.OpListener<Version> listener) {
-        Version version = transactionId.getVersion();
-        assert(version instanceof ZkVersion);
-        ZkVersion zkVersion = (ZkVersion) version;
-        byte[] data = DLUtils.serializeTransactionId(transactionId.getValue());
-        Op setDataOp = Op.setData(logMetadata.getMaxTxIdPath(), data, zkVersion.getZnodeVersion());
-        ZKOp zkOp = new ZKVersionedSetOp(setDataOp, listener);
-        txn.addOp(zkOp);
-    }
-
-    // updates
-
-    @Override
-    public Transaction<Object> transaction() {
-        return new ZKTransaction(zkc);
-    }
-
-    @Override
-    public void createLogSegment(Transaction<Object> txn,
-                                 LogSegmentMetadata segment,
-                                 OpListener<Void> listener) {
-        byte[] finalisedData = segment.getFinalisedData().getBytes(UTF_8);
-        Op createOp = Op.create(
-                segment.getZkPath(),
-                finalisedData,
-                zkc.getDefaultACL(),
-                CreateMode.PERSISTENT);
-        txn.addOp(DefaultZKOp.of(createOp, listener));
-    }
-
-    @Override
-    public void deleteLogSegment(Transaction<Object> txn,
-                                 final LogSegmentMetadata segment,
-                                 final OpListener<Void> listener) {
-        Op deleteOp = Op.delete(
-                segment.getZkPath(),
-                -1);
-        logger.info("Delete segment : {}", segment);
-        txn.addOp(DefaultZKOp.of(deleteOp, new OpListener<Void>() {
-            @Override
-            public void onCommit(Void r) {
-                if (null != listener) {
-                    listener.onCommit(r);
-                }
-            }
-
-            @Override
-            public void onAbort(Throwable t) {
-                logger.info("Aborted transaction on deleting segment {}", segment);
-                KeeperException.Code kc;
-                if (t instanceof KeeperException) {
-                    kc = ((KeeperException) t).code();
-                } else if (t instanceof ZKException) {
-                    kc = ((ZKException) t).getKeeperExceptionCode();
-                } else {
-                    abortListener(t);
-                    return;
-                }
-                if (KeeperException.Code.NONODE == kc) {
-                    abortListener(new LogSegmentNotFoundException(segment.getZkPath()));
-                    return;
-                }
-                abortListener(t);
-            }
-
-            private void abortListener(Throwable t) {
-                if (null != listener) {
-                    listener.onAbort(t);
-                }
-            }
-        }));
-    }
-
-    @Override
-    public void updateLogSegment(Transaction<Object> txn, LogSegmentMetadata segment) {
-        byte[] finalisedData = segment.getFinalisedData().getBytes(UTF_8);
-        Op updateOp = Op.setData(segment.getZkPath(), finalisedData, -1);
-        txn.addOp(DefaultZKOp.of(updateOp, null));
-    }
-
-    // reads
-
-    /**
-     * Process the watched events for registered listeners
-     */
-    @Override
-    public void process(WatchedEvent event) {
-        if (Event.EventType.None == event.getType()
-                && Event.KeeperState.Expired == event.getState()) {
-            Set<String> keySet = new HashSet<String>(listeners.keySet());
-            for (String logSegmentsPath : keySet) {
-                scheduleTask(logSegmentsPath, new ReadLogSegmentsTask(logSegmentsPath, this), 0L);
-            }
-            return;
-        }
-        String path = event.getPath();
-        if (null == path) {
-            return;
-        }
-        switch (event.getType()) {
-            case NodeDeleted:
-                notifyLogStreamDeleted(path, listeners.remove(path));
-                break;
-            case NodeChildrenChanged:
-                new ReadLogSegmentsTask(path, this).run();
-                break;
-            default:
-                break;
-        }
-    }
-
-    @Override
-    public Future<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>>>();
-        try {
-            zkc.get().getChildren(logSegmentsPath, watcher, this, result);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            result.setException(FutureUtils.zkException(e, logSegmentsPath));
-        } catch (InterruptedException e) {
-            result.setException(FutureUtils.zkException(e, logSegmentsPath));
-        }
-        return result;
-    }
-
-    @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);
-        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));
-        } else if (KeeperException.Code.NONODE.intValue() == rc) {
-            result.setException(new LogNotFoundException("Log " + path + " not found"));
-        } else {
-            result.setException(new ZKException("Failed to get log segments from " + path,
-                    KeeperException.Code.get(rc)));
-        }
-    }
-
-    @Override
-    public Future<Versioned<List<String>>> getLogSegmentNames(String logSegmentsPath,
-                                                              LogSegmentNamesListener listener) {
-        Watcher zkWatcher;
-        if (null == listener) {
-            zkWatcher = null;
-        } else {
-            closeLock.readLock().lock();
-            try {
-                if (closed) {
-                    zkWatcher = null;
-                } else {
-                    Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> listenerSet =
-                            listeners.get(logSegmentsPath);
-                    if (null == listenerSet) {
-                        Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> newListenerSet =
-                                new HashMap<LogSegmentNamesListener, VersionedLogSegmentNamesListener>();
-                        Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> oldListenerSet =
-                                listeners.putIfAbsent(logSegmentsPath, newListenerSet);
-                        if (null != oldListenerSet) {
-                            listenerSet = oldListenerSet;
-                        } else {
-                            listenerSet = newListenerSet;
-                        }
-                    }
-                    synchronized (listenerSet) {
-                        listenerSet.put(listener, new VersionedLogSegmentNamesListener(listener));
-                        if (!listeners.containsKey(logSegmentsPath)) {
-                            // listener set has been removed, add it back
-                            if (null != listeners.putIfAbsent(logSegmentsPath, listenerSet)) {
-                                logger.debug("Listener set is already found for log segments path {}", logSegmentsPath);
-                            }
-                        }
-                    }
-                    zkWatcher = ZKLogSegmentMetadataStore.this;
-                }
-            } finally {
-                closeLock.readLock().unlock();
-            }
-        }
-        Future<Versioned<List<String>>> getLogSegmentNamesResult = zkGetLogSegmentNames(logSegmentsPath, zkWatcher);
-        if (null != listener) {
-            getLogSegmentNamesResult.addEventListener(new ReadLogSegmentsTask(logSegmentsPath, this));
-        }
-        return zkGetLogSegmentNames(logSegmentsPath, zkWatcher);
-    }
-
-    @Override
-    public void unregisterLogSegmentListener(String logSegmentsPath,
-                                             LogSegmentNamesListener listener) {
-        closeLock.readLock().lock();
-        try {
-            if (closed) {
-                return;
-            }
-            Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> listenerSet =
-                    listeners.get(logSegmentsPath);
-            if (null == listenerSet) {
-                return;
-            }
-            synchronized (listenerSet) {
-                listenerSet.remove(listener);
-                if (listenerSet.isEmpty()) {
-                    listeners.remove(logSegmentsPath, listenerSet);
-                }
-            }
-        } finally {
-            closeLock.readLock().unlock();
-        }
-    }
-
-    @Override
-    public void close() throws IOException {
-        closeLock.writeLock().lock();
-        try {
-            if (closed) {
-                return;
-            }
-            closed = true;
-        } finally {
-            closeLock.writeLock().unlock();
-        }
-    }
-
-    // Notifications
-
-    void notifyLogStreamDeleted(String logSegmentsPath,
-                                final Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> listeners) {
-        if (null == listeners) {
-            return;
-        }
-        this.submitTask(logSegmentsPath, new Runnable() {
-            @Override
-            public void run() {
-                for (LogSegmentNamesListener listener : listeners.keySet()) {
-                    listener.onLogStreamDeleted();
-                }
-            }
-        });
-
-    }
-
-    void notifyLogSegmentsUpdated(String logSegmentsPath,
-                                  final Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> listeners,
-                                  final Versioned<List<String>> segments) {
-        if (null == listeners) {
-            return;
-        }
-        this.submitTask(logSegmentsPath, new Runnable() {
-            @Override
-            public void run() {
-                for (VersionedLogSegmentNamesListener listener : listeners.values()) {
-                    listener.onSegmentsUpdated(segments);
-                }
-            }
-        });
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java
deleted file mode 100644
index eeda804..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java
+++ /dev/null
@@ -1,264 +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 com.twitter.distributedlog.impl;
-
-import java.io.IOException;
-import java.net.URI;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.MetadataAccessor;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.exceptions.AlreadyClosedException;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.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;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static com.twitter.distributedlog.impl.BKNamespaceDriver.getZKServersFromDLUri;
-
-public class ZKMetadataAccessor implements MetadataAccessor {
-    static final Logger LOG = LoggerFactory.getLogger(ZKMetadataAccessor.class);
-    protected final String name;
-    protected Promise<Void> closePromise;
-    protected final URI uri;
-    // zookeeper clients
-    // NOTE: The actual zookeeper client is initialized lazily when it is referenced by
-    //       {@link com.twitter.distributedlog.ZooKeeperClient#get()}. So it is safe to
-    //       keep builders and their client wrappers here, as they will be used when
-    //       instantiating readers or writers.
-    protected final ZooKeeperClientBuilder writerZKCBuilder;
-    protected final ZooKeeperClient writerZKC;
-    protected final boolean ownWriterZKC;
-    protected final ZooKeeperClientBuilder readerZKCBuilder;
-    protected final ZooKeeperClient readerZKC;
-    protected final boolean ownReaderZKC;
-
-    ZKMetadataAccessor(String name,
-                       DistributedLogConfiguration conf,
-                       URI uri,
-                       ZooKeeperClientBuilder writerZKCBuilder,
-                       ZooKeeperClientBuilder readerZKCBuilder,
-                       StatsLogger statsLogger) {
-        this.name = name;
-        this.uri = uri;
-
-        if (null == writerZKCBuilder) {
-            RetryPolicy retryPolicy = null;
-            if (conf.getZKNumRetries() > 0) {
-                retryPolicy = new BoundExponentialBackoffRetryPolicy(
-                    conf.getZKRetryBackoffStartMillis(),
-                    conf.getZKRetryBackoffMaxMillis(), conf.getZKNumRetries());
-            }
-            this.writerZKCBuilder = ZooKeeperClientBuilder.newBuilder()
-                    .name(String.format("dlzk:%s:dlm_writer_shared", name))
-                    .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                    .retryThreadCount(conf.getZKClientNumberRetryThreads())
-                    .requestRateLimit(conf.getZKRequestRateLimit())
-                    .zkAclId(conf.getZkAclId())
-                    .uri(uri)
-                    .retryPolicy(retryPolicy)
-                    .statsLogger(statsLogger.scope("dlzk_dlm_writer_shared"));
-            this.ownWriterZKC = true;
-        } else {
-            this.writerZKCBuilder = writerZKCBuilder;
-            this.ownWriterZKC = false;
-        }
-        this.writerZKC = this.writerZKCBuilder.build();
-
-        if (null == readerZKCBuilder) {
-            String zkServersForWriter = getZKServersFromDLUri(uri);
-            String zkServersForReader;
-            try {
-                BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(this.writerZKC, uri);
-                zkServersForReader = bkdlConfig.getDlZkServersForReader();
-            } catch (IOException e) {
-                LOG.warn("Error on resolving dl metadata bindings for {} : ", uri, e);
-                zkServersForReader = zkServersForWriter;
-            }
-            if (zkServersForReader.equals(zkServersForWriter)) {
-                LOG.info("Used same zookeeper servers '{}' for both writers and readers for {}.",
-                         zkServersForWriter, name);
-                this.readerZKCBuilder = this.writerZKCBuilder;
-                this.ownReaderZKC = false;
-            } else {
-                RetryPolicy retryPolicy = null;
-                if (conf.getZKNumRetries() > 0) {
-                    retryPolicy = new BoundExponentialBackoffRetryPolicy(
-                        conf.getZKRetryBackoffStartMillis(),
-                        conf.getZKRetryBackoffMaxMillis(), conf.getZKNumRetries());
-                }
-                this.readerZKCBuilder = ZooKeeperClientBuilder.newBuilder()
-                        .name(String.format("dlzk:%s:dlm_reader_shared", name))
-                        .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                        .retryThreadCount(conf.getZKClientNumberRetryThreads())
-                        .requestRateLimit(conf.getZKRequestRateLimit())
-                        .zkServers(zkServersForReader)
-                        .retryPolicy(retryPolicy)
-                        .zkAclId(conf.getZkAclId())
-                        .statsLogger(statsLogger.scope("dlzk_dlm_reader_shared"));
-                this.ownReaderZKC = true;
-            }
-        } else {
-            this.readerZKCBuilder = readerZKCBuilder;
-            this.ownReaderZKC = false;
-        }
-        this.readerZKC = this.readerZKCBuilder.build();
-    }
-
-    /**
-     * Get the name of the stream managed by this log manager
-     *
-     * @return streamName
-     */
-    @Override
-    public String getStreamName() {
-        return name;
-    }
-
-    /**
-     * Creates or update the metadata stored at the node associated with the
-     * name and URI
-     * @param metadata opaque metadata to be stored for the node
-     * @throws IOException
-     */
-    @Override
-    public void createOrUpdateMetadata(byte[] metadata) throws IOException {
-        checkClosedOrInError("createOrUpdateMetadata");
-
-        String zkPath = getZKPath();
-        LOG.debug("Setting application specific metadata on {}", zkPath);
-        try {
-            Stat currentStat = writerZKC.get().exists(zkPath, false);
-            if (currentStat == null) {
-                if (metadata.length > 0) {
-                    Utils.zkCreateFullPathOptimistic(writerZKC,
-                            zkPath,
-                            metadata,
-                            writerZKC.getDefaultACL(),
-                            CreateMode.PERSISTENT);
-                }
-            } else {
-                writerZKC.get().setData(zkPath, metadata, currentStat.getVersion());
-            }
-        } catch (InterruptedException ie) {
-            throw new DLInterruptedException("Interrupted on creating or updating container metadata", ie);
-        } catch (Exception exc) {
-            throw new IOException("Exception creating or updating container metadata", exc);
-        }
-    }
-
-    /**
-     * Delete the metadata stored at the associated node. This only deletes the metadata
-     * and not the node itself
-     * @throws IOException
-     */
-    @Override
-    public void deleteMetadata() throws IOException {
-        checkClosedOrInError("createOrUpdateMetadata");
-        createOrUpdateMetadata(null);
-    }
-
-    /**
-     * Retrieve the metadata stored at the node
-     * @return byte array containing the metadata
-     * @throws IOException
-     */
-    @Override
-    public byte[] getMetadata() throws IOException {
-        checkClosedOrInError("createOrUpdateMetadata");
-        String zkPath = getZKPath();
-        LOG.debug("Getting application specific metadata from {}", zkPath);
-        try {
-            Stat currentStat = readerZKC.get().exists(zkPath, false);
-            if (currentStat == null) {
-                return null;
-            } else {
-                return readerZKC.get().getData(zkPath, false, currentStat);
-            }
-        } catch (InterruptedException ie) {
-            throw new DLInterruptedException("Error reading the max tx id from zk", ie);
-        } catch (Exception e) {
-            throw new IOException("Error reading the max tx id from zk", e);
-        }
-    }
-
-    /**
-     * Close the metadata accessor, freeing any resources it may hold.
-     * @return future represents the close result.
-     */
-    @Override
-    public Future<Void> asyncClose() {
-        Promise<Void> closeFuture;
-        synchronized (this) {
-            if (null != closePromise) {
-                return closePromise;
-            }
-            closeFuture = closePromise = new Promise<Void>();
-        }
-        // NOTE: ownWriterZKC and ownReaderZKC are mostly used by tests
-        //       the managers created by the namespace - whose zkc will be closed by namespace
-        try {
-            if (ownWriterZKC) {
-                writerZKC.close();
-            }
-            if (ownReaderZKC) {
-                readerZKC.close();
-            }
-        } catch (Exception e) {
-            LOG.warn("Exception while closing distributed log manager", e);
-        }
-        FutureUtils.setValue(closeFuture, null);
-        return closeFuture;
-    }
-
-    @Override
-    public void close() throws IOException {
-        FutureUtils.result(asyncClose());
-    }
-
-    public synchronized void checkClosedOrInError(String operation) throws AlreadyClosedException {
-        if (null != closePromise) {
-            throw new AlreadyClosedException("Executing " + operation + " on already closed ZKMetadataAccessor");
-        }
-    }
-
-    protected String getZKPath() {
-        return String.format("%s/%s", uri.getPath(), name);
-    }
-
-    @VisibleForTesting
-    protected ZooKeeperClient getReaderZKC() {
-        return readerZKC;
-    }
-
-    @VisibleForTesting
-    protected ZooKeeperClient getWriterZKC() {
-        return writerZKC;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKNamespaceWatcher.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKNamespaceWatcher.java
deleted file mode 100644
index 06bc8fb..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKNamespaceWatcher.java
+++ /dev/null
@@ -1,133 +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 com.twitter.distributedlog.impl;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.namespace.NamespaceWatcher;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static com.twitter.distributedlog.util.DLUtils.*;
-
-/**
- * Watcher on watching a given namespace
- */
-public class ZKNamespaceWatcher extends NamespaceWatcher
-        implements Runnable, Watcher, AsyncCallback.Children2Callback {
-
-    static final Logger logger = LoggerFactory.getLogger(ZKNamespaceWatcher.class);
-
-    private final DistributedLogConfiguration conf;
-    private final URI uri;
-    private final ZooKeeperClient zkc;
-    private final OrderedScheduler scheduler;
-    private final AtomicBoolean namespaceWatcherSet = new AtomicBoolean(false);
-
-    public ZKNamespaceWatcher(DistributedLogConfiguration conf,
-                              URI uri,
-                              ZooKeeperClient zkc,
-                              OrderedScheduler scheduler) {
-        this.conf = conf;
-        this.uri = uri;
-        this.zkc = zkc;
-        this.scheduler = scheduler;
-    }
-
-    private void scheduleTask(Runnable r, long ms) {
-        try {
-            scheduler.schedule(r, ms, TimeUnit.MILLISECONDS);
-        } catch (RejectedExecutionException ree) {
-            logger.error("Task {} scheduled in {} ms is rejected : ", new Object[]{r, ms, ree});
-        }
-    }
-
-    @Override
-    public void run() {
-        try {
-            doWatchNamespaceChanges();
-        } catch (Exception e) {
-            logger.error("Encountered unknown exception on watching namespace {} ", uri, e);
-        }
-    }
-
-    public void watchNamespaceChanges() {
-        if (!namespaceWatcherSet.compareAndSet(false, true)) {
-            return;
-        }
-        doWatchNamespaceChanges();
-    }
-
-    private void doWatchNamespaceChanges() {
-        try {
-            zkc.get().getChildren(uri.getPath(), this, this, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
-        } catch (InterruptedException e) {
-            logger.warn("Interrupted on watching namespace changes for {} : ", uri, e);
-            scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
-        }
-    }
-
-    @Override
-    public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
-        if (KeeperException.Code.OK.intValue() == rc) {
-            logger.info("Received updated logs under {} : {}", uri, children);
-            List<String> result = new ArrayList<String>(children.size());
-            for (String s : children) {
-                if (isReservedStreamName(s)) {
-                    continue;
-                }
-                result.add(s);
-            }
-            for (NamespaceListener listener : listeners) {
-                listener.onStreamsChanged(result.iterator());
-            }
-        } else {
-            scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
-        }
-    }
-
-    @Override
-    public void process(WatchedEvent event) {
-        if (event.getType() == Event.EventType.None) {
-            if (event.getState() == Event.KeeperState.Expired) {
-                scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
-            }
-            return;
-        }
-        if (event.getType() == Event.EventType.NodeChildrenChanged) {
-            // watch namespace changes again.
-            doWatchNamespaceChanges();
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java
deleted file mode 100644
index 8126723..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java
+++ /dev/null
@@ -1,232 +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 com.twitter.distributedlog.impl.acl;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.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;
-import org.apache.thrift.transport.TMemoryInputTransport;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.data.Stat;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-public class ZKAccessControl {
-
-    private static final int BUFFER_SIZE = 4096;
-
-    public static final AccessControlEntry DEFAULT_ACCESS_CONTROL_ENTRY = new AccessControlEntry();
-
-    public static class CorruptedAccessControlException extends IOException {
-
-        private static final long serialVersionUID = 5391285182476211603L;
-
-        public CorruptedAccessControlException(String zkPath, Throwable t) {
-            super("Access Control @ " + zkPath + " is corrupted.", t);
-        }
-    }
-
-    protected final AccessControlEntry accessControlEntry;
-    protected final String zkPath;
-    private int zkVersion;
-
-    public ZKAccessControl(AccessControlEntry ace, String zkPath) {
-        this(ace, zkPath, -1);
-    }
-
-    private ZKAccessControl(AccessControlEntry ace, String zkPath, int zkVersion) {
-        this.accessControlEntry = ace;
-        this.zkPath = zkPath;
-        this.zkVersion = zkVersion;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hashCode(zkPath, accessControlEntry);
-    }
-
-    @Override
-    public boolean equals(Object obj) {
-        if (!(obj instanceof ZKAccessControl)) {
-            return false;
-        }
-        ZKAccessControl other = (ZKAccessControl) obj;
-        return Objects.equal(zkPath, other.zkPath) &&
-                Objects.equal(accessControlEntry, other.accessControlEntry);
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("entry(path=").append(zkPath).append(", acl=")
-                .append(accessControlEntry).append(")");
-        return sb.toString();
-    }
-
-    @VisibleForTesting
-    public String getZKPath() {
-        return zkPath;
-    }
-
-    @VisibleForTesting
-    public AccessControlEntry getAccessControlEntry() {
-        return accessControlEntry;
-    }
-
-    public Future<ZKAccessControl> create(ZooKeeperClient zkc) {
-        final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
-        try {
-            zkc.get().create(zkPath, serialize(accessControlEntry), zkc.getDefaultACL(), CreateMode.PERSISTENT,
-                    new AsyncCallback.StringCallback() {
-                        @Override
-                        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);
-                            } else {
-                                promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
-                            }
-                        }
-                    }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
-        } catch (InterruptedException e) {
-            promise.setException(e);
-        } catch (IOException e) {
-            promise.setException(e);
-        }
-        return promise;
-    }
-
-    public Future<ZKAccessControl> update(ZooKeeperClient zkc) {
-        final Promise<ZKAccessControl> promise = new Promise<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);
-                    } else {
-                        promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
-                    }
-                }
-            }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
-        } catch (InterruptedException e) {
-            promise.setException(e);
-        } catch (IOException e) {
-            promise.setException(e);
-        }
-        return promise;
-    }
-
-    public static Future<ZKAccessControl> read(final ZooKeeperClient zkc, final String zkPath, Watcher watcher) {
-        final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
-
-        try {
-            zkc.get().getData(zkPath, watcher, new AsyncCallback.DataCallback() {
-                @Override
-                public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
-                    if (KeeperException.Code.OK.intValue() == rc) {
-                        try {
-                            AccessControlEntry ace = deserialize(zkPath, data);
-                            promise.setValue(new ZKAccessControl(ace, zkPath, stat.getVersion()));
-                        } catch (IOException ioe) {
-                            promise.setException(ioe);
-                        }
-                    } else {
-                        promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
-                    }
-                }
-            }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
-        } catch (InterruptedException e) {
-            promise.setException(e);
-        }
-        return promise;
-    }
-
-    public static Future<Void> delete(final ZooKeeperClient zkc, final String zkPath) {
-        final Promise<Void> promise = new Promise<Void>();
-
-        try {
-            zkc.get().delete(zkPath, -1, new AsyncCallback.VoidCallback() {
-                @Override
-                public void processResult(int rc, String path, Object ctx) {
-                    if (KeeperException.Code.OK.intValue() == rc ||
-                            KeeperException.Code.NONODE.intValue() == rc) {
-                        promise.setValue(null);
-                    } else {
-                        promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
-                    }
-                }
-            }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
-        } catch (InterruptedException e) {
-            promise.setException(e);
-        }
-        return promise;
-    }
-
-    static byte[] serialize(AccessControlEntry ace) throws IOException {
-        TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
-        TJSONProtocol protocol = new TJSONProtocol(transport);
-        try {
-            ace.write(protocol);
-            transport.flush();
-            return transport.toString(UTF_8.name()).getBytes(UTF_8);
-        } catch (TException e) {
-            throw new IOException("Failed to serialize access control entry : ", e);
-        } catch (UnsupportedEncodingException uee) {
-            throw new IOException("Failed to serialize acesss control entry : ", uee);
-        }
-    }
-
-    static AccessControlEntry deserialize(String zkPath, byte[] data) throws IOException {
-        if (data.length == 0) {
-            return DEFAULT_ACCESS_CONTROL_ENTRY;
-        }
-
-        AccessControlEntry ace = new AccessControlEntry();
-        TMemoryInputTransport transport = new TMemoryInputTransport(data);
-        TJSONProtocol protocol = new TJSONProtocol(transport);
-        try {
-            ace.read(protocol);
-        } catch (TException e) {
-            throw new CorruptedAccessControlException(zkPath, e);
-        }
-        return ace;
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java
deleted file mode 100644
index 0c90a50..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java
+++ /dev/null
@@ -1,374 +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 com.twitter.distributedlog.impl.acl;
-
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.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.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * ZooKeeper Based {@link com.twitter.distributedlog.acl.AccessControlManager}
- */
-public class ZKAccessControlManager implements AccessControlManager, Watcher {
-
-    private static final Logger logger = LoggerFactory.getLogger(ZKAccessControlManager.class);
-
-    private static final int ZK_RETRY_BACKOFF_MS = 500;
-
-    protected final DistributedLogConfiguration conf;
-    protected final ZooKeeperClient zkc;
-    protected final String zkRootPath;
-    protected final ScheduledExecutorService scheduledExecutorService;
-
-    protected final ConcurrentMap<String, ZKAccessControl> streamEntries;
-    protected ZKAccessControl defaultAccessControl;
-    protected volatile boolean closed = false;
-
-    public ZKAccessControlManager(DistributedLogConfiguration conf,
-                                  ZooKeeperClient zkc,
-                                  String zkRootPath,
-                                  ScheduledExecutorService scheduledExecutorService) throws IOException {
-        this.conf = conf;
-        this.zkc = zkc;
-        this.zkRootPath = zkRootPath;
-        this.scheduledExecutorService = scheduledExecutorService;
-        this.streamEntries = new ConcurrentHashMap<String, ZKAccessControl>();
-        try {
-            Await.result(fetchDefaultAccessControlEntry());
-        } catch (Throwable t) {
-            if (t instanceof InterruptedException) {
-                throw new DLInterruptedException("Interrupted on getting default access control entry for " + zkRootPath, t);
-            } else if (t instanceof KeeperException) {
-                throw new IOException("Encountered zookeeper exception on getting default access control entry for " + zkRootPath, t);
-            } else if (t instanceof IOException) {
-                throw (IOException) t;
-            } else {
-                throw new IOException("Encountered unknown exception on getting access control entries for " + zkRootPath, t);
-            }
-        }
-
-        try {
-            Await.result(fetchAccessControlEntries());
-        } catch (Throwable t) {
-            if (t instanceof InterruptedException) {
-                throw new DLInterruptedException("Interrupted on getting access control entries for " + zkRootPath, t);
-            } else if (t instanceof KeeperException) {
-                throw new IOException("Encountered zookeeper exception on getting access control entries for " + zkRootPath, t);
-            } else if (t instanceof IOException) {
-                throw (IOException) t;
-            } else {
-                throw new IOException("Encountered unknown exception on getting access control entries for " + zkRootPath, t);
-            }
-        }
-    }
-
-    protected AccessControlEntry getAccessControlEntry(String stream) {
-        ZKAccessControl entry = streamEntries.get(stream);
-        entry = null == entry ? defaultAccessControl : entry;
-        return entry.getAccessControlEntry();
-    }
-
-    @Override
-    public boolean allowWrite(String stream) {
-        return !getAccessControlEntry(stream).isDenyWrite();
-    }
-
-    @Override
-    public boolean allowTruncate(String stream) {
-        return !getAccessControlEntry(stream).isDenyTruncate();
-    }
-
-    @Override
-    public boolean allowDelete(String stream) {
-        return !getAccessControlEntry(stream).isDenyDelete();
-    }
-
-    @Override
-    public boolean allowAcquire(String stream) {
-        return !getAccessControlEntry(stream).isDenyAcquire();
-    }
-
-    @Override
-    public boolean allowRelease(String stream) {
-        return !getAccessControlEntry(stream).isDenyRelease();
-    }
-
-    @Override
-    public void close() {
-        closed = true;
-    }
-
-    private Future<Void> fetchAccessControlEntries() {
-        final Promise<Void> promise = new Promise<Void>();
-        fetchAccessControlEntries(promise);
-        return promise;
-    }
-
-    private void fetchAccessControlEntries(final Promise<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)));
-                        return;
-                    }
-                    Set<String> streamsReceived = new HashSet<String>();
-                    streamsReceived.addAll(children);
-                    Set<String> streamsCached = streamEntries.keySet();
-                    Set<String> streamsRemoved = Sets.difference(streamsCached, streamsReceived).immutableCopy();
-                    for (String s : streamsRemoved) {
-                        ZKAccessControl accessControl = streamEntries.remove(s);
-                        if (null != accessControl) {
-                            logger.info("Removed Access Control Entry for stream {} : {}", s, accessControl.getAccessControlEntry());
-                        }
-                    }
-                    if (streamsReceived.isEmpty()) {
-                        promise.setValue(null);
-                        return;
-                    }
-                    final AtomicInteger numPendings = new AtomicInteger(streamsReceived.size());
-                    final AtomicInteger numFailures = new AtomicInteger(0);
-                    for (String s : streamsReceived) {
-                        final String streamName = s;
-                        ZKAccessControl.read(zkc, zkRootPath + "/" + streamName, null)
-                                .addEventListener(new FutureEventListener<ZKAccessControl>() {
-
-                                    @Override
-                                    public void onSuccess(ZKAccessControl accessControl) {
-                                        streamEntries.put(streamName, accessControl);
-                                        logger.info("Added overrided access control for stream {} : {}", streamName, accessControl.getAccessControlEntry());
-                                        complete();
-                                    }
-
-                                    @Override
-                                    public void onFailure(Throwable cause) {
-                                        if (cause instanceof KeeperException.NoNodeException) {
-                                            streamEntries.remove(streamName);
-                                        } else if (cause instanceof ZKAccessControl.CorruptedAccessControlException) {
-                                            logger.warn("Access control is corrupted for stream {} @ {}, skipped it ...",
-                                                        new Object[] { streamName, zkRootPath, cause });
-                                            streamEntries.remove(streamName);
-                                        } else {
-                                            if (1 == numFailures.incrementAndGet()) {
-                                                promise.setException(cause);
-                                            }
-                                        }
-                                        complete();
-                                    }
-
-                                    private void complete() {
-                                        if (0 == numPendings.decrementAndGet() && numFailures.get() == 0) {
-                                            promise.setValue(null);
-                                        }
-                                    }
-                                });
-                    }
-                }
-            }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
-        } catch (InterruptedException e) {
-            promise.setException(e);
-        }
-    }
-
-    private Future<ZKAccessControl> fetchDefaultAccessControlEntry() {
-        final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
-        fetchDefaultAccessControlEntry(promise);
-        return promise;
-    }
-
-    private void fetchDefaultAccessControlEntry(final Promise<ZKAccessControl> promise) {
-        ZKAccessControl.read(zkc, zkRootPath, this)
-            .addEventListener(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);
-                }
-
-                @Override
-                public void onFailure(Throwable cause) {
-                    if (cause instanceof KeeperException.NoNodeException) {
-                        logger.info("Default Access Control is missing, creating one for {} ...", zkRootPath);
-                        createDefaultAccessControlEntryIfNeeded(promise);
-                    } else {
-                        promise.setException(cause);
-                    }
-                }
-            });
-    }
-
-    private void createDefaultAccessControlEntryIfNeeded(final Promise<ZKAccessControl> promise) {
-        ZooKeeper zk;
-        try {
-            zk = zkc.get();
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
-            return;
-        } catch (InterruptedException e) {
-            promise.setException(e);
-            return;
-        }
-        ZkUtils.asyncCreateFullPathOptimistic(zk, zkRootPath, new byte[0], zkc.getDefaultACL(),
-                CreateMode.PERSISTENT, new AsyncCallback.StringCallback() {
-            @Override
-            public void processResult(int rc, String path, Object ctx, String name) {
-                if (KeeperException.Code.OK.intValue() == rc) {
-                    logger.info("Created zk path {} for default ACL.", zkRootPath);
-                    fetchDefaultAccessControlEntry(promise);
-                } else {
-                    promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
-                }
-            }
-        }, null);
-    }
-
-    private void refetchDefaultAccessControlEntry(final int delayMs) {
-        if (closed) {
-            return;
-        }
-        scheduledExecutorService.schedule(new Runnable() {
-            @Override
-            public void run() {
-                fetchDefaultAccessControlEntry().addEventListener(new FutureEventListener<ZKAccessControl>() {
-                    @Override
-                    public void onSuccess(ZKAccessControl value) {
-                        // no-op
-                    }
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        if (cause instanceof ZKAccessControl.CorruptedAccessControlException) {
-                            logger.warn("Default access control entry is corrupted, ignore this update : ", cause);
-                            return;
-                        }
-
-                        logger.warn("Encountered an error on refetching default access control entry, retrying in {} ms : ",
-                                    ZK_RETRY_BACKOFF_MS, cause);
-                        refetchDefaultAccessControlEntry(ZK_RETRY_BACKOFF_MS);
-                    }
-                });
-            }
-        }, delayMs, TimeUnit.MILLISECONDS);
-    }
-
-    private void refetchAccessControlEntries(final int delayMs) {
-        if (closed) {
-            return;
-        }
-        scheduledExecutorService.schedule(new Runnable() {
-            @Override
-            public void run() {
-                fetchAccessControlEntries().addEventListener(new FutureEventListener<Void>() {
-                    @Override
-                    public void onSuccess(Void value) {
-                        // no-op
-                    }
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        logger.warn("Encountered an error on refetching access control entries, retrying in {} ms : ",
-                                    ZK_RETRY_BACKOFF_MS, cause);
-                        refetchAccessControlEntries(ZK_RETRY_BACKOFF_MS);
-                    }
-                });
-            }
-        }, delayMs, TimeUnit.MILLISECONDS);
-    }
-
-    private void refetchAllAccessControlEntries(final int delayMs) {
-        if (closed) {
-            return;
-        }
-        scheduledExecutorService.schedule(new Runnable() {
-            @Override
-            public void run() {
-                fetchDefaultAccessControlEntry().addEventListener(new FutureEventListener<ZKAccessControl>() {
-                    @Override
-                    public void onSuccess(ZKAccessControl value) {
-                        fetchAccessControlEntries().addEventListener(new FutureEventListener<Void>() {
-                            @Override
-                            public void onSuccess(Void value) {
-                                // no-op
-                            }
-
-                            @Override
-                            public void onFailure(Throwable cause) {
-                                logger.warn("Encountered an error on fetching all access control entries, retrying in {} ms : ",
-                                            ZK_RETRY_BACKOFF_MS, cause);
-                                refetchAccessControlEntries(ZK_RETRY_BACKOFF_MS);
-                            }
-                        });
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        logger.warn("Encountered an error on refetching all access control entries, retrying in {} ms : ",
-                                    ZK_RETRY_BACKOFF_MS, cause);
-                        refetchAllAccessControlEntries(ZK_RETRY_BACKOFF_MS);
-                    }
-                });
-            }
-        }, delayMs, TimeUnit.MILLISECONDS);
-    }
-
-    @Override
-    public void process(WatchedEvent event) {
-        if (Event.EventType.None.equals(event.getType())) {
-            if (event.getState() == Event.KeeperState.Expired) {
-                refetchAllAccessControlEntries(0);
-            }
-        } else if (Event.EventType.NodeDataChanged.equals(event.getType())) {
-            logger.info("Default ACL for {} is changed, refetching ...", zkRootPath);
-            refetchDefaultAccessControlEntry(0);
-        } else if (Event.EventType.NodeChildrenChanged.equals(event.getType())) {
-            logger.info("List of ACLs for {} are changed, refetching ...", zkRootPath);
-            refetchAccessControlEntries(0);
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/federated/FederatedZKLogMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
deleted file mode 100644
index 0a8f28b..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
+++ /dev/null
@@ -1,760 +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 com.twitter.distributedlog.impl.federated;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
-import com.google.common.base.Optional;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.exceptions.LogExistsException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.impl.ZKNamespaceWatcher;
-import com.twitter.distributedlog.metadata.LogMetadataStore;
-import com.twitter.distributedlog.namespace.NamespaceWatcher;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.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;
-import org.apache.zookeeper.KeeperException.Code;
-import org.apache.zookeeper.OpResult;
-import org.apache.zookeeper.Transaction;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-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;
-import java.net.URISyntaxException;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.ConcurrentSkipListMap;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * A Federated ZooKeeper Based Log Metadata Store.
- *
- * To Upgrade a simple ZKLogMetadataStore to FederatedZKLogMetadataStore, following steps should be taken in sequence:
- * a) deploy the new code with disabling createStreamsIfNotExists in all writer.
- * b) once all proxies disable the flag, update namespace binding to enable federated namespace.
- * c) restart writers to take federated namespace in place.
- *
- * 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>> {
-
-    static final Logger logger = LoggerFactory.getLogger(FederatedZKLogMetadataStore.class);
-
-    private final static String ZNODE_SUB_NAMESPACES = ".subnamespaces";
-    private final static String SUB_NAMESPACE_PREFIX = "NS_";
-
-    /**
-     * Create the federated namespace.
-     *
-     * @param namespace
-     *          namespace to create
-     * @param zkc
-     *          zookeeper client
-     * @throws InterruptedException
-     * @throws ZooKeeperClient.ZooKeeperConnectionException
-     * @throws KeeperException
-     */
-    public static void createFederatedNamespace(URI namespace, ZooKeeperClient zkc)
-            throws InterruptedException, ZooKeeperClient.ZooKeeperConnectionException, KeeperException {
-        String zkSubNamespacesPath = namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES;
-        Utils.zkCreateFullPathOptimistic(zkc, zkSubNamespacesPath, new byte[0],
-                zkc.getDefaultACL(), CreateMode.PERSISTENT);
-    }
-
-    /**
-     * Represent a sub namespace inside the federated namespace.
-     */
-    class SubNamespace implements NamespaceListener {
-        final URI uri;
-        final ZKNamespaceWatcher watcher;
-        Promise<Set<String>> logsFuture = new Promise<Set<String>>();
-
-        SubNamespace(URI uri) {
-            this.uri = uri;
-            this.watcher = new ZKNamespaceWatcher(conf, uri, zkc, scheduler);
-            this.watcher.registerListener(this);
-        }
-
-        void watch() {
-            this.watcher.watchNamespaceChanges();
-        }
-
-        synchronized Future<Set<String>> getLogs() {
-            return logsFuture;
-        }
-
-        @Override
-        public void onStreamsChanged(Iterator<String> newLogsIter) {
-            Set<String> newLogs = Sets.newHashSet(newLogsIter);
-            Set<String> oldLogs = Sets.newHashSet();
-
-            // update the sub namespace cache
-            Promise<Set<String>> newLogsPromise;
-            synchronized (this) {
-                if (logsFuture.isDefined()) { // the promise is already satisfied
-                    try {
-                        oldLogs = FutureUtils.result(logsFuture);
-                    } catch (IOException e) {
-                        logger.error("Unexpected exception when getting logs from a satisified future of {} : ",
-                                uri, e);
-                    }
-                    logsFuture = new Promise<Set<String>>();
-                }
-
-                // update the reverse cache
-                for (String logName : newLogs) {
-                    URI oldURI = log2Locations.putIfAbsent(logName, uri);
-                    if (null != oldURI && !Objects.equal(uri, oldURI)) {
-                        logger.error("Log {} is found duplicated in multiple locations : old location = {}," +
-                                " new location = {}", new Object[] { logName, oldURI, uri });
-                        duplicatedLogFound.set(true);
-                    }
-                }
-
-                // remove the gone streams
-                Set<String> deletedLogs = Sets.difference(oldLogs, newLogs);
-                for (String logName : deletedLogs) {
-                    log2Locations.remove(logName, uri);
-                }
-                newLogsPromise = logsFuture;
-            }
-            newLogsPromise.setValue(newLogs);
-
-            // notify namespace changes
-            notifyOnNamespaceChanges();
-        }
-    }
-
-    final DistributedLogConfiguration conf;
-    final URI namespace;
-    final ZooKeeperClient zkc;
-    final OrderedScheduler scheduler;
-    final String zkSubnamespacesPath;
-    final AtomicBoolean duplicatedLogFound = new AtomicBoolean(false);
-    final AtomicReference<String> duplicatedLogName = new AtomicReference<String>(null);
-    final AtomicReference<Integer> zkSubnamespacesVersion = new AtomicReference<Integer>(null);
-
-    final int maxLogsPerSubnamespace;
-    // sub namespaces
-    final ConcurrentSkipListMap<URI, SubNamespace> subNamespaces;
-    // map between log name and its location
-    final ConcurrentMap<String, URI> log2Locations;
-    // final
-    final boolean forceCheckLogExistence;
-
-    public FederatedZKLogMetadataStore(
-            DistributedLogConfiguration conf,
-            URI namespace,
-            ZooKeeperClient zkc,
-            OrderedScheduler scheduler) throws IOException {
-        this.conf = conf;
-        this.namespace = namespace;
-        this.zkc = zkc;
-        this.scheduler = scheduler;
-        this.forceCheckLogExistence = conf.getFederatedCheckExistenceWhenCacheMiss();
-        this.subNamespaces = new ConcurrentSkipListMap<URI, SubNamespace>();
-        this.log2Locations = new ConcurrentHashMap<String, URI>();
-        this.zkSubnamespacesPath = namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES;
-        this.maxLogsPerSubnamespace = conf.getFederatedMaxLogsPerSubnamespace();
-
-        // fetch the sub namespace
-        Set<URI> uris = FutureUtils.result(fetchSubNamespaces(this));
-        for (URI uri : uris) {
-            SubNamespace subNs = new SubNamespace(uri);
-            if (null == subNamespaces.putIfAbsent(uri, subNs)) {
-                subNs.watch();
-                logger.info("Watched sub namespace {}", uri);
-            }
-        }
-
-        logger.info("Federated ZK LogMetadataStore is initialized for {}", namespace);
-    }
-
-    private void scheduleTask(Runnable r, long ms) {
-        if (duplicatedLogFound.get()) {
-            logger.error("Scheduler is halted for federated namespace {} as duplicated log found",
-                    namespace);
-            return;
-        }
-        try {
-            scheduler.schedule(r, ms, TimeUnit.MILLISECONDS);
-        } catch (RejectedExecutionException ree) {
-            logger.error("Task {} scheduled in {} ms is rejected : ", new Object[]{r, ms, ree});
-        }
-    }
-
-    private <T> Future<T> postStateCheck(Future<T> future) {
-        final Promise<T> postCheckedPromise = new Promise<T>();
-        future.addEventListener(new FutureEventListener<T>() {
-            @Override
-            public void onSuccess(T value) {
-                if (duplicatedLogFound.get()) {
-                    postCheckedPromise.setException(new UnexpectedException("Duplicate log found under " + namespace));
-                } else {
-                    postCheckedPromise.setValue(value);
-                }
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                postCheckedPromise.setException(cause);
-            }
-        });
-        return postCheckedPromise;
-    }
-
-    //
-    // SubNamespace Related Methods
-    //
-
-    @VisibleForTesting
-    Set<URI> getSubnamespaces() {
-        return subNamespaces.keySet();
-    }
-
-    @VisibleForTesting
-    void removeLogFromCache(String logName) {
-        log2Locations.remove(logName);
-    }
-
-    private URI getSubNamespaceURI(String ns) throws URISyntaxException {
-        return new URI(
-                namespace.getScheme(),
-                namespace.getUserInfo(),
-                namespace.getHost(),
-                namespace.getPort(),
-                namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES + "/" + ns,
-                namespace.getQuery(),
-                namespace.getFragment());
-    }
-
-    Future<Set<URI>> getCachedSubNamespaces() {
-        Set<URI> nsSet = subNamespaces.keySet();
-        return Future.value(nsSet);
-    }
-
-    Future<Set<URI>> fetchSubNamespaces(final Watcher watcher) {
-        final Promise<Set<URI>> promise = new Promise<Set<URI>>();
-        try {
-            zkc.get().sync(this.zkSubnamespacesPath, new AsyncCallback.VoidCallback() {
-                @Override
-                public void processResult(int rc, String path, Object ctx) {
-                    if (Code.OK.intValue() == rc) {
-                        fetchSubNamespaces(watcher, promise);
-                    } else {
-                        promise.setException(KeeperException.create(Code.get(rc)));
-                    }
-                }
-            }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
-        } catch (InterruptedException e) {
-            promise.setException(e);
-        }
-        return promise;
-    }
-
-    private void fetchSubNamespaces(Watcher watcher,
-                                    final Promise<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(
-                                        "The subnamespaces don't exist for the federated namespace " + namespace));
-                            } else if (Code.OK.intValue() == rc) {
-                                Set<URI> subnamespaces = Sets.newHashSet();
-                                subnamespaces.add(namespace);
-                                try {
-                                    for (String ns : children) {
-                                        subnamespaces.add(getSubNamespaceURI(ns));
-                                    }
-                                } catch (URISyntaxException use) {
-                                    logger.error("Invalid sub namespace uri found : ", use);
-                                    promise.setException(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);
-                            }
-                        }
-                    }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
-        } catch (InterruptedException e) {
-            promise.setException(e);
-        }
-    }
-
-    @Override
-    public void run() {
-        fetchSubNamespaces(this).addEventListener(this);
-    }
-
-    @Override
-    public void onSuccess(Set<URI> uris) {
-        for (URI uri : uris) {
-            if (subNamespaces.containsKey(uri)) {
-                continue;
-            }
-            SubNamespace subNs = new SubNamespace(uri);
-            if (null == subNamespaces.putIfAbsent(uri, subNs)) {
-                subNs.watch();
-                logger.info("Watched new sub namespace {}.", uri);
-                notifyOnNamespaceChanges();
-            }
-        }
-    }
-
-    @Override
-    public void onFailure(Throwable cause) {
-        // failed to fetch namespaces, retry later
-        scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
-    }
-
-    @Override
-    public void process(WatchedEvent watchedEvent) {
-        if (Event.EventType.None == watchedEvent.getType() &&
-                Event.KeeperState.Expired == watchedEvent.getState()) {
-            scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
-            return;
-        }
-        if (Event.EventType.NodeChildrenChanged == watchedEvent.getType()) {
-            // fetch the namespace
-            fetchSubNamespaces(this).addEventListener(this);
-        }
-    }
-
-    //
-    // Log Related Methods
-    //
-
-    private <A> Future<A> duplicatedLogException(String logName) {
-        return Future.exception(new UnexpectedException("Duplicated log " + logName
-                + " found in namespace " + namespace));
-    }
-
-    @Override
-    public Future<URI> createLog(final String logName) {
-        if (duplicatedLogFound.get()) {
-            return duplicatedLogException(duplicatedLogName.get());
-        }
-        Promise<URI> createPromise = new Promise<URI>();
-        doCreateLog(logName, createPromise);
-        return postStateCheck(createPromise);
-    }
-
-    void doCreateLog(final String logName, final Promise<URI> createPromise) {
-        getLogLocation(logName).addEventListener(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()));
-                } else {
-                    getCachedSubNamespacesAndCreateLog(logName, createPromise);
-                }
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                createPromise.setException(cause);
-            }
-        });
-    }
-
-    private void getCachedSubNamespacesAndCreateLog(final String logName,
-                                                    final Promise<URI> createPromise) {
-        getCachedSubNamespaces().addEventListener(new FutureEventListener<Set<URI>>() {
-            @Override
-            public void onSuccess(Set<URI> uris) {
-                findSubNamespaceToCreateLog(logName, uris, createPromise);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                createPromise.setException(cause);
-            }
-        });
-    }
-
-    private void fetchSubNamespacesAndCreateLog(final String logName,
-                                                final Promise<URI> createPromise) {
-        fetchSubNamespaces(null).addEventListener(new FutureEventListener<Set<URI>>() {
-            @Override
-            public void onSuccess(Set<URI> uris) {
-                findSubNamespaceToCreateLog(logName, uris, createPromise);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                createPromise.setException(cause);
-            }
-        });
-    }
-
-    private void findSubNamespaceToCreateLog(final String logName,
-                                             final Set<URI> uris,
-                                             final Promise<URI> createPromise) {
-        final List<URI> uriList = Lists.newArrayListWithExpectedSize(uris.size());
-        List<Future<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"));
-                return;
-            }
-            futureList.add(subNs.getLogs());
-            uriList.add(uri);
-        }
-        Future.collect(futureList).addEventListener(new FutureEventListener<List<Set<String>>>() {
-            @Override
-            public void onSuccess(List<Set<String>> resultList) {
-                for (int i = resultList.size() - 1; i >= 0; i--) {
-                    Set<String> logs = resultList.get(i);
-                    if (logs.size() < maxLogsPerSubnamespace) {
-                        URI uri = uriList.get(i);
-                        createLogInNamespace(uri, logName, createPromise);
-                        return;
-                    }
-                }
-                // All sub namespaces are full
-                createSubNamespace().addEventListener(new FutureEventListener<URI>() {
-                    @Override
-                    public void onSuccess(URI uri) {
-                        // the new namespace will be propagated to the namespace cache by the namespace listener
-                        // so we don't need to cache it here. we could go ahead to create the stream under this
-                        // namespace, as we are using sequential znode. we are mostly the first guy who create
-                        // the log under this namespace.
-                        createLogInNamespace(uri, logName, createPromise);
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        createPromise.setException(cause);
-                    }
-                });
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                createPromise.setException(cause);
-            }
-        });
-    }
-
-    private String getNamespaceFromZkPath(String zkPath) throws UnexpectedException {
-        String parts[] = zkPath.split(SUB_NAMESPACE_PREFIX);
-        if (parts.length <= 0) {
-            throw new UnexpectedException("Invalid namespace @ " + zkPath);
-        }
-        return SUB_NAMESPACE_PREFIX + parts[parts.length - 1];
-    }
-
-    Future<URI> createSubNamespace() {
-        final Promise<URI> promise = new Promise<URI>();
-
-        final String nsPath = namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES + "/" + SUB_NAMESPACE_PREFIX;
-        try {
-            zkc.get().create(nsPath, new byte[0], zkc.getDefaultACL(), CreateMode.PERSISTENT_SEQUENTIAL,
-                    new AsyncCallback.StringCallback() {
-                        @Override
-                        public void processResult(int rc, String path, Object ctx, String name) {
-                            if (Code.OK.intValue() == rc) {
-                                try {
-                                    URI newUri = getSubNamespaceURI(getNamespaceFromZkPath(name));
-                                    logger.info("Created sub namespace {}", newUri);
-                                    promise.setValue(newUri);
-                                } catch (UnexpectedException ue) {
-                                    promise.setException(ue);
-                                } catch (URISyntaxException e) {
-                                    promise.setException(new UnexpectedException("Invalid namespace " + name + " is created."));
-                                }
-                            } else {
-                                promise.setException(KeeperException.create(Code.get(rc)));
-                            }
-                        }
-                    }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
-        } catch (InterruptedException e) {
-            promise.setException(e);
-        }
-
-        return promise;
-    }
-
-    /**
-     * Create a log under the namespace. To guarantee there is only one creation happens at time
-     * in a federated namespace, we use CAS operation in zookeeper.
-     *
-     * @param uri
-     *          namespace
-     * @param logName
-     *          name of the log
-     * @param createPromise
-     *          the promise representing the creation result.
-     */
-    private void createLogInNamespace(final URI uri,
-                                      final String logName,
-                                      final Promise<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);
-                } catch (InterruptedException e) {
-                    createPromise.setException(e);
-                } catch (IOException e) {
-                    createPromise.setException(e);
-                } catch (KeeperException.BadVersionException bve) {
-                    fetchSubNamespacesAndCreateLog(logName, createPromise);
-                } catch (KeeperException e) {
-                    createPromise.setException(e);
-                }
-            }
-        });
-    }
-
-    void createLogInNamespaceSync(URI uri, String logName)
-            throws InterruptedException, IOException, KeeperException {
-        Transaction txn = zkc.get().transaction();
-        // we don't have the zk version yet. set it to 0 instead of -1, to prevent non CAS operation.
-        int zkVersion = null == zkSubnamespacesVersion.get() ? 0 : zkSubnamespacesVersion.get();
-        txn.setData(zkSubnamespacesPath, uri.getPath().getBytes(UTF_8), zkVersion);
-        String logPath = uri.getPath() + "/" + logName;
-        txn.create(logPath, new byte[0], zkc.getDefaultACL(), CreateMode.PERSISTENT);
-        try {
-            txn.commit();
-            // if the transaction succeed, the zk version is advanced
-            setZkSubnamespacesVersion(zkVersion + 1);
-        } catch (KeeperException ke) {
-            List<OpResult> opResults = ke.getResults();
-            OpResult createResult = opResults.get(1);
-            if (createResult instanceof OpResult.ErrorResult) {
-                OpResult.ErrorResult errorResult = (OpResult.ErrorResult) createResult;
-                if (Code.NODEEXISTS.intValue() == errorResult.getErr()) {
-                    throw new LogExistsException("Log " + logName + " already exists");
-                }
-            }
-            OpResult setResult = opResults.get(0);
-            if (setResult instanceof OpResult.ErrorResult) {
-                OpResult.ErrorResult errorResult = (OpResult.ErrorResult) setResult;
-                if (Code.BADVERSION.intValue() == errorResult.getErr()) {
-                    throw KeeperException.create(Code.BADVERSION);
-                }
-            }
-            throw new ZKException("ZK exception in creating log " + logName + " in " + uri, ke);
-        }
-    }
-
-    void setZkSubnamespacesVersion(int zkVersion) {
-        Integer oldVersion;
-        boolean done = false;
-        while (!done) {
-            oldVersion = zkSubnamespacesVersion.get();
-            if (null == oldVersion) {
-                done = zkSubnamespacesVersion.compareAndSet(null, zkVersion);
-                continue;
-            }
-            if (oldVersion < zkVersion) {
-                done = zkSubnamespacesVersion.compareAndSet(oldVersion, zkVersion);
-                continue;
-            } else {
-                done = true;
-            }
-        }
-    }
-
-    @Override
-    public Future<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)));
-        }
-        if (!forceCheckLogExistence) {
-            Optional<URI> result = Optional.absent();
-            return Future.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;
-                    }
-                }));
-    }
-
-    private Future<Optional<URI>> fetchLogLocation(final String logName) {
-        final Promise<Optional<URI>> fetchPromise = new Promise<Optional<URI>>();
-
-        Set<URI> uris = subNamespaces.keySet();
-        List<Future<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>>>() {
-            @Override
-            public void onSuccess(List<Optional<URI>> fetchResults) {
-                Optional<URI> result = Optional.absent();
-                for (Optional<URI> fetchResult : fetchResults) {
-                    if (result.isPresent()) {
-                        if (fetchResult.isPresent()) {
-                            logger.error("Log {} is found in multiple sub namespaces : {} & {}.",
-                                    new Object[] { logName, result.get(), fetchResult.get() });
-                            duplicatedLogName.compareAndSet(null, logName);
-                            duplicatedLogFound.set(true);
-                            fetchPromise.setException(new UnexpectedException("Log " + logName
-                                    + " is found in multiple sub namespaces : "
-                                    + result.get() + " & " + fetchResult.get()));
-                            return;
-                        }
-                    } else {
-                        result = fetchResult;
-                    }
-                }
-                fetchPromise.setValue(result);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                fetchPromise.setException(cause);
-            }
-        });
-        return fetchPromise;
-    }
-
-    private Future<Optional<URI>> fetchLogLocation(final URI uri, String logName) {
-        final Promise<Optional<URI>> fetchPromise = new Promise<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));
-                    } else if (Code.NONODE.intValue() == rc) {
-                        fetchPromise.setValue(Optional.<URI>absent());
-                    } else {
-                        fetchPromise.setException(KeeperException.create(Code.get(rc)));
-                    }
-                }
-            }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            fetchPromise.setException(e);
-        } catch (InterruptedException e) {
-            fetchPromise.setException(e);
-        }
-        return fetchPromise;
-    }
-
-    @Override
-    public Future<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);
-                    }
-                }));
-    }
-
-    private Future<List<Set<String>>> retrieveLogs() {
-        Collection<SubNamespace> subNss = subNamespaces.values();
-        List<Future<Set<String>>> logsList = Lists.newArrayListWithExpectedSize(subNss.size());
-        for (SubNamespace subNs : subNss) {
-            logsList.add(subNs.getLogs());
-        }
-        return Future.collect(logsList);
-    }
-
-    private Iterator<String> getIterator(List<Set<String>> resultList) {
-        List<Iterator<String>> iterList = Lists.newArrayListWithExpectedSize(resultList.size());
-        for (Set<String> result : resultList) {
-            iterList.add(result.iterator());
-        }
-        return Iterators.concat(iterList.iterator());
-    }
-
-    @Override
-    public void registerNamespaceListener(NamespaceListener listener) {
-        registerListener(listener);
-    }
-
-    @Override
-    protected void watchNamespaceChanges() {
-        // as the federated namespace already started watching namespace changes,
-        // we don't need to do any actions here
-    }
-
-    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;
-            }
-        });
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentAllocator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
deleted file mode 100644
index d7ff4fb..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
+++ /dev/null
@@ -1,85 +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 com.twitter.distributedlog.impl.logsegment;
-
-import com.twitter.distributedlog.bk.LedgerAllocator;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
-import com.twitter.distributedlog.util.Allocator;
-import com.twitter.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> {
-
-        static final Function1<LedgerHandle, LogSegmentEntryWriter> INSTANCE =
-                new NewLogSegmentEntryWriterFn();
-
-        private NewLogSegmentEntryWriterFn() {}
-
-        @Override
-        public LogSegmentEntryWriter apply(LedgerHandle lh) {
-            return new BKLogSegmentEntryWriter(lh);
-        }
-    }
-
-    LedgerAllocator allocator;
-
-    BKLogSegmentAllocator(LedgerAllocator allocator) {
-        this.allocator = allocator;
-    }
-
-    @Override
-    public void allocate() throws IOException {
-        allocator.allocate();
-    }
-
-    @Override
-    public Future<LogSegmentEntryWriter> tryObtain(Transaction<Object> txn,
-                                                   final Transaction.OpListener<LogSegmentEntryWriter> listener) {
-        return allocator.tryObtain(txn, new Transaction.OpListener<LedgerHandle>() {
-            @Override
-            public void onCommit(LedgerHandle lh) {
-                listener.onCommit(new BKLogSegmentEntryWriter(lh));
-            }
-
-            @Override
-            public void onAbort(Throwable t) {
-                listener.onAbort(t);
-            }
-        }).map(NewLogSegmentEntryWriterFn.INSTANCE);
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        return allocator.asyncClose();
-    }
-
-    @Override
-    public Future<Void> delete() {
-        return allocator.delete();
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
deleted file mode 100644
index f85760d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
+++ /dev/null
@@ -1,837 +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 com.twitter.distributedlog.impl.logsegment;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.Entry;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.EndOfLogSegmentException;
-import com.twitter.distributedlog.exceptions.ReadCancelledException;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryReader;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.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;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Enumeration;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.CopyOnWriteArraySet;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * BookKeeper ledger based log segment entry reader.
- */
-public class BKLogSegmentEntryReader implements Runnable, LogSegmentEntryReader, AsyncCallback.OpenCallback {
-
-    private static final Logger logger = LoggerFactory.getLogger(BKLogSegmentEntryReader.class);
-
-    private class CacheEntry implements Runnable, AsyncCallback.ReadCallback,
-            AsyncCallback.ReadLastConfirmedAndEntryCallback {
-
-        protected final long entryId;
-        private boolean done;
-        private LedgerEntry entry;
-        private int rc;
-
-        private CacheEntry(long entryId) {
-            this.entryId = entryId;
-            this.entry = null;
-            this.rc = BKException.Code.UnexpectedConditionException;
-            this.done = false;
-        }
-
-        long getEntryId() {
-            return entryId;
-        }
-
-        synchronized boolean isDone() {
-            return done;
-        }
-
-        void setValue(LedgerEntry entry) {
-            synchronized (this) {
-                if (done) {
-                    return;
-                }
-                this.rc = BKException.Code.OK;
-                this.entry = entry;
-            }
-            setDone(true);
-        }
-
-        void setException(int rc) {
-            synchronized (this) {
-                if (done) {
-                    return;
-                }
-                this.rc = rc;
-            }
-            setDone(false);
-        }
-
-        void setDone(boolean success) {
-            synchronized (this) {
-                this.done = true;
-            }
-            onReadEntryDone(success);
-        }
-
-        synchronized boolean isSuccess() {
-            return BKException.Code.OK == rc;
-        }
-
-        synchronized LedgerEntry getEntry() {
-            return this.entry;
-        }
-
-        synchronized int getRc() {
-            return rc;
-        }
-
-        @Override
-        public void readComplete(int rc,
-                                 LedgerHandle lh,
-                                 Enumeration<LedgerEntry> entries,
-                                 Object ctx) {
-            if (failureInjector.shouldInjectCorruption(entryId, entryId)) {
-                rc = BKException.Code.DigestMatchException;
-            }
-            processReadEntries(rc, lh, entries, ctx);
-        }
-
-        void processReadEntries(int rc,
-                                LedgerHandle lh,
-                                Enumeration<LedgerEntry> entries,
-                                Object ctx) {
-            if (isDone()) {
-                return;
-            }
-            if (!checkReturnCodeAndHandleFailure(rc, false)) {
-                return;
-            }
-            LedgerEntry entry = null;
-            while (entries.hasMoreElements()) {
-                // more entries are returned
-                if (null != entry) {
-                    setException(BKException.Code.UnexpectedConditionException);
-                    return;
-                }
-                entry = entries.nextElement();
-            }
-            if (null == entry || entry.getEntryId() != entryId) {
-                setException(BKException.Code.UnexpectedConditionException);
-                return;
-            }
-            setValue(entry);
-        }
-
-        @Override
-        public void readLastConfirmedAndEntryComplete(int rc,
-                                                      long entryId,
-                                                      LedgerEntry entry,
-                                                      Object ctx) {
-            if (failureInjector.shouldInjectCorruption(this.entryId, this.entryId)) {
-                rc = BKException.Code.DigestMatchException;
-            }
-            processReadEntry(rc, entryId, entry, ctx);
-        }
-
-        void processReadEntry(int rc,
-                              long entryId,
-                              LedgerEntry entry,
-                              Object ctx) {
-            if (isDone()) {
-                return;
-            }
-            if (!checkReturnCodeAndHandleFailure(rc, true)) {
-                return;
-            }
-            if (null != entry && this.entryId == entryId) {
-                setValue(entry);
-                return;
-            }
-            // the long poll is timeout or interrupted; we will retry it again.
-            issueRead(this);
-        }
-
-        /**
-         * Check return code and retry if needed.
-         *
-         * @param rc the return code
-         * @param isLongPoll is it a long poll request
-         * @return is the request successful or not
-         */
-        boolean checkReturnCodeAndHandleFailure(int rc, boolean isLongPoll) {
-            if (BKException.Code.OK == rc) {
-                numReadErrors.set(0);
-                return true;
-            }
-            if (BKException.Code.BookieHandleNotAvailableException == rc ||
-                    (isLongPoll && BKException.Code.NoSuchLedgerExistsException == rc)) {
-                int numErrors = Math.max(1, numReadErrors.incrementAndGet());
-                int nextReadBackoffTime = Math.min(numErrors * readAheadWaitTime, maxReadBackoffTime);
-                scheduler.schedule(
-                        getSegment().getLogSegmentId(),
-                        this,
-                        nextReadBackoffTime,
-                        TimeUnit.MILLISECONDS);
-            } else {
-                setException(rc);
-            }
-            return false;
-        }
-
-        @Override
-        public void run() {
-            issueRead(this);
-        }
-    }
-
-    private class PendingReadRequest {
-        private final int numEntries;
-        private final List<Entry.Reader> entries;
-        private final Promise<List<Entry.Reader>> promise;
-
-        PendingReadRequest(int numEntries) {
-            this.numEntries = numEntries;
-            if (numEntries == 1) {
-                this.entries = new ArrayList<Entry.Reader>(1);
-            } else {
-                this.entries = new ArrayList<Entry.Reader>();
-            }
-            this.promise = new Promise<List<Entry.Reader>>();
-        }
-
-        Promise<List<Entry.Reader>> getPromise() {
-            return promise;
-        }
-
-        void setException(Throwable throwable) {
-            FutureUtils.setException(promise, throwable);
-        }
-
-        void addEntry(Entry.Reader entry) {
-            entries.add(entry);
-        }
-
-        void complete() {
-            FutureUtils.setValue(promise, entries);
-            onEntriesConsumed(entries.size());
-        }
-
-        boolean hasReadEntries() {
-            return entries.size() > 0;
-        }
-
-        boolean hasReadEnoughEntries() {
-            return entries.size() >= numEntries;
-        }
-    }
-
-    private final BookKeeper bk;
-    private final DistributedLogConfiguration conf;
-    private final OrderedScheduler scheduler;
-    private final long lssn;
-    private final long startSequenceId;
-    private final boolean envelopeEntries;
-    private final boolean deserializeRecordSet;
-    private final int numPrefetchEntries;
-    private final int maxPrefetchEntries;
-    // state
-    private Promise<Void> closePromise = null;
-    private LogSegmentMetadata metadata;
-    private LedgerHandle lh;
-    private final List<LedgerHandle> openLedgerHandles;
-    private CacheEntry outstandingLongPoll;
-    private long nextEntryId;
-    private final AtomicReference<Throwable> lastException = new AtomicReference<Throwable>(null);
-    private final AtomicLong scheduleCount = new AtomicLong(0);
-    private volatile boolean hasCaughtupOnInprogress = false;
-    private final CopyOnWriteArraySet<StateChangeListener> stateChangeListeners =
-            new CopyOnWriteArraySet<StateChangeListener>();
-    // read retries
-    private int readAheadWaitTime;
-    private final int maxReadBackoffTime;
-    private final AtomicInteger numReadErrors = new AtomicInteger(0);
-    private final boolean skipBrokenEntries;
-    // readahead cache
-    int cachedEntries = 0;
-    int numOutstandingEntries = 0;
-    final LinkedBlockingQueue<CacheEntry> readAheadEntries;
-    // request queue
-    final LinkedList<PendingReadRequest> readQueue;
-
-    // failure injector
-    private final AsyncFailureInjector failureInjector;
-    // Stats
-    private final Counter skippedBrokenEntriesCounter;
-
-    BKLogSegmentEntryReader(LogSegmentMetadata metadata,
-                            LedgerHandle lh,
-                            long startEntryId,
-                            BookKeeper bk,
-                            OrderedScheduler scheduler,
-                            DistributedLogConfiguration conf,
-                            StatsLogger statsLogger,
-                            AsyncFailureInjector failureInjector) {
-        this.metadata = metadata;
-        this.lssn = metadata.getLogSegmentSequenceNumber();
-        this.startSequenceId = metadata.getStartSequenceId();
-        this.envelopeEntries = metadata.getEnvelopeEntries();
-        this.deserializeRecordSet = conf.getDeserializeRecordSetOnReads();
-        this.lh = lh;
-        this.nextEntryId = Math.max(startEntryId, 0);
-        this.bk = bk;
-        this.conf = conf;
-        this.numPrefetchEntries = conf.getNumPrefetchEntriesPerLogSegment();
-        this.maxPrefetchEntries = conf.getMaxPrefetchEntriesPerLogSegment();
-        this.scheduler = scheduler;
-        this.openLedgerHandles = Lists.newArrayList();
-        this.openLedgerHandles.add(lh);
-        this.outstandingLongPoll = null;
-        // create the readahead queue
-        this.readAheadEntries = new LinkedBlockingQueue<CacheEntry>();
-        // create the read request queue
-        this.readQueue = new LinkedList<PendingReadRequest>();
-        // read backoff settings
-        this.readAheadWaitTime = conf.getReadAheadWaitTime();
-        this.maxReadBackoffTime = 4 * conf.getReadAheadWaitTime();
-        // other read settings
-        this.skipBrokenEntries = conf.getReadAheadSkipBrokenEntries();
-
-        // Failure Injection
-        this.failureInjector = failureInjector;
-        // Stats
-        this.skippedBrokenEntriesCounter = statsLogger.getCounter("skipped_broken_entries");
-    }
-
-    @VisibleForTesting
-    public synchronized CacheEntry getOutstandingLongPoll() {
-        return outstandingLongPoll;
-    }
-
-    @VisibleForTesting
-    LinkedBlockingQueue<CacheEntry> getReadAheadEntries() {
-        return this.readAheadEntries;
-    }
-
-    synchronized LedgerHandle getLh() {
-        return lh;
-    }
-
-    @Override
-    public synchronized LogSegmentMetadata getSegment() {
-        return metadata;
-    }
-
-    @VisibleForTesting
-    synchronized long getNextEntryId() {
-        return nextEntryId;
-    }
-
-    @Override
-    public void start() {
-        prefetchIfNecessary();
-    }
-
-    @Override
-    public boolean hasCaughtUpOnInprogress() {
-        return hasCaughtupOnInprogress;
-    }
-
-    @Override
-    public LogSegmentEntryReader registerListener(StateChangeListener listener) {
-        stateChangeListeners.add(listener);
-        return this;
-    }
-
-    @Override
-    public LogSegmentEntryReader unregisterListener(StateChangeListener listener) {
-        stateChangeListeners.remove(listener);
-        return this;
-    }
-
-    private void notifyCaughtupOnInprogress() {
-        for (StateChangeListener listener : stateChangeListeners) {
-            listener.onCaughtupOnInprogress();
-        }
-    }
-
-    //
-    // Process on Log Segment Metadata Updates
-    //
-
-    @Override
-    public synchronized void onLogSegmentMetadataUpdated(LogSegmentMetadata segment) {
-        if (metadata == segment ||
-                LogSegmentMetadata.COMPARATOR.compare(metadata, segment) == 0 ||
-                !(metadata.isInProgress() && !segment.isInProgress())) {
-            return;
-        }
-        // segment is closed from inprogress, then re-open the log segment
-        bk.asyncOpenLedger(
-                segment.getLogSegmentId(),
-                BookKeeper.DigestType.CRC32,
-                conf.getBKDigestPW().getBytes(UTF_8),
-                this,
-                segment);
-    }
-
-    @Override
-    public void openComplete(int rc, LedgerHandle lh, Object ctx) {
-        LogSegmentMetadata segment = (LogSegmentMetadata) ctx;
-        if (BKException.Code.OK != rc) {
-            // fail current reader or retry opening the reader
-            failOrRetryOpenLedger(rc, segment);
-            return;
-        }
-        // switch to new ledger handle if the log segment is moved to completed.
-        CacheEntry longPollRead = null;
-        synchronized (this) {
-            if (isClosed()) {
-                lh.asyncClose(new AsyncCallback.CloseCallback() {
-                    @Override
-                    public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
-                        logger.debug("Close the open ledger {} since the log segment reader is already closed",
-                                lh.getId());
-                    }
-                }, null);
-                return;
-            }
-            this.metadata = segment;
-            this.lh = lh;
-            this.openLedgerHandles.add(lh);
-            longPollRead = outstandingLongPoll;
-        }
-        if (null != longPollRead) {
-            // reissue the long poll read when the log segment state is changed
-            issueRead(longPollRead);
-        }
-        // notify readers
-        notifyReaders();
-    }
-
-    private void failOrRetryOpenLedger(int rc, final LogSegmentMetadata segment) {
-        if (isClosed()) {
-            return;
-        }
-        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),
-                    true);
-            return;
-        }
-        // the reader is still catching up, retry opening the log segment later
-        scheduler.schedule(segment.getLogSegmentId(), new Runnable() {
-            @Override
-            public void run() {
-                onLogSegmentMetadataUpdated(segment);
-            }
-        }, conf.getZKRetryBackoffStartMillis(), TimeUnit.MILLISECONDS);
-    }
-
-    //
-    // Change the state of this reader
-    //
-
-    private boolean checkClosedOrInError() {
-        if (null != lastException.get()) {
-            cancelAllPendingReads(lastException.get());
-            return true;
-        }
-        return false;
-    }
-
-    /**
-     * Set the reader into error state with return code <i>rc</i>.
-     *
-     * @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) {
-        lastException.compareAndSet(null, throwable);
-        if (isBackground) {
-            notifyReaders();
-        }
-    }
-
-    /**
-     * Notify the readers with the state change.
-     */
-    private void notifyReaders() {
-        processReadRequests();
-    }
-
-    private void cancelAllPendingReads(Throwable throwExc) {
-        List<PendingReadRequest> requestsToCancel;
-        synchronized (readQueue) {
-            requestsToCancel = Lists.newArrayListWithExpectedSize(readQueue.size());
-            requestsToCancel.addAll(readQueue);
-            readQueue.clear();
-        }
-        for (PendingReadRequest request : requestsToCancel) {
-            request.setException(throwExc);
-        }
-    }
-
-    //
-    // Background Read Operations
-    //
-
-    private void onReadEntryDone(boolean success) {
-        // we successfully read an entry
-        synchronized (this) {
-            --numOutstandingEntries;
-        }
-        // notify reader that there is entry ready
-        notifyReaders();
-        // stop prefetch if we already encountered exceptions
-        if (success) {
-            prefetchIfNecessary();
-        }
-    }
-
-    private void onEntriesConsumed(int numEntries) {
-        synchronized (this) {
-            cachedEntries -= numEntries;
-        }
-        prefetchIfNecessary();
-    }
-
-    private void prefetchIfNecessary() {
-        List<CacheEntry> entriesToFetch;
-        synchronized (this) {
-            if (cachedEntries >= maxPrefetchEntries) {
-                return;
-            }
-            // we don't have enough entries, do prefetch
-            int numEntriesToFetch = numPrefetchEntries - numOutstandingEntries;
-            if (numEntriesToFetch <= 0) {
-                return;
-            }
-            entriesToFetch = new ArrayList<CacheEntry>(numEntriesToFetch);
-            for (int i = 0; i < numEntriesToFetch; i++) {
-                if (cachedEntries >= maxPrefetchEntries) {
-                    break;
-                }
-                if ((isLedgerClosed() && nextEntryId > getLastAddConfirmed()) ||
-                        (!isLedgerClosed() && nextEntryId > getLastAddConfirmed() + 1)) {
-                    break;
-                }
-                CacheEntry entry = new CacheEntry(nextEntryId);
-                entriesToFetch.add(entry);
-                readAheadEntries.add(entry);
-                ++numOutstandingEntries;
-                ++cachedEntries;
-                ++nextEntryId;
-            }
-        }
-        for (CacheEntry entry : entriesToFetch) {
-            issueRead(entry);
-        }
-    }
-
-
-    private void issueRead(CacheEntry cacheEntry) {
-        if (isClosed()) {
-            return;
-        }
-        if (isLedgerClosed()) {
-            if (isNotBeyondLastAddConfirmed(cacheEntry.getEntryId())) {
-                issueSimpleRead(cacheEntry);
-                return;
-            } else {
-                // Reach the end of stream
-                notifyReaders();
-            }
-        } else { // the ledger is still in progress
-            if (isNotBeyondLastAddConfirmed(cacheEntry.getEntryId())) {
-                issueSimpleRead(cacheEntry);
-            } else {
-                issueLongPollRead(cacheEntry);
-            }
-        }
-    }
-
-    private void issueSimpleRead(CacheEntry cacheEntry) {
-        getLh().asyncReadEntries(cacheEntry.entryId, cacheEntry.entryId, cacheEntry, null);
-    }
-
-    private void issueLongPollRead(CacheEntry cacheEntry) {
-        // register the read as outstanding reads
-        synchronized (this) {
-            this.outstandingLongPoll = cacheEntry;
-        }
-
-        if (!hasCaughtupOnInprogress) {
-            hasCaughtupOnInprogress = true;
-            notifyCaughtupOnInprogress();
-        }
-        getLh().asyncReadLastConfirmedAndEntry(
-                cacheEntry.entryId,
-                conf.getReadLACLongPollTimeout(),
-                false,
-                cacheEntry,
-                null);
-    }
-
-    //
-    // Foreground Read Operations
-    //
-
-    Entry.Reader processReadEntry(LedgerEntry entry) throws IOException {
-        return Entry.newBuilder()
-                .setLogSegmentInfo(lssn, startSequenceId)
-                .setEntryId(entry.getEntryId())
-                .setEnvelopeEntry(envelopeEntries)
-                .deserializeRecordSet(deserializeRecordSet)
-                .setInputStream(entry.getEntryInputStream())
-                .buildReader();
-    }
-
-    @Override
-    public Future<List<Entry.Reader>> readNext(int numEntries) {
-        final PendingReadRequest readRequest = new PendingReadRequest(numEntries);
-
-        if (checkClosedOrInError()) {
-            readRequest.setException(lastException.get());
-        } else {
-            boolean wasQueueEmpty;
-            synchronized (readQueue) {
-                wasQueueEmpty = readQueue.isEmpty();
-                readQueue.add(readRequest);
-            }
-            if (wasQueueEmpty) {
-                processReadRequests();
-            }
-        }
-        return readRequest.getPromise();
-    }
-
-    private void processReadRequests() {
-        if (isClosed()) {
-            // the reader is already closed.
-            return;
-        }
-
-        long prevCount = scheduleCount.getAndIncrement();
-        if (0 == prevCount) {
-            scheduler.submit(getSegment().getLogSegmentId(), this);
-        }
-    }
-
-    /**
-     * The core function to propagate fetched entries to read requests
-     */
-    @Override
-    public void run() {
-        long scheduleCountLocal = scheduleCount.get();
-        while (true) {
-            PendingReadRequest nextRequest = null;
-            synchronized (readQueue) {
-                nextRequest = readQueue.peek();
-            }
-
-            // if read queue is empty, nothing to read, return
-            if (null == nextRequest) {
-                scheduleCount.set(0L);
-                return;
-            }
-
-            // if the oldest pending promise is interrupted then we must
-            // 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 the reader is in error state, stop read
-            if (checkClosedOrInError()) {
-                return;
-            }
-
-            // read entries from readahead cache to satisfy next read request
-            readEntriesFromReadAheadCache(nextRequest);
-
-            // check if we can satisfy the read request
-            if (nextRequest.hasReadEntries()) {
-                PendingReadRequest request;
-                synchronized (readQueue) {
-                    request = readQueue.poll();
-                }
-                if (null != request && nextRequest == request) {
-                    request.complete();
-                } else {
-                    DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at reading from "
-                            + getSegment());
-                    nextRequest.setException(ise);
-                    if (null != request) {
-                        request.setException(ise);
-                    }
-                    setException(ise, false);
-                }
-            } else {
-                if (0 == scheduleCountLocal) {
-                    return;
-                }
-                scheduleCountLocal = scheduleCount.decrementAndGet();
-            }
-        }
-    }
-
-    private void readEntriesFromReadAheadCache(PendingReadRequest nextRequest) {
-        while (!nextRequest.hasReadEnoughEntries()) {
-            CacheEntry entry;
-            boolean hitEndOfLogSegment;
-            synchronized (this) {
-                entry = readAheadEntries.peek();
-                hitEndOfLogSegment = (null == entry) && isEndOfLogSegment();
-            }
-            // reach end of log segment
-            if (hitEndOfLogSegment) {
-                setException(new EndOfLogSegmentException(getSegment().getZNodeName()), false);
-                return;
-            }
-            if (null == entry) {
-                return;
-            }
-            // entry is not complete yet.
-            if (!entry.isDone()) {
-                // we already reached end of the log segment
-                if (isEndOfLogSegment(entry.getEntryId())) {
-                    setException(new EndOfLogSegmentException(getSegment().getZNodeName()), false);
-                }
-                return;
-            }
-            if (entry.isSuccess()) {
-                CacheEntry removedEntry = readAheadEntries.poll();
-                if (entry != removedEntry) {
-                    DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at reading from "
-                            + getSegment());
-                    setException(ise, false);
-                    return;
-                }
-                try {
-                    nextRequest.addEntry(processReadEntry(entry.getEntry()));
-                } catch (IOException e) {
-                    setException(e, false);
-                    return;
-                }
-            } else if (skipBrokenEntries && BKException.Code.DigestMatchException == entry.getRc()) {
-                // skip this entry and move forward
-                skippedBrokenEntriesCounter.inc();
-                readAheadEntries.poll();
-                continue;
-            } else {
-                setException(new BKTransmitException("Encountered issue on reading entry " + entry.getEntryId()
-                        + " @ log segment " + getSegment(), entry.getRc()), false);
-                return;
-            }
-        }
-    }
-
-    //
-    // State Management
-    //
-
-    private synchronized boolean isEndOfLogSegment() {
-        return isEndOfLogSegment(nextEntryId);
-    }
-
-    private boolean isEndOfLogSegment(long entryId) {
-        return isLedgerClosed() && entryId > getLastAddConfirmed();
-    }
-
-    @Override
-    public synchronized boolean isBeyondLastAddConfirmed() {
-        return isBeyondLastAddConfirmed(nextEntryId);
-    }
-
-    private boolean isBeyondLastAddConfirmed(long entryId) {
-        return entryId > getLastAddConfirmed();
-    }
-
-    private boolean isNotBeyondLastAddConfirmed(long entryId) {
-        return entryId <= getLastAddConfirmed();
-    }
-
-    private boolean isLedgerClosed() {
-        return getLh().isClosed();
-    }
-
-    @Override
-    public long getLastAddConfirmed() {
-        return getLh().getLastAddConfirmed();
-    }
-
-    synchronized boolean isClosed() {
-        return null != closePromise;
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        final Promise<Void> closeFuture;
-        ReadCancelledException exception;
-        LedgerHandle[] lhsToClose;
-        synchronized (this) {
-            if (null != closePromise) {
-                return closePromise;
-            }
-            closeFuture = closePromise = new Promise<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);
-        }
-
-        // cancel all pending reads
-        cancelAllPendingReads(exception);
-
-        // close all the open ledger
-        BKUtils.closeLedgers(lhsToClose).proxyTo(closeFuture);
-        return closeFuture;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
deleted file mode 100644
index 91e6dec..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
+++ /dev/null
@@ -1,286 +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 com.twitter.distributedlog.impl.logsegment;
-
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.bk.DynamicQuorumConfigProvider;
-import com.twitter.distributedlog.bk.LedgerAllocator;
-import com.twitter.distributedlog.bk.LedgerAllocatorDelegator;
-import com.twitter.distributedlog.bk.QuorumConfigProvider;
-import com.twitter.distributedlog.bk.SimpleLedgerAllocator;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryReader;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
-import com.twitter.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.util.Allocator;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.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;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * BookKeeper Based Entry Store
- */
-public class BKLogSegmentEntryStore implements
-        LogSegmentEntryStore,
-        AsyncCallback.OpenCallback,
-        AsyncCallback.DeleteCallback {
-
-    private static final Logger logger = LoggerFactory.getLogger(BKLogSegmentEntryReader.class);
-
-    private static class OpenReaderRequest {
-
-        private final LogSegmentMetadata segment;
-        private final long startEntryId;
-        private final Promise<LogSegmentEntryReader> openPromise;
-
-        OpenReaderRequest(LogSegmentMetadata segment,
-                          long startEntryId) {
-            this.segment = segment;
-            this.startEntryId = startEntryId;
-            this.openPromise = new Promise<LogSegmentEntryReader>();
-        }
-
-    }
-
-    private static class DeleteLogSegmentRequest {
-
-        private final LogSegmentMetadata segment;
-        private final Promise<LogSegmentMetadata> deletePromise;
-
-        DeleteLogSegmentRequest(LogSegmentMetadata segment) {
-            this.segment = segment;
-            this.deletePromise = new Promise<LogSegmentMetadata>();
-        }
-
-    }
-
-    private final byte[] passwd;
-    private final ZooKeeperClient zkc;
-    private final BookKeeperClient bkc;
-    private final OrderedScheduler scheduler;
-    private final DistributedLogConfiguration conf;
-    private final DynamicDistributedLogConfiguration dynConf;
-    private final StatsLogger statsLogger;
-    private final AsyncFailureInjector failureInjector;
-    // ledger allocator
-    private final LedgerAllocator allocator;
-
-    public BKLogSegmentEntryStore(DistributedLogConfiguration conf,
-                                  DynamicDistributedLogConfiguration dynConf,
-                                  ZooKeeperClient zkc,
-                                  BookKeeperClient bkc,
-                                  OrderedScheduler scheduler,
-                                  LedgerAllocator allocator,
-                                  StatsLogger statsLogger,
-                                  AsyncFailureInjector failureInjector) {
-        this.conf = conf;
-        this.dynConf = dynConf;
-        this.zkc = zkc;
-        this.bkc = bkc;
-        this.passwd = conf.getBKDigestPW().getBytes(UTF_8);
-        this.scheduler = scheduler;
-        this.allocator = allocator;
-        this.statsLogger = statsLogger;
-        this.failureInjector = failureInjector;
-    }
-
-    @Override
-    public Future<LogSegmentMetadata> deleteLogSegment(LogSegmentMetadata segment) {
-        DeleteLogSegmentRequest request = new DeleteLogSegmentRequest(segment);
-        BookKeeper bk;
-        try {
-            bk = this.bkc.get();
-        } catch (IOException e) {
-            return Future.exception(e);
-        }
-        bk.asyncDeleteLedger(segment.getLogSegmentId(), this, request);
-        return request.deletePromise;
-    }
-
-    @Override
-    public void deleteComplete(int rc, Object ctx) {
-        DeleteLogSegmentRequest deleteRequest = (DeleteLogSegmentRequest) ctx;
-        if (BKException.Code.NoSuchLedgerExistsException == rc) {
-            logger.warn("No ledger {} found to delete for {}.",
-                    deleteRequest.segment.getLogSegmentId(), deleteRequest.segment);
-        } else if (BKException.Code.OK != rc) {
-            logger.error("Couldn't delete ledger {} from bookkeeper for {} : {}",
-                    new Object[]{ deleteRequest.segment.getLogSegmentId(), deleteRequest.segment,
-                            BKException.getMessage(rc) });
-            FutureUtils.setException(deleteRequest.deletePromise,
-                    new BKTransmitException("Couldn't delete log segment " + deleteRequest.segment, rc));
-            return;
-        }
-        FutureUtils.setValue(deleteRequest.deletePromise, deleteRequest.segment);
-    }
-
-    //
-    // Writers
-    //
-
-    LedgerAllocator createLedgerAllocator(LogMetadataForWriter logMetadata,
-                                          DynamicDistributedLogConfiguration dynConf)
-            throws IOException {
-        LedgerAllocator ledgerAllocatorDelegator;
-        if (null == allocator || !dynConf.getEnableLedgerAllocatorPool()) {
-            QuorumConfigProvider quorumConfigProvider =
-                    new DynamicQuorumConfigProvider(dynConf);
-            LedgerAllocator allocator = new SimpleLedgerAllocator(
-                    logMetadata.getAllocationPath(),
-                    logMetadata.getAllocationData(),
-                    quorumConfigProvider,
-                    zkc,
-                    bkc);
-            ledgerAllocatorDelegator = new LedgerAllocatorDelegator(allocator, true);
-        } else {
-            ledgerAllocatorDelegator = allocator;
-        }
-        return ledgerAllocatorDelegator;
-    }
-
-    @Override
-    public Allocator<LogSegmentEntryWriter, Object> newLogSegmentAllocator(
-            LogMetadataForWriter logMetadata,
-            DynamicDistributedLogConfiguration dynConf) throws IOException {
-        // Build the ledger allocator
-        LedgerAllocator allocator = createLedgerAllocator(logMetadata, dynConf);
-        return new BKLogSegmentAllocator(allocator);
-    }
-
-    //
-    // Readers
-    //
-
-    @Override
-    public Future<LogSegmentEntryReader> openReader(LogSegmentMetadata segment,
-                                                    long startEntryId) {
-        BookKeeper bk;
-        try {
-            bk = this.bkc.get();
-        } catch (IOException e) {
-            return Future.exception(e);
-        }
-        OpenReaderRequest request = new OpenReaderRequest(segment, startEntryId);
-        if (segment.isInProgress()) {
-            bk.asyncOpenLedgerNoRecovery(
-                    segment.getLogSegmentId(),
-                    BookKeeper.DigestType.CRC32,
-                    passwd,
-                    this,
-                    request);
-        } else {
-            bk.asyncOpenLedger(
-                    segment.getLogSegmentId(),
-                    BookKeeper.DigestType.CRC32,
-                    passwd,
-                    this,
-                    request);
-        }
-        return request.openPromise;
-    }
-
-    @Override
-    public void openComplete(int rc, LedgerHandle lh, Object ctx) {
-        OpenReaderRequest request = (OpenReaderRequest) ctx;
-        if (BKException.Code.OK != rc) {
-            FutureUtils.setException(
-                    request.openPromise,
-                    new BKTransmitException("Failed to open ledger handle for log segment " + request.segment, rc));
-            return;
-        }
-        // successfully open a ledger
-        try {
-            LogSegmentEntryReader reader = new BKLogSegmentEntryReader(
-                    request.segment,
-                    lh,
-                    request.startEntryId,
-                    bkc.get(),
-                    scheduler,
-                    conf,
-                    statsLogger,
-                    failureInjector);
-            FutureUtils.setValue(request.openPromise, reader);
-        } catch (IOException e) {
-            FutureUtils.setException(request.openPromise, e);
-        }
-
-    }
-
-    @Override
-    public Future<LogSegmentRandomAccessEntryReader> openRandomAccessReader(final LogSegmentMetadata segment,
-                                                                            final boolean fence) {
-        final BookKeeper bk;
-        try {
-            bk = this.bkc.get();
-        } catch (IOException e) {
-            return Future.exception(e);
-        }
-        final Promise<LogSegmentRandomAccessEntryReader> openPromise = new Promise<LogSegmentRandomAccessEntryReader>();
-        AsyncCallback.OpenCallback openCallback = new AsyncCallback.OpenCallback() {
-            @Override
-            public void openComplete(int rc, LedgerHandle lh, Object ctx) {
-                if (BKException.Code.OK != rc) {
-                    FutureUtils.setException(
-                            openPromise,
-                            new BKTransmitException("Failed to open ledger handle for log segment " + segment, rc));
-                    return;
-                }
-                LogSegmentRandomAccessEntryReader reader = new BKLogSegmentRandomAccessEntryReader(
-                        segment,
-                        lh,
-                        conf);
-                FutureUtils.setValue(openPromise, reader);
-            }
-        };
-        if (segment.isInProgress() && !fence) {
-            bk.asyncOpenLedgerNoRecovery(
-                    segment.getLogSegmentId(),
-                    BookKeeper.DigestType.CRC32,
-                    passwd,
-                    openCallback,
-                    null);
-        } else {
-            bk.asyncOpenLedger(
-                    segment.getLogSegmentId(),
-                    BookKeeper.DigestType.CRC32,
-                    passwd,
-                    openCallback,
-                    null);
-        }
-        return openPromise;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java
deleted file mode 100644
index 34fe1c3..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java
+++ /dev/null
@@ -1,61 +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 com.twitter.distributedlog.impl.logsegment;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
-import org.apache.bookkeeper.client.AsyncCallback;
-import org.apache.bookkeeper.client.LedgerHandle;
-
-/**
- * Ledger based log segment entry writer.
- */
-public class BKLogSegmentEntryWriter implements LogSegmentEntryWriter {
-
-    private final LedgerHandle lh;
-
-    public BKLogSegmentEntryWriter(LedgerHandle lh) {
-        this.lh = lh;
-    }
-
-    @VisibleForTesting
-    public LedgerHandle getLedgerHandle() {
-        return this.lh;
-    }
-
-    @Override
-    public long getLogSegmentId() {
-        return lh.getId();
-    }
-
-    @Override
-    public void asyncClose(AsyncCallback.CloseCallback callback, Object ctx) {
-        lh.asyncClose(callback, ctx);
-    }
-
-    @Override
-    public void asyncAddEntry(byte[] data, int offset, int length,
-                              AsyncCallback.AddCallback callback, Object ctx) {
-        lh.asyncAddEntry(data, offset, length, callback, ctx);
-    }
-
-    @Override
-    public long size() {
-        return lh.getLength();
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
deleted file mode 100644
index 9cec80c..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
+++ /dev/null
@@ -1,119 +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 com.twitter.distributedlog.impl.logsegment;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.Entry;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.client.AsyncCallback.ReadCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-
-import java.io.IOException;
-import java.util.Enumeration;
-import java.util.List;
-
-/**
- * BookKeeper ledger based random access entry reader.
- */
-class BKLogSegmentRandomAccessEntryReader implements
-        LogSegmentRandomAccessEntryReader,
-        ReadCallback {
-
-    private final long lssn;
-    private final long startSequenceId;
-    private final boolean envelopeEntries;
-    private final boolean deserializeRecordSet;
-    // state
-    private final LogSegmentMetadata metadata;
-    private final LedgerHandle lh;
-    private Promise<Void> closePromise = null;
-
-    BKLogSegmentRandomAccessEntryReader(LogSegmentMetadata metadata,
-                                        LedgerHandle lh,
-                                        DistributedLogConfiguration conf) {
-        this.metadata = metadata;
-        this.lssn = metadata.getLogSegmentSequenceNumber();
-        this.startSequenceId = metadata.getStartSequenceId();
-        this.envelopeEntries = metadata.getEnvelopeEntries();
-        this.deserializeRecordSet = conf.getDeserializeRecordSetOnReads();
-        this.lh = lh;
-    }
-
-    @Override
-    public long getLastAddConfirmed() {
-        return lh.getLastAddConfirmed();
-    }
-
-    @Override
-    public Future<List<Entry.Reader>> readEntries(long startEntryId, long endEntryId) {
-        Promise<List<Entry.Reader>> promise = new Promise<List<Entry.Reader>>();
-        lh.asyncReadEntries(startEntryId, endEntryId, this, promise);
-        return promise;
-    }
-
-    Entry.Reader processReadEntry(LedgerEntry entry) throws IOException {
-        return Entry.newBuilder()
-                .setLogSegmentInfo(lssn, startSequenceId)
-                .setEntryId(entry.getEntryId())
-                .setEnvelopeEntry(envelopeEntries)
-                .deserializeRecordSet(deserializeRecordSet)
-                .setInputStream(entry.getEntryInputStream())
-                .buildReader();
-    }
-
-    @Override
-    public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> entries, Object ctx) {
-        Promise<List<Entry.Reader>> promise = (Promise<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);
-                    return;
-                }
-            }
-            FutureUtils.setValue(promise, entryList);
-        } else {
-            FutureUtils.setException(promise,
-                    new BKTransmitException("Failed to read entries :", rc));
-        }
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        final Promise<Void> closeFuture;
-        synchronized (this) {
-            if (null != closePromise) {
-                return closePromise;
-            }
-            closeFuture = closePromise = new Promise<Void>();
-        }
-        BKUtils.closeLedgers(lh).proxyTo(closeFuture);
-        return closeFuture;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKUtils.java
deleted file mode 100644
index c71c67e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKUtils.java
+++ /dev/null
@@ -1,72 +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 com.twitter.distributedlog.impl.logsegment;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.function.VoidFunctions;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Future;
-import com.twitter.util.Futures;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.client.AsyncCallback;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.LedgerHandle;
-
-import java.util.List;
-
-/**
- * BookKeeper Util Functions
- */
-public class BKUtils {
-
-    /**
-     * Close a ledger <i>lh</i>.
-     *
-     * @param lh ledger handle
-     * @return future represents close result.
-     */
-    public static Future<Void> closeLedger(LedgerHandle lh) {
-        final Promise<Void> closePromise = new Promise<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));
-                } else {
-                    FutureUtils.setValue(closePromise, null);
-                }
-            }
-        }, null);
-        return closePromise;
-    }
-
-    /**
-     * Close a list of ledgers <i>lhs</i>.
-     *
-     * @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);
-        for (LedgerHandle lh : lhs) {
-            closeResults.add(closeLedger(lh));
-        }
-        return Futures.collect(closeResults).map(VoidFunctions.LIST_TO_VOID_FUNC);
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java
deleted file mode 100644
index 3e859fb..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java
+++ /dev/null
@@ -1,400 +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 com.twitter.distributedlog.impl.metadata;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Objects;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.metadata.DLConfig;
-import com.twitter.distributedlog.thrift.BKDLConfigFormat;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TJSONProtocol;
-import org.apache.thrift.transport.TMemoryBuffer;
-import org.apache.thrift.transport.TMemoryInputTransport;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.net.URI;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Configurations for BookKeeper based DL.
- */
-public class BKDLConfig implements DLConfig {
-
-    private static final Logger LOG = LoggerFactory.getLogger(BKDLConfig.class);
-
-    private static final int BUFFER_SIZE = 4096;
-    private static final ConcurrentMap<URI, DLConfig> cachedDLConfigs =
-            new ConcurrentHashMap<URI, DLConfig>();
-
-    public static void propagateConfiguration(BKDLConfig bkdlConfig, DistributedLogConfiguration dlConf) {
-        dlConf.setEncodeRegionIDInLogSegmentMetadata(bkdlConfig.getEncodeRegionID());
-        dlConf.setFirstLogSegmentSequenceNumber(bkdlConfig.getFirstLogSegmentSeqNo());
-        if (bkdlConfig.isFederatedNamespace()) {
-            dlConf.setCreateStreamIfNotExists(false);
-            LOG.info("Disabled createIfNotExists for federated namespace.");
-        }
-        LOG.info("Propagate BKDLConfig to DLConfig : encodeRegionID = {}," +
-                        " firstLogSegmentSequenceNumber = {}, createStreamIfNotExists = {}, isFederated = {}.",
-                new Object[] { dlConf.getEncodeRegionIDInLogSegmentMetadata(),
-                        dlConf.getFirstLogSegmentSequenceNumber(), dlConf.getCreateStreamIfNotExists(),
-                        bkdlConfig.isFederatedNamespace() });
-    }
-
-    public static BKDLConfig resolveDLConfig(ZooKeeperClient zkc, URI uri) throws IOException {
-        DLConfig dlConfig = cachedDLConfigs.get(uri);
-        if (dlConfig == null) {
-            dlConfig = (new ZkMetadataResolver(zkc).resolve(uri)).getDLConfig();
-            DLConfig oldDLConfig = cachedDLConfigs.putIfAbsent(uri, dlConfig);
-            if (null != oldDLConfig) {
-                dlConfig = oldDLConfig;
-            }
-        }
-        assert (dlConfig instanceof BKDLConfig);
-        return (BKDLConfig)dlConfig;
-    }
-
-    @VisibleForTesting
-    public static void clearCachedDLConfigs() {
-        cachedDLConfigs.clear();
-    }
-
-    private String bkZkServersForWriter;
-    private String bkZkServersForReader;
-    private String bkLedgersPath;
-    private boolean sanityCheckTxnID = true;
-    private boolean encodeRegionID = false;
-    private String dlZkServersForWriter;
-    private String dlZkServersForReader;
-    private String aclRootPath;
-    private Long firstLogSegmentSeqNo;
-    private boolean isFederatedNamespace = false;
-
-    /**
-     * Construct a empty config with given <i>uri</i>.
-     */
-    public BKDLConfig(URI uri) {
-        this(BKNamespaceDriver.getZKServersFromDLUri(uri),
-             BKNamespaceDriver.getZKServersFromDLUri(uri),
-             null, null, null);
-    }
-
-    /**
-     * The caller should make sure both dl and bk use same zookeeper server.
-     *
-     * @param zkServers
-     *          zk servers used for both dl and bk.
-     * @param ledgersPath
-     *          ledgers path.
-     */
-    @VisibleForTesting
-    public BKDLConfig(String zkServers, String ledgersPath) {
-        this(zkServers, zkServers, zkServers, zkServers, ledgersPath);
-    }
-
-    public BKDLConfig(String dlZkServersForWriter,
-                      String dlZkServersForReader,
-                      String bkZkServersForWriter,
-                      String bkZkServersForReader,
-                      String bkLedgersPath) {
-        this.dlZkServersForWriter = dlZkServersForWriter;
-        this.dlZkServersForReader = dlZkServersForReader;
-        this.bkZkServersForWriter = bkZkServersForWriter;
-        this.bkZkServersForReader = bkZkServersForReader;
-        this.bkLedgersPath = bkLedgersPath;
-    }
-
-    /**
-     * @return zk servers used for bk for writers
-     */
-    public String getBkZkServersForWriter() {
-        return bkZkServersForWriter;
-    }
-
-    /**
-     * @return zk servers used for bk for readers
-     */
-    public String getBkZkServersForReader() {
-        return bkZkServersForReader;
-    }
-
-    /**
-     * @return zk servers used for dl for writers
-     */
-    public String getDlZkServersForWriter() {
-        return dlZkServersForWriter;
-    }
-
-    /**
-     * @return zk servers used for dl for readers
-     */
-    public String getDlZkServersForReader() {
-        return dlZkServersForReader;
-    }
-
-    /**
-     * @return ledgers path for bk
-     */
-    public String getBkLedgersPath() {
-        return bkLedgersPath;
-    }
-
-    /**
-     * Enable/Disable sanity check txn id.
-     *
-     * @param enabled
-     *          flag to enable/disable sanity check txn id.
-     * @return bk dl config.
-     */
-    public BKDLConfig setSanityCheckTxnID(boolean enabled) {
-        this.sanityCheckTxnID = enabled;
-        return this;
-    }
-
-    /**
-     * @return flag to sanity check highest txn id.
-     */
-    public boolean getSanityCheckTxnID() {
-        return sanityCheckTxnID;
-    }
-
-    /**
-     * Enable/Disable encode region id.
-     *
-     * @param enabled
-     *          flag to enable/disable encoding region id.
-     * @return bk dl config
-     */
-    public BKDLConfig setEncodeRegionID(boolean enabled) {
-        this.encodeRegionID = enabled;
-        return this;
-    }
-
-    /**
-     * @return flag to encode region id.
-     */
-    public boolean getEncodeRegionID() {
-        return encodeRegionID;
-    }
-
-    /**
-     * Set the root path of zk based ACL manager.
-     *
-     * @param aclRootPath
-     *          root path of zk based ACL manager.
-     * @return bk dl config
-     */
-    public BKDLConfig setACLRootPath(String aclRootPath) {
-        this.aclRootPath = aclRootPath;
-        return this;
-    }
-
-    /**
-     * Get the root path of zk based ACL manager.
-     *
-     * @return root path of zk based ACL manager.
-     */
-    public String getACLRootPath() {
-        return aclRootPath;
-    }
-
-    /**
-     * Set the value at which ledger sequence number should start for streams that are being
-     * upgraded and did not have ledger sequence number to start with or for newly created
-     * streams
-     *
-     * @param firstLogSegmentSeqNo first ledger sequence number
-     * @return bk dl config
-     */
-    public BKDLConfig setFirstLogSegmentSeqNo(long firstLogSegmentSeqNo) {
-        this.firstLogSegmentSeqNo = firstLogSegmentSeqNo;
-        return this;
-    }
-
-    /**
-     * Get the value at which ledger sequence number should start for streams that are being
-     * upgraded and did not have ledger sequence number to start with or for newly created
-     * streams
-     *
-     * @return first ledger sequence number
-     */
-    public Long getFirstLogSegmentSeqNo() {
-        if (null == firstLogSegmentSeqNo) {
-            return DistributedLogConstants.FIRST_LOGSEGMENT_SEQNO;
-        }
-        return firstLogSegmentSeqNo;
-    }
-
-    /**
-     * Set the namespace to federated <i>isFederatedNamespace</i>.
-     *
-     * @param isFederatedNamespace
-     *          is the namespace federated?
-     * @return bk dl config
-     */
-    public BKDLConfig setFederatedNamespace(boolean isFederatedNamespace) {
-        this.isFederatedNamespace = isFederatedNamespace;
-        return this;
-    }
-
-    /**
-     * Whether the namespace is federated namespace
-     *
-     * @return true if the namespace is a federated namespace. otherwise false.
-     */
-    public boolean isFederatedNamespace() {
-        return this.isFederatedNamespace;
-    }
-
-    @Override
-    public int hashCode() {
-        return Objects.hashCode(bkZkServersForWriter, bkZkServersForReader,
-                                dlZkServersForWriter, dlZkServersForReader,
-                                bkLedgersPath);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof BKDLConfig)) {
-            return false;
-        }
-        BKDLConfig another = (BKDLConfig) o;
-        return Objects.equal(bkZkServersForWriter, another.bkZkServersForWriter) &&
-               Objects.equal(bkZkServersForReader, another.bkZkServersForReader) &&
-               Objects.equal(dlZkServersForWriter, another.dlZkServersForWriter) &&
-               Objects.equal(dlZkServersForReader, another.dlZkServersForReader) &&
-               Objects.equal(bkLedgersPath, another.bkLedgersPath) &&
-               sanityCheckTxnID == another.sanityCheckTxnID &&
-               encodeRegionID == another.encodeRegionID &&
-               Objects.equal(aclRootPath, another.aclRootPath) &&
-               Objects.equal(firstLogSegmentSeqNo, another.firstLogSegmentSeqNo) &&
-               Objects.equal(isFederatedNamespace, another.isFederatedNamespace);
-
-    }
-
-    @Override
-    public String toString() {
-        return serialize();
-    }
-
-    @Override
-    public String serialize() {
-        BKDLConfigFormat configFormat = new BKDLConfigFormat();
-        if (null != bkZkServersForWriter) {
-            configFormat.setBkZkServers(bkZkServersForWriter);
-        }
-        if (null != bkZkServersForReader) {
-            configFormat.setBkZkServersForReader(bkZkServersForReader);
-        }
-        if (null != dlZkServersForWriter) {
-            configFormat.setDlZkServersForWriter(dlZkServersForWriter);
-        }
-        if (null != dlZkServersForReader) {
-            configFormat.setDlZkServersForReader(dlZkServersForReader);
-        }
-        if (null != bkLedgersPath) {
-            configFormat.setBkLedgersPath(bkLedgersPath);
-        }
-        configFormat.setSanityCheckTxnID(sanityCheckTxnID);
-        configFormat.setEncodeRegionID(encodeRegionID);
-        if (null != aclRootPath) {
-            configFormat.setAclRootPath(aclRootPath);
-        }
-        if (null != firstLogSegmentSeqNo) {
-            configFormat.setFirstLogSegmentSeqNo(firstLogSegmentSeqNo);
-        }
-        if (isFederatedNamespace) {
-            configFormat.setFederatedNamespace(true);
-        }
-        return serialize(configFormat);
-    }
-
-    String serialize(BKDLConfigFormat configFormat) {
-        TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
-        TJSONProtocol protocol = new TJSONProtocol(transport);
-        try {
-            configFormat.write(protocol);
-            transport.flush();
-            return transport.toString("UTF-8");
-        } catch (TException e) {
-            throw new RuntimeException("Failed to serialize BKDLConfig : ", e);
-        } catch (UnsupportedEncodingException e) {
-            throw new RuntimeException("Failed to serialize BKDLConfig : ", e);
-        }
-    }
-
-    @Override
-    public void deserialize(byte[] data) throws IOException {
-        BKDLConfigFormat configFormat = new BKDLConfigFormat();
-        TMemoryInputTransport transport = new TMemoryInputTransport(data);
-        TJSONProtocol protocol = new TJSONProtocol(transport);
-        try {
-            configFormat.read(protocol);
-        } catch (TException e) {
-            throw new IOException("Failed to deserialize data '" +
-                    new String(data, UTF_8) + "' : ", e);
-        }
-        // bookkeeper cluster settings
-        if (configFormat.isSetBkZkServers()) {
-            bkZkServersForWriter = configFormat.getBkZkServers();
-        }
-        if (configFormat.isSetBkZkServersForReader()) {
-            bkZkServersForReader = configFormat.getBkZkServersForReader();
-        } else {
-            bkZkServersForReader = bkZkServersForWriter;
-        }
-        if (configFormat.isSetBkLedgersPath()) {
-            bkLedgersPath = configFormat.getBkLedgersPath();
-        }
-        // dl zookeeper cluster settings
-        if (configFormat.isSetDlZkServersForWriter()) {
-            dlZkServersForWriter = configFormat.getDlZkServersForWriter();
-        }
-        if (configFormat.isSetDlZkServersForReader()) {
-            dlZkServersForReader = configFormat.getDlZkServersForReader();
-        } else {
-            dlZkServersForReader = dlZkServersForWriter;
-        }
-        // dl settings
-        sanityCheckTxnID = !configFormat.isSetSanityCheckTxnID() || configFormat.isSanityCheckTxnID();
-        encodeRegionID = configFormat.isSetEncodeRegionID() && configFormat.isEncodeRegionID();
-        if (configFormat.isSetAclRootPath()) {
-            aclRootPath = configFormat.getAclRootPath();
-        }
-
-        if (configFormat.isSetFirstLogSegmentSeqNo()) {
-            firstLogSegmentSeqNo = configFormat.getFirstLogSegmentSeqNo();
-        }
-        isFederatedNamespace = configFormat.isSetFederatedNamespace() && configFormat.isFederatedNamespace();
-
-        // Validate the settings
-        if (null == bkZkServersForWriter || null == bkZkServersForReader || null == bkLedgersPath ||
-                null == dlZkServersForWriter || null == dlZkServersForReader) {
-            throw new IOException("Missing zk/bk settings in BKDL Config : " + new String(data, UTF_8));
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
deleted file mode 100644
index c76a5a5..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
+++ /dev/null
@@ -1,633 +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 com.twitter.distributedlog.impl.metadata;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Preconditions;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-import com.twitter.distributedlog.exceptions.LockCancelledException;
-import com.twitter.distributedlog.exceptions.LogExistsException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.impl.ZKLogSegmentMetadataStore;
-import com.twitter.distributedlog.lock.DistributedLock;
-import com.twitter.distributedlog.lock.SessionLockFactory;
-import com.twitter.distributedlog.lock.ZKDistributedLock;
-import com.twitter.distributedlog.lock.ZKSessionLockFactory;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
-import com.twitter.distributedlog.metadata.LogMetadata;
-import com.twitter.distributedlog.metadata.LogMetadataForReader;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.distributedlog.zk.LimitedPermitManager;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.PermitManager;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.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;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
-import org.apache.zookeeper.ZKUtil;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.common.PathUtils;
-import org.apache.zookeeper.data.ACL;
-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;
-import java.util.concurrent.TimeUnit;
-
-import static com.twitter.distributedlog.metadata.LogMetadata.*;
-
-/**
- * zookeeper based {@link LogStreamMetadataStore}
- */
-public class ZKLogStreamMetadataStore implements LogStreamMetadataStore {
-
-    private final static Logger LOG = LoggerFactory.getLogger(ZKLogStreamMetadataStore.class);
-
-    private final String clientId;
-    private final DistributedLogConfiguration conf;
-    private final ZooKeeperClient zooKeeperClient;
-    private final OrderedScheduler scheduler;
-    private final StatsLogger statsLogger;
-    private final LogSegmentMetadataStore logSegmentStore;
-    private final LimitedPermitManager permitManager;
-    // lock
-    private SessionLockFactory lockFactory;
-    private OrderedScheduler lockStateExecutor;
-
-    public ZKLogStreamMetadataStore(String clientId,
-                                    DistributedLogConfiguration conf,
-                                    ZooKeeperClient zkc,
-                                    OrderedScheduler scheduler,
-                                    StatsLogger statsLogger) {
-        this.clientId = clientId;
-        this.conf = conf;
-        this.zooKeeperClient = zkc;
-        this.scheduler = scheduler;
-        this.statsLogger = statsLogger;
-        // create the log segment metadata store and the permit manager (used for log segment rolling)
-        this.logSegmentStore = new ZKLogSegmentMetadataStore(conf, zooKeeperClient, scheduler);
-        this.permitManager = new LimitedPermitManager(
-                conf.getLogSegmentRollingConcurrency(),
-                1,
-                TimeUnit.MINUTES,
-                scheduler);
-        this.zooKeeperClient.register(permitManager);
-    }
-
-    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;
-    }
-
-    private synchronized SessionLockFactory getLockFactory(boolean createIfNull) {
-        if (createIfNull && null == lockFactory) {
-            lockFactory = new ZKSessionLockFactory(
-                    zooKeeperClient,
-                    clientId,
-                    getLockStateExecutor(createIfNull),
-                    conf.getZKNumRetries(),
-                    conf.getLockTimeoutMilliSeconds(),
-                    conf.getZKRetryBackoffStartMillis(),
-                    statsLogger);
-        }
-        return lockFactory;
-    }
-
-    @Override
-    public void close() throws IOException {
-        this.zooKeeperClient.unregister(permitManager);
-        this.permitManager.close();
-        this.logSegmentStore.close();
-        SchedulerUtils.shutdownScheduler(
-                getLockStateExecutor(false),
-                conf.getSchedulerShutdownTimeoutMs(),
-                TimeUnit.MILLISECONDS);
-    }
-
-    @Override
-    public LogSegmentMetadataStore getLogSegmentMetadataStore() {
-        return logSegmentStore;
-    }
-
-    @Override
-    public PermitManager getPermitManager() {
-        return this.permitManager;
-    }
-
-    @Override
-    public Transaction<Object> newTransaction() {
-        return new ZKTransaction(zooKeeperClient);
-    }
-
-    @Override
-    public Future<Void> logExists(URI uri, final String logName) {
-        final String logSegmentsPath = LogMetadata.getLogSegmentsPath(
-                uri, logName, conf.getUnpartitionedStreamName());
-        final Promise<Void> promise = new Promise<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(
-                                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,
-                                KeeperException.create(KeeperException.Code.get(syncRc))));
-                        return;
-                    }
-                    zk.exists(logSegmentsPath, false, new AsyncCallback.StatCallback() {
-                        @Override
-                        public void processResult(int rc, String path, Object ctx, Stat stat) {
-                            if (KeeperException.Code.OK.intValue() == rc) {
-                                promise.setValue(null);
-                            } else if (KeeperException.Code.NONODE.intValue() == rc) {
-                                promise.setException(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,
-                                        KeeperException.create(KeeperException.Code.get(rc))));
-                            }
-                        }
-                    }, null);
-                }
-            }, null);
-
-        } catch (InterruptedException ie) {
-            LOG.error("Interrupted while reading {}", logSegmentsPath, ie);
-            promise.setException(new DLInterruptedException("Interrupted while checking "
-                    + logSegmentsPath, ie));
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            promise.setException(e);
-        }
-        return promise;
-    }
-
-    //
-    // Create Write Lock
-    //
-
-    @Override
-    public DistributedLock createWriteLock(LogMetadataForWriter metadata) {
-        return new ZKDistributedLock(
-                getLockStateExecutor(true),
-                getLockFactory(true),
-                metadata.getLockPath(),
-                conf.getLockTimeoutMilliSeconds(),
-                statsLogger);
-    }
-
-    //
-    // Create Read Lock
-    //
-
-    private Future<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;
-            }
-        });
-        Optional<String> parentPathShouldNotCreate = Optional.of(logMetadata.getLogRootPath());
-        Utils.zkAsyncCreateFullPathOptimisticRecursive(zooKeeperClient, readLockPath, parentPathShouldNotCreate,
-                new byte[0], zooKeeperClient.getDefaultACL(), CreateMode.PERSISTENT,
-                new org.apache.zookeeper.AsyncCallback.StringCallback() {
-                    @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(
-                                    String.format("Log %s does not exist or has been deleted",
-                                            logMetadata.getFullyQualifiedName())));
-                        } else if (KeeperException.Code.OK.intValue() == rc) {
-                            FutureUtils.setValue(promise, null);
-                            LOG.trace("Created path {}.", path);
-                        } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
-                            FutureUtils.setValue(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));
-                        } else if (DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE == rc) {
-                            FutureUtils.setException(promise, new DLInterruptedException(path));
-                        } else {
-                            FutureUtils.setException(promise, KeeperException.create(KeeperException.Code.get(rc)));
-                        }
-                    }
-                }, null);
-        return promise;
-    }
-
-    @Override
-    public Future<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"));
-                    }
-                });
-            }
-        });
-    }
-
-    //
-    // Create Log
-    //
-
-    static class MetadataIndex {
-        static final int LOG_ROOT_PARENT = 0;
-        static final int LOG_ROOT = 1;
-        static final int MAX_TXID = 2;
-        static final int VERSION = 3;
-        static final int LOCK = 4;
-        static final int READ_LOCK = 5;
-        static final int LOGSEGMENTS = 6;
-        static final int ALLOCATION = 7;
-    }
-
-    static int bytesToInt(byte[] b) {
-        assert b.length >= 4;
-        return b[0] << 24 | b[1] << 16 | b[2] << 8 | b[3];
-    }
-
-    static byte[] intToBytes(int i) {
-        return new byte[]{
-            (byte) (i >> 24),
-            (byte) (i >> 16),
-            (byte) (i >> 8),
-            (byte) (i)};
-    }
-
-    static Future<List<Versioned<byte[]>>> checkLogMetadataPaths(ZooKeeper zk,
-                                                                 String logRootPath,
-                                                                 boolean ownAllocator) {
-        // Note re. persistent lock state initialization: the read lock persistent state (path) is
-        // initialized here but only used in the read handler. The reason is its more convenient and
-        // less error prone to manage all stream structure in one place.
-        final String logRootParentPath = new File(logRootPath).getParent();
-        final String logSegmentsPath = logRootPath + LOGSEGMENTS_PATH;
-        final String maxTxIdPath = logRootPath + MAX_TXID_PATH;
-        final String lockPath = logRootPath + LOCK_PATH;
-        final String readLockPath = logRootPath + READ_LOCK_PATH;
-        final String versionPath = logRootPath + VERSION_PATH;
-        final String allocationPath = logRootPath + ALLOCATION_PATH;
-
-        int numPaths = ownAllocator ? MetadataIndex.ALLOCATION + 1 : MetadataIndex.LOGSEGMENTS + 1;
-        List<Future<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));
-        checkFutures.add(Utils.zkGetData(zk, versionPath, false));
-        checkFutures.add(Utils.zkGetData(zk, lockPath, false));
-        checkFutures.add(Utils.zkGetData(zk, readLockPath, false));
-        checkFutures.add(Utils.zkGetData(zk, logSegmentsPath, false));
-        if (ownAllocator) {
-            checkFutures.add(Utils.zkGetData(zk, allocationPath, false));
-        }
-
-        return Future.collect(checkFutures);
-    }
-
-    static boolean pathExists(Versioned<byte[]> metadata) {
-        return null != metadata.getValue() && null != metadata.getVersion();
-    }
-
-    static void ensureMetadataExist(Versioned<byte[]> metadata) {
-        Preconditions.checkNotNull(metadata.getValue());
-        Preconditions.checkNotNull(metadata.getVersion());
-    }
-
-    static void createMissingMetadata(final ZooKeeper zk,
-                                      final String logRootPath,
-                                      final List<Versioned<byte[]>> metadatas,
-                                      final List<ACL> acl,
-                                      final boolean ownAllocator,
-                                      final boolean createIfNotExists,
-                                      final Promise<List<Versioned<byte[]>>> promise) {
-        final List<byte[]> pathsToCreate = Lists.newArrayListWithExpectedSize(metadatas.size());
-        final List<Op> zkOps = Lists.newArrayListWithExpectedSize(metadatas.size());
-        CreateMode createMode = CreateMode.PERSISTENT;
-
-        // log root parent path
-        if (pathExists(metadatas.get(MetadataIndex.LOG_ROOT_PARENT))) {
-            pathsToCreate.add(null);
-        } else {
-            String logRootParentPath = new File(logRootPath).getParent();
-            pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
-            zkOps.add(Op.create(logRootParentPath, DistributedLogConstants.EMPTY_BYTES, acl, createMode));
-        }
-
-        // log root path
-        if (pathExists(metadatas.get(MetadataIndex.LOG_ROOT))) {
-            pathsToCreate.add(null);
-        } else {
-            pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
-            zkOps.add(Op.create(logRootPath, DistributedLogConstants.EMPTY_BYTES, acl, createMode));
-        }
-
-        // max id
-        if (pathExists(metadatas.get(MetadataIndex.MAX_TXID))) {
-            pathsToCreate.add(null);
-        } else {
-            byte[] zeroTxnIdData = DLUtils.serializeTransactionId(0L);
-            pathsToCreate.add(zeroTxnIdData);
-            zkOps.add(Op.create(logRootPath + MAX_TXID_PATH, zeroTxnIdData, acl, createMode));
-        }
-        // version
-        if (pathExists(metadatas.get(MetadataIndex.VERSION))) {
-            pathsToCreate.add(null);
-        } else {
-            byte[] versionData = intToBytes(LAYOUT_VERSION);
-            pathsToCreate.add(versionData);
-            zkOps.add(Op.create(logRootPath + VERSION_PATH, versionData, acl, createMode));
-        }
-        // lock path
-        if (pathExists(metadatas.get(MetadataIndex.LOCK))) {
-            pathsToCreate.add(null);
-        } else {
-            pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
-            zkOps.add(Op.create(logRootPath + LOCK_PATH, DistributedLogConstants.EMPTY_BYTES, acl, createMode));
-        }
-        // read lock path
-        if (pathExists(metadatas.get(MetadataIndex.READ_LOCK))) {
-            pathsToCreate.add(null);
-        } else {
-            pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
-            zkOps.add(Op.create(logRootPath + READ_LOCK_PATH, DistributedLogConstants.EMPTY_BYTES, acl, createMode));
-        }
-        // log segments path
-        if (pathExists(metadatas.get(MetadataIndex.LOGSEGMENTS))) {
-            pathsToCreate.add(null);
-        } else {
-            byte[] logSegmentsData = DLUtils.serializeLogSegmentSequenceNumber(
-                    DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO);
-            pathsToCreate.add(logSegmentsData);
-            zkOps.add(Op.create(logRootPath + LOGSEGMENTS_PATH, logSegmentsData, acl, createMode));
-        }
-        // allocation path
-        if (ownAllocator) {
-            if (pathExists(metadatas.get(MetadataIndex.ALLOCATION))) {
-                pathsToCreate.add(null);
-            } else {
-                pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
-                zkOps.add(Op.create(logRootPath + ALLOCATION_PATH,
-                        DistributedLogConstants.EMPTY_BYTES, acl, createMode));
-            }
-        }
-        if (zkOps.isEmpty()) {
-            // nothing missed
-            promise.setValue(metadatas);
-            return;
-        }
-        if (!createIfNotExists) {
-            promise.setException(new LogNotFoundException("Log " + logRootPath + " not found"));
-            return;
-        }
-
-        zk.multi(zkOps, new AsyncCallback.MultiCallback() {
-            @Override
-            public void processResult(int rc, String path, Object ctx, List<OpResult> resultList) {
-                if (KeeperException.Code.OK.intValue() == rc) {
-                    List<Versioned<byte[]>> finalMetadatas =
-                            Lists.newArrayListWithExpectedSize(metadatas.size());
-                    for (int i = 0; i < pathsToCreate.size(); i++) {
-                        byte[] dataCreated = pathsToCreate.get(i);
-                        if (null == dataCreated) {
-                            finalMetadatas.add(metadatas.get(i));
-                        } else {
-                            finalMetadatas.add(new Versioned<byte[]>(dataCreated, new ZkVersion(0)));
-                        }
-                    }
-                    promise.setValue(finalMetadatas);
-                } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
-                    promise.setException(new LogExistsException("Someone just created log "
-                            + logRootPath));
-                } else {
-                    if (LOG.isDebugEnabled()) {
-                        StringBuilder builder = new StringBuilder();
-                        for (OpResult result : resultList) {
-                            if (result instanceof OpResult.ErrorResult) {
-                                OpResult.ErrorResult errorResult = (OpResult.ErrorResult) result;
-                                builder.append(errorResult.getErr()).append(",");
-                            } else {
-                                builder.append(0).append(",");
-                            }
-                        }
-                        String resultCodeList = builder.substring(0, builder.length() - 1);
-                        LOG.debug("Failed to create log, full rc list = {}", resultCodeList);
-                    }
-
-                    promise.setException(new ZKException("Failed to create log " + logRootPath,
-                            KeeperException.Code.get(rc)));
-                }
-            }
-        }, null);
-    }
-
-    static LogMetadataForWriter processLogMetadatas(URI uri,
-                                                    String logName,
-                                                    String logIdentifier,
-                                                    List<Versioned<byte[]>> metadatas,
-                                                    boolean ownAllocator)
-            throws UnexpectedException {
-        try {
-            // max id
-            Versioned<byte[]> maxTxnIdData = metadatas.get(MetadataIndex.MAX_TXID);
-            ensureMetadataExist(maxTxnIdData);
-            // version
-            Versioned<byte[]> versionData = metadatas.get(MetadataIndex.VERSION);
-            ensureMetadataExist(maxTxnIdData);
-            Preconditions.checkArgument(LAYOUT_VERSION == bytesToInt(versionData.getValue()));
-            // lock path
-            ensureMetadataExist(metadatas.get(MetadataIndex.LOCK));
-            // read lock path
-            ensureMetadataExist(metadatas.get(MetadataIndex.READ_LOCK));
-            // max lssn
-            Versioned<byte[]> maxLSSNData = metadatas.get(MetadataIndex.LOGSEGMENTS);
-            ensureMetadataExist(maxLSSNData);
-            try {
-                DLUtils.deserializeLogSegmentSequenceNumber(maxLSSNData.getValue());
-            } catch (NumberFormatException nfe) {
-                throw new UnexpectedException("Invalid max sequence number found in log " + logName, nfe);
-            }
-            // allocation path
-            Versioned<byte[]>  allocationData;
-            if (ownAllocator) {
-                allocationData = metadatas.get(MetadataIndex.ALLOCATION);
-                ensureMetadataExist(allocationData);
-            } else {
-                allocationData = new Versioned<byte[]>(null, null);
-            }
-            return new LogMetadataForWriter(uri, logName, logIdentifier,
-                    maxLSSNData, maxTxnIdData, allocationData);
-        } catch (IllegalArgumentException iae) {
-            throw new UnexpectedException("Invalid log " + logName, iae);
-        } catch (NullPointerException npe) {
-            throw new UnexpectedException("Invalid log " + logName, npe);
-        }
-    }
-
-    static Future<LogMetadataForWriter> getLog(final URI uri,
-                                               final String logName,
-                                               final String logIdentifier,
-                                               final ZooKeeperClient zooKeeperClient,
-                                               final boolean ownAllocator,
-                                               final boolean createIfNotExists) {
-        final String logRootPath = LogMetadata.getLogRootPath(uri, logName, logIdentifier);
-        try {
-            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"));
-        }
-
-        try {
-            final ZooKeeper zk = zooKeeperClient.get();
-            return checkLogMetadataPaths(zk, logRootPath, ownAllocator)
-                    .flatMap(new AbstractFunction1<List<Versioned<byte[]>>, Future<List<Versioned<byte[]>>>>() {
-                        @Override
-                        public Future<List<Versioned<byte[]>>> apply(List<Versioned<byte[]>> metadatas) {
-                            Promise<List<Versioned<byte[]>>> promise =
-                                    new Promise<List<Versioned<byte[]>>>();
-                            createMissingMetadata(zk, logRootPath, metadatas, zooKeeperClient.getDefaultACL(),
-                                    ownAllocator, createIfNotExists, promise);
-                            return promise;
-                        }
-                    }).map(new ExceptionalFunction<List<Versioned<byte[]>>, LogMetadataForWriter>() {
-                        @Override
-                        public LogMetadataForWriter applyE(List<Versioned<byte[]>> metadatas) throws DLException {
-                            return processLogMetadatas(
-                                    uri,
-                                    logName,
-                                    logIdentifier,
-                                    metadatas,
-                                    ownAllocator);
-                        }
-                    });
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            return Future.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));
-        }
-    }
-
-    @Override
-    public Future<LogMetadataForWriter> getLog(final URI uri,
-                                               final String logName,
-                                               final boolean ownAllocator,
-                                               final boolean createIfNotExists) {
-        return getLog(
-                uri,
-                logName,
-                conf.getUnpartitionedStreamName(),
-                zooKeeperClient,
-                ownAllocator,
-                createIfNotExists);
-    }
-
-    //
-    // Delete Log
-    //
-
-    @Override
-    public Future<Void> deleteLog(URI uri, final String logName) {
-        final Promise<Void> promise = new Promise<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,
-                                new ZKException("Encountered zookeeper issue on deleting log stream "
-                                        + logName, KeeperException.Code.get(rc)));
-                        return;
-                    }
-                    FutureUtils.setValue(promise, null);
-                }
-            }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-            FutureUtils.setException(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 "
-                    + logName));
-        } catch (KeeperException e) {
-            FutureUtils.setException(promise, new ZKException("Encountered zookeeper issue on deleting log stream "
-                    + logName, e));
-        }
-        return promise;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java
deleted file mode 100644
index 6b7a231..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java
+++ /dev/null
@@ -1,72 +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 com.twitter.distributedlog.impl.metadata;
-
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.metadata.DLMetadata;
-import com.twitter.distributedlog.metadata.MetadataResolver;
-import org.apache.commons.lang.StringUtils;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.common.PathUtils;
-import org.apache.zookeeper.data.Stat;
-
-import java.io.IOException;
-import java.net.URI;
-
-public class ZkMetadataResolver implements MetadataResolver {
-
-    private final ZooKeeperClient zkc;
-
-    public ZkMetadataResolver(ZooKeeperClient zkc) {
-        this.zkc = zkc;
-    }
-
-    @Override
-    public DLMetadata resolve(URI uri) throws IOException {
-        String dlPath = uri.getPath();
-        PathUtils.validatePath(dlPath);
-        // Normal case the dl metadata is stored in the last segment
-        // so lookup last segment first.
-        String[] parts = StringUtils.split(dlPath, '/');
-        if (null == parts || 0 == parts.length) {
-            throw new IOException("Invalid dlPath to resolve dl metadata : " + dlPath);
-        }
-        for (int i = parts.length; i >= 0; i--) {
-            String pathToResolve = String.format("/%s", StringUtils.join(parts, '/', 0, i));
-            byte[] data;
-            try {
-                data = zkc.get().getData(pathToResolve, false, new Stat());
-            } catch (KeeperException.NoNodeException nne) {
-                continue;
-            } catch (KeeperException ke) {
-                throw new IOException("Fail to resolve dl path : " + pathToResolve);
-            } catch (InterruptedException ie) {
-                throw new IOException("Interrupted when resolving dl path : " + pathToResolve);
-            }
-            if (null == data || data.length == 0) {
-                continue;
-            }
-            try {
-                return DLMetadata.deserialize(uri, data);
-            } catch (IOException ie) {
-                throw new IOException("Failed to deserialize uri : " + uri);
-            }
-        }
-        throw new IOException("No bkdl config bound under dl path : " + dlPath);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/package-info.java
deleted file mode 100644
index 7c5c2e4..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * The BookKeeper Based DistributedLog Implementation.
- */
-package com.twitter.distributedlog.impl;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
deleted file mode 100644
index b067ee9..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
+++ /dev/null
@@ -1,121 +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 com.twitter.distributedlog.impl.subscription;
-
-import java.io.IOException;
-import java.util.concurrent.atomic.AtomicReference;
-
-import com.twitter.distributedlog.subscription.SubscriptionStateStore;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.BoxedUnit;
-
-import com.google.common.base.Charsets;
-
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.data.Stat;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-
-public class ZKSubscriptionStateStore implements SubscriptionStateStore {
-
-    static final Logger logger = LoggerFactory.getLogger(ZKSubscriptionStateStore.class);
-
-    private final ZooKeeperClient zooKeeperClient;
-    private final String zkPath;
-    private AtomicReference<DLSN> lastCommittedPosition = new AtomicReference<DLSN>(null);
-
-    public ZKSubscriptionStateStore(ZooKeeperClient zooKeeperClient, String zkPath) {
-        this.zooKeeperClient = zooKeeperClient;
-        this.zkPath = zkPath;
-    }
-
-    @Override
-    public void close() throws IOException {
-    }
-
-    /**
-     * Get the last committed position stored for this subscription
-     */
-    @Override
-    public Future<DLSN> getLastCommitPosition() {
-        if (null != lastCommittedPosition.get()) {
-            return Future.value(lastCommittedPosition.get());
-        } else {
-            return getLastCommitPositionFromZK();
-        }
-    }
-
-    Future<DLSN> getLastCommitPositionFromZK() {
-        final Promise<DLSN> result = new Promise<DLSN>();
-        try {
-            logger.debug("Reading last commit position from path {}", zkPath);
-            zooKeeperClient.get().getData(zkPath, false, new AsyncCallback.DataCallback() {
-                @Override
-                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);
-                    } else if (KeeperException.Code.OK.intValue() != rc) {
-                        result.setException(KeeperException.create(KeeperException.Code.get(rc), path));
-                    } else {
-                        try {
-                            DLSN dlsn = DLSN.deserialize(new String(data, Charsets.UTF_8));
-                            result.setValue(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);
-                        }
-                    }
-                }
-            }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
-            result.setException(zkce);
-        } catch (InterruptedException ie) {
-            result.setException(new DLInterruptedException("getLastCommitPosition was interrupted", ie));
-        }
-        return result;
-    }
-
-    /**
-     * Advances the position associated with the subscriber
-     *
-     * @param newPosition - new commit position
-     */
-    @Override
-    public Future<BoxedUnit> advanceCommitPosition(DLSN newPosition) {
-        if (null == lastCommittedPosition.get() ||
-            (newPosition.compareTo(lastCommittedPosition.get()) > 0)) {
-            lastCommittedPosition.set(newPosition);
-            return Utils.zkAsyncCreateFullPathOptimisticAndSetData(zooKeeperClient,
-                zkPath, newPosition.serialize().getBytes(Charsets.UTF_8),
-                zooKeeperClient.getDefaultACL(),
-                CreateMode.PERSISTENT);
-        } else {
-            return Future.Done();
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java
deleted file mode 100644
index 17ba943..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java
+++ /dev/null
@@ -1,164 +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 com.twitter.distributedlog.impl.subscription;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.subscription.SubscriptionStateStore;
-import com.twitter.distributedlog.subscription.SubscriptionsStore;
-import com.twitter.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.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * ZooKeeper Based Subscriptions Store.
- */
-public class ZKSubscriptionsStore implements SubscriptionsStore {
-
-    private final ZooKeeperClient zkc;
-    private final String zkPath;
-    private final ConcurrentMap<String, ZKSubscriptionStateStore> subscribers =
-            new ConcurrentHashMap<String, ZKSubscriptionStateStore>();
-
-    public ZKSubscriptionsStore(ZooKeeperClient zkc, String zkPath) {
-        this.zkc = zkc;
-        this.zkPath = zkPath;
-    }
-
-    private ZKSubscriptionStateStore getSubscriber(String subscriberId) {
-        ZKSubscriptionStateStore ss = subscribers.get(subscriberId);
-        if (ss == null) {
-            ZKSubscriptionStateStore newSS = new ZKSubscriptionStateStore(zkc,
-                getSubscriberZKPath(subscriberId));
-            ZKSubscriptionStateStore oldSS = subscribers.putIfAbsent(subscriberId, newSS);
-            if (oldSS == null) {
-                ss = newSS;
-            } else {
-                try {
-                    newSS.close();
-                } catch (IOException e) {
-                    // ignore the exception
-                }
-                ss = oldSS;
-            }
-        }
-        return ss;
-    }
-
-    private String getSubscriberZKPath(String subscriberId) {
-        return String.format("%s/%s", zkPath, subscriberId);
-    }
-
-    @Override
-    public Future<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>>();
-        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>());
-                    } else if (KeeperException.Code.OK.intValue() != rc) {
-                        result.setException(KeeperException.create(KeeperException.Code.get(rc), path));
-                    } else {
-                        getLastCommitPositions(result, children);
-                    }
-                }
-            }, null);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
-            result.setException(zkce);
-        } catch (InterruptedException ie) {
-            result.setException(new DLInterruptedException("getLastCommitPositions was interrupted", ie));
-        }
-        return result;
-    }
-
-    private void getLastCommitPositions(final Promise<Map<String, DLSN>> result,
-                                        List<String> subscribers) {
-        List<Future<Pair<String, DLSN>>> futures =
-                new ArrayList<Future<Pair<String, DLSN>>>(subscribers.size());
-        for (String s : subscribers) {
-            final String subscriber = s;
-            Future<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);
-                            }
-                        });
-            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;
-                }
-            });
-    }
-
-    @Override
-    public Future<BoxedUnit> advanceCommitPosition(String subscriberId, DLSN newPosition) {
-        return getSubscriber(subscriberId).advanceCommitPosition(newPosition);
-    }
-
-    @Override
-    public Future<Boolean> deleteSubscriber(String subscriberId) {
-        subscribers.remove(subscriberId);
-        String path = getSubscriberZKPath(subscriberId);
-        return Utils.zkDeleteIfNotExist(zkc, path, new ZkVersion(-1));
-    }
-
-    @Override
-    public void close() throws IOException {
-        // no-op
-        for (SubscriptionStateStore store : subscribers.values()) {
-            store.close();
-        }
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncFailureInjector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncFailureInjector.java
deleted file mode 100644
index 4145040..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncFailureInjector.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 com.twitter.distributedlog.injector;
-
-/**
- * Failure Injector that works in asynchronous way
- */
-public interface AsyncFailureInjector {
-
-    AsyncFailureInjector NULL = new AsyncFailureInjector() {
-        @Override
-        public void injectErrors(boolean enabled) {
-            // no-op
-        }
-
-        @Override
-        public boolean shouldInjectErrors() {
-            return false;
-        }
-
-        @Override
-        public void injectDelays(boolean enabled) {
-            // no-op
-        }
-
-        @Override
-        public boolean shouldInjectDelays() {
-            return false;
-        }
-
-        @Override
-        public int getInjectedDelayMs() {
-            return 0;
-        }
-
-        @Override
-        public void injectStops(boolean enabled) {
-            // no-op
-        }
-
-        @Override
-        public boolean shouldInjectStops() {
-            return false;
-        }
-
-        @Override
-        public boolean shouldInjectCorruption(long startEntryId, long endEntryId) {
-            return false;
-        }
-
-        @Override
-        public String toString() {
-            return "NULL";
-        }
-    };
-
-    /**
-     * Enable or disable error injection.
-     *
-     * @param enabled
-     *          flag to enable or disable error injection.
-     */
-    void injectErrors(boolean enabled);
-
-    /**
-     * Return the flag indicating if should inject errors.
-     *
-     * @return true to inject errors otherwise false.
-     */
-    boolean shouldInjectErrors();
-
-    /**
-     * Enable or disable delay injection.
-     *
-     * @param enabled
-     *          flag to enable or disable delay injection.
-     */
-    void injectDelays(boolean enabled);
-
-    /**
-     * Return the flag indicating if should inject delays.
-     *
-     * @return true to inject delays otherwise false.
-     */
-    boolean shouldInjectDelays();
-
-    /**
-     * Return the injected delay in milliseconds.
-     *
-     * @return the injected delay in milliseconds.
-     */
-    int getInjectedDelayMs();
-
-    /**
-     * Enable or disable injecting stops. This could be used
-     * for simulating stopping an action.
-     */
-    void injectStops(boolean enabled);
-
-    /**
-     * Return the flag indicating if should inject stops.
-     *
-     * @return true to inject stops otherwise false.
-     */
-    boolean shouldInjectStops();
-
-    /**
-     * Return the flag indicating if should inject corruption.
-     *
-     * @param startEntryId the start entry id
-     * @param endEntryId the end entry id
-     * @return true to inject corruption otherwise false.
-     */
-    boolean shouldInjectCorruption(long startEntryId, long endEntryId);
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncRandomFailureInjector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncRandomFailureInjector.java
deleted file mode 100644
index f3bfea9..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncRandomFailureInjector.java
+++ /dev/null
@@ -1,186 +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 com.twitter.distributedlog.injector;
-
-import com.twitter.distributedlog.util.Utils;
-
-import java.util.Random;
-
-/**
- * Failure injector based on {@link java.util.Random}
- */
-public class AsyncRandomFailureInjector implements AsyncFailureInjector {
-
-    private static final Random random = new Random(System.currentTimeMillis());
-
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    public static class Builder {
-
-        private boolean _simulateDelays = false;
-        private boolean _simulateErrors = false;
-        private boolean _simulateStops = false;
-        private boolean _simulateCorruption = false;
-        private int _injectedDelayPercent = 0;
-        private int _injectedErrorPercent = 0;
-        private int _injectedStopPercent = 0;
-        private int _maxInjectedDelayMs = Integer.MAX_VALUE;
-
-        private Builder() {}
-
-        public Builder injectDelays(boolean simulateDelays,
-                                    int injectedDelayPercent,
-                                    int maxInjectedDelayMs) {
-            this._simulateDelays = simulateDelays;
-            this._injectedDelayPercent = injectedDelayPercent;
-            this._maxInjectedDelayMs = maxInjectedDelayMs;
-            return this;
-        }
-
-        public Builder injectErrors(boolean simulateErrors,
-                                    int injectedErrorPercent) {
-            this._simulateErrors = simulateErrors;
-            this._injectedErrorPercent = injectedErrorPercent;
-            return this;
-        }
-
-        public Builder injectCorruption(boolean simulateCorruption) {
-            this._simulateCorruption = simulateCorruption;
-            return this;
-        }
-
-        public Builder injectStops(boolean simulateStops,
-                                   int injectedStopPercent) {
-            this._simulateStops = simulateStops;
-            this._injectedStopPercent = injectedStopPercent;
-            return this;
-        }
-
-        public AsyncFailureInjector build() {
-            return new AsyncRandomFailureInjector(
-                    _simulateDelays,
-                    _injectedDelayPercent,
-                    _maxInjectedDelayMs,
-                    _simulateErrors,
-                    _injectedErrorPercent,
-                    _simulateStops,
-                    _injectedStopPercent,
-                    _simulateCorruption);
-        }
-
-    }
-
-    private boolean simulateDelays;
-    private boolean simulateErrors;
-    private boolean simulateStops;
-    private boolean simulateCorruption;
-    private final int injectedDelayPercent;
-    private final int injectedErrorPercent;
-    private final int injectedStopPercent;
-    private final int maxInjectedDelayMs;
-
-    private AsyncRandomFailureInjector(boolean simulateDelays,
-                                       int injectedDelayPercent,
-                                       int maxInjectedDelayMs,
-                                       boolean simulateErrors,
-                                       int injectedErrorPercent,
-                                       boolean simulateStops,
-                                       int injectedStopPercent,
-                                       boolean simulateCorruption) {
-        this.simulateDelays = simulateDelays;
-        this.injectedDelayPercent = injectedDelayPercent;
-        this.maxInjectedDelayMs = maxInjectedDelayMs;
-        this.simulateErrors = simulateErrors;
-        this.injectedErrorPercent = injectedErrorPercent;
-        this.simulateStops = simulateStops;
-        this.injectedStopPercent = injectedStopPercent;
-        this.simulateCorruption = simulateCorruption;
-    }
-
-    @Override
-    public void injectErrors(boolean enabled) {
-        this.simulateErrors = enabled;
-    }
-
-    @Override
-    public boolean shouldInjectErrors() {
-        return simulateErrors && Utils.randomPercent(injectedErrorPercent);
-    }
-
-    @Override
-    public void injectDelays(boolean enabled) {
-        this.simulateDelays = enabled;
-    }
-
-    @Override
-    public boolean shouldInjectDelays() {
-        return simulateDelays && Utils.randomPercent(injectedDelayPercent);
-    }
-
-    @Override
-    public int getInjectedDelayMs() {
-        if (maxInjectedDelayMs > 0) {
-            return random.nextInt(maxInjectedDelayMs);
-        }
-        return 0;
-    }
-
-    @Override
-    public void injectStops(boolean enabled) {
-        this.simulateStops = enabled;
-    }
-
-    @Override
-    public boolean shouldInjectStops() {
-        return simulateStops && Utils.randomPercent(injectedStopPercent);
-    }
-
-    @Override
-    public boolean shouldInjectCorruption(long startEntryId, long endEntryId) {
-        if (!simulateCorruption) {
-            return false;
-        }
-        if (startEntryId == endEntryId) {
-            return startEntryId % 10 == 0;
-        }
-        for (long i = startEntryId; i <= endEntryId; i++) {
-            if (i % 10 == 0) {
-                return true;
-            }
-        }
-        return false;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("FailureInjector[");
-        sb.append("errors=(").append(simulateErrors).append(", pct=")
-                .append(injectedErrorPercent).append("), ");
-        sb.append("delays=(").append(simulateDelays).append(", pct=")
-                .append(injectedDelayPercent).append(", max=")
-                .append(maxInjectedDelayMs).append("), ");
-        sb.append("stops=(").append(simulateStops).append(", pct=")
-                .append(injectedStopPercent).append(")");
-        sb.append("corruption=(").append(simulateCorruption).append(")");
-        sb.append("]");
-        return sb.toString();
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/FailureInjector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/FailureInjector.java
deleted file mode 100644
index 16c8e4e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/FailureInjector.java
+++ /dev/null
@@ -1,37 +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 com.twitter.distributedlog.injector;
-
-/**
- * Failure injector.
- */
-public interface FailureInjector {
-
-    /**
-     * No-op failure injector, which does nothing.
-     */
-    public static FailureInjector NULL = new FailureInjector() {
-        @Override
-        public void inject() {
-            // no-op;
-        }
-    };
-
-    // inject failures
-    void inject();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/RandomDelayFailureInjector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/RandomDelayFailureInjector.java
deleted file mode 100644
index 73aad5b..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/RandomDelayFailureInjector.java
+++ /dev/null
@@ -1,60 +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 com.twitter.distributedlog.injector;
-
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.util.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Injector that injects random delays
- */
-public class RandomDelayFailureInjector implements FailureInjector {
-
-    private static final Logger LOG = LoggerFactory.getLogger(RandomDelayFailureInjector.class);
-
-    private final DynamicDistributedLogConfiguration dynConf;
-
-    public RandomDelayFailureInjector(DynamicDistributedLogConfiguration dynConf) {
-        this.dynConf = dynConf;
-    }
-
-    private int delayMs() {
-        return dynConf.getEIInjectedWriteDelayMs();
-    }
-
-    private double delayPct() {
-        return dynConf.getEIInjectedWriteDelayPercent();
-    }
-
-    private boolean enabled() {
-        return delayMs() > 0 && delayPct() > 0;
-    }
-
-    @Override
-    public void inject() {
-        try {
-            if (enabled() && Utils.randomPercent(delayPct())) {
-                Thread.sleep(delayMs());
-            }
-        } catch (InterruptedException ex) {
-            LOG.warn("delay was interrupted ", ex);
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/package-info.java
deleted file mode 100644
index ffee340..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Failure Injection
- */
-package com.twitter.distributedlog.injector;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortable.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortable.java
deleted file mode 100644
index 0d0b389..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortable.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 com.twitter.distributedlog.io;
-
-import java.io.IOException;
-
-/**
- * An {@code Abortable} is a source or destination of data that can be aborted.
- * The abort method is invoked to release resources that the object is holding
- * (such as open files). The abort happens when the object is in an error state,
- * which it couldn't be closed gracefully.
- *
- * @see java.io.Closeable
- * @since 0.3.32
- */
-public interface Abortable {
-
-    /**
-     * Aborts the object and releases any resources associated with it.
-     * If the object is already aborted then invoking this method has no
-     * effect.
-     *
-     * @throws IOException if an I/O error occurs.
-     */
-    public void abort() throws IOException;
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortables.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortables.java
deleted file mode 100644
index 4599574..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortables.java
+++ /dev/null
@@ -1,183 +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 com.twitter.distributedlog.io;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.function.VoidFunctions;
-import com.twitter.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.ExecutorService;
-
-/**
- * Utility methods for working with {@link Abortable} objects.
- *
- * @since 0.3.32
- */
-public final class Abortables {
-
-    static final Logger logger = LoggerFactory.getLogger(Abortables.class);
-
-    private Abortables() {}
-
-    public static Future<Void> asyncAbort(@Nullable AsyncAbortable abortable,
-                                          boolean swallowIOException) {
-        if (null == abortable) {
-            return Future.Void();
-        } else if (swallowIOException) {
-            return FutureUtils.ignore(abortable.asyncAbort());
-        } else {
-            return abortable.asyncAbort();
-        }
-    }
-
-    /**
-     * Aborts a {@link Abortable}, with control over whether an {@link IOException} may be thrown.
-     * This is primarily useful in a finally block, where a thrown exception needs to be logged but
-     * not propagated (otherwise the original exception will be lost).
-     *
-     * <p>If {@code swallowIOException} is true then we never throw {@code IOException} but merely log it.
-     *
-     * <p>Example: <pre>   {@code
-     *
-     *   public void abortStreamNicely() throws IOException {
-     *      SomeStream stream = new SomeStream("foo");
-     *      try {
-     *          // ... code which does something with the stream ...
-     *      } catch (IOException ioe) {
-     *          // If an exception occurs, we might abort the stream.
-     *          Abortables.abort(stream, true);
-     *      }
-     *   }}</pre>
-     *
-     * @param abortable the {@code Abortable} 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 Abortable abortable,
-                             boolean swallowIOException)
-        throws IOException {
-        if (null == abortable) {
-            return;
-        }
-        try {
-            abortable.abort();
-        } catch (IOException ioe) {
-            if (swallowIOException) {
-                logger.warn("IOException thrown while aborting Abortable {} : ", abortable, ioe);
-            } else {
-                throw ioe;
-            }
-        }
-    }
-
-    /**
-     * 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}
-     * @see #abort(Abortable, boolean)
-     */
-    public static void abort(@Nullable AsyncAbortable abortable,
-                             boolean swallowIOException)
-            throws IOException {
-        if (null == abortable) {
-            return;
-        }
-        try {
-            FutureUtils.result(abortable.asyncAbort());
-        } catch (IOException ioe) {
-            if (swallowIOException) {
-                logger.warn("IOException thrown while aborting Abortable {} : ", abortable, ioe);
-            } else {
-                throw ioe;
-            }
-        }
-    }
-
-    /**
-     * 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
-     * I/O resource, it should generally be safe in the case of a resource that's being used only for
-     * reading.
-     *
-     * @param abortable the {@code Abortable} to be closed, or {@code null} in which case this method
-     *                  does nothing.
-     */
-    public static void abortQuietly(@Nullable Abortable abortable) {
-        try {
-            abort(abortable, true);
-        } catch (IOException e) {
-            logger.error("Unexpected IOException thrown while aborting Abortable {} quietly : ", abortable, e);
-        }
-    }
-
-    /**
-     * 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
-     * I/O resource, it should generally be safe in the case of a resource that's being used only for
-     * reading.
-     *
-     * @param abortable the {@code AsyncAbortable} to be closed, or {@code null} in which case this method
-     *                  does nothing.
-     */
-    public static void abortQuietly(@Nullable AsyncAbortable abortable) {
-        try {
-            abort(abortable, true);
-        } catch (IOException e) {
-            logger.error("Unexpected IOException thrown while aborting Abortable {} quietly : ", abortable, e);
-        }
-    }
-
-    /**
-     * Abort the abortables in sequence.
-     *
-     * @param executorService
-     *          executor service to execute
-     * @param abortables
-     *          abortables to abort
-     * @return future represents the abort future
-     */
-    public static Future<Void> abortSequence(ExecutorService executorService,
-                                             AsyncAbortable... abortables) {
-        List<AsyncAbortable> abortableList = Lists.newArrayListWithExpectedSize(abortables.length);
-        for (AsyncAbortable abortable : abortables) {
-            if (null == abortable) {
-                abortableList.add(AsyncAbortable.NULL);
-            } else {
-                abortableList.add(abortable);
-            }
-        }
-        return FutureUtils.processList(
-                abortableList,
-                AsyncAbortable.ABORT_FUNC,
-                executorService).map(VoidFunctions.LIST_TO_VOID_FUNC);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncAbortable.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncAbortable.java
deleted file mode 100644
index ed1062a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncAbortable.java
+++ /dev/null
@@ -1,57 +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 com.twitter.distributedlog.io;
-
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-
-/**
- * An {@code Abortable} is a source or destination of data that can be aborted.
- * The abort method is invoked to release resources that the object is holding
- * (such as open files). The abort happens when the object is in an error state,
- * which it couldn't be closed gracefully.
- *
- * @see AsyncCloseable
- * @see Abortable
- * @since 0.3.43
- */
-public interface AsyncAbortable {
-
-    Function<AsyncAbortable, Future<Void>> ABORT_FUNC = new Function<AsyncAbortable, Future<Void>>() {
-        @Override
-        public Future<Void> apply(AsyncAbortable abortable) {
-            return abortable.asyncAbort();
-        }
-    };
-
-    AsyncAbortable NULL = new AsyncAbortable() {
-        @Override
-        public Future<Void> asyncAbort() {
-            return Future.Void();
-        }
-    };
-
-    /**
-     * 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/src/main/java/com/twitter/distributedlog/io/AsyncCloseable.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncCloseable.java
deleted file mode 100644
index 817a8e2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncCloseable.java
+++ /dev/null
@@ -1,60 +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 com.twitter.distributedlog.io;
-
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-
-/**
- * A {@code AsyncCloseable} is a source or destination of data that can be closed asynchronously.
- * The close method is invoked to release resources that the object is
- * holding (such as open files).
- */
-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, Future<Void>> CLOSE_FUNC_IGNORE_ERRORS = new Function<AsyncCloseable, Future<Void>>() {
-        @Override
-        public Future<Void> apply(AsyncCloseable closeable) {
-            return FutureUtils.ignore(closeable.asyncClose());
-        }
-    };
-
-    AsyncCloseable NULL = new AsyncCloseable() {
-        @Override
-        public Future<Void> asyncClose() {
-            return Future.Void();
-        }
-    };
-
-    /**
-     * 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/src/main/java/com/twitter/distributedlog/io/AsyncDeleteable.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncDeleteable.java
deleted file mode 100644
index 203895e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncDeleteable.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 com.twitter.distributedlog.io;
-
-import com.twitter.util.Future;
-
-/**
- * A {@code AsyncDeleteable} is a source or destination of data that can be deleted asynchronously.
- * This delete method is invoked to delete the source.
- */
-public interface AsyncDeleteable {
-    /**
-     * Releases any system resources associated with this and delete the source. If the source is
-     * already deleted then invoking this method has no effect.
-     *
-     * @return future representing the deletion result.
-     */
-    Future<Void> delete();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/io/package-info.java
deleted file mode 100644
index df2e91f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/io/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * IO Utils for distributedlog
- */
-package com.twitter.distributedlog.io;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ChainedRequestLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ChainedRequestLimiter.java
deleted file mode 100644
index 60eacd5..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ChainedRequestLimiter.java
+++ /dev/null
@@ -1,76 +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 com.twitter.distributedlog.limiter;
-
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.ImmutableList;
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-
-import java.util.concurrent.TimeUnit;
-
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Chain request limiters for easier management of multi limiter policy.
- */
-public class ChainedRequestLimiter<Request> implements RequestLimiter<Request> {
-    private final ImmutableList<RequestLimiter<Request>> limiters;
-    private final OpStatsLogger applyTime;
-
-    public static class Builder<Request> {
-        private final ImmutableList.Builder<RequestLimiter<Request>> limitersBuilder;
-        private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
-
-        public Builder() {
-            this.limitersBuilder = new ImmutableList.Builder<RequestLimiter<Request>>();
-        }
-
-        public Builder<Request> addLimiter(RequestLimiter<Request> limiter) {
-            this.limitersBuilder.add(limiter);
-            return this;
-        }
-
-        public Builder<Request> statsLogger(StatsLogger statsLogger) {
-            this.statsLogger = statsLogger;
-            return this;
-        }
-
-        public ChainedRequestLimiter<Request> build() {
-            return new ChainedRequestLimiter<Request>(limitersBuilder.build(), statsLogger);
-        }
-    }
-
-    private ChainedRequestLimiter(ImmutableList<RequestLimiter<Request>> limiters,
-                                  StatsLogger statsLogger) {
-        this.limiters = limiters;
-        this.applyTime = statsLogger.getOpStatsLogger("apply");
-    }
-
-    public void apply(Request request) throws OverCapacityException {
-        Stopwatch stopwatch = Stopwatch.createStarted();
-        try {
-            for (RequestLimiter<Request> limiter : limiters) {
-                limiter.apply(request);
-            }
-        } finally {
-            applyTime.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ComposableRequestLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ComposableRequestLimiter.java
deleted file mode 100644
index 55e4c8b..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ComposableRequestLimiter.java
+++ /dev/null
@@ -1,73 +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 com.twitter.distributedlog.limiter;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Preconditions;
-
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-import com.twitter.distributedlog.limiter.GuavaRateLimiter;
-import com.twitter.distributedlog.limiter.RateLimiter;
-
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Collect rate limiter implementation, cost(Request), overlimit, etc. behavior.
- */
-public class ComposableRequestLimiter<Request> implements RequestLimiter<Request> {
-    protected static final Logger LOG = LoggerFactory.getLogger(ComposableRequestLimiter.class);
-
-    private final RateLimiter limiter;
-    private final OverlimitFunction<Request> overlimitFunction;
-    private final CostFunction<Request> costFunction;
-    private final Counter overlimitCounter;
-
-    static public interface OverlimitFunction<Request> {
-        void apply(Request request) throws OverCapacityException;
-    }
-    static public interface CostFunction<Request> {
-        int apply(Request request);
-    }
-
-    public ComposableRequestLimiter(
-            RateLimiter limiter,
-            OverlimitFunction<Request> overlimitFunction,
-            CostFunction<Request> costFunction,
-            StatsLogger statsLogger) {
-        Preconditions.checkNotNull(limiter);
-        Preconditions.checkNotNull(overlimitFunction);
-        Preconditions.checkNotNull(costFunction);
-        this.limiter = limiter;
-        this.overlimitFunction = overlimitFunction;
-        this.costFunction = costFunction;
-        this.overlimitCounter = statsLogger.getCounter("overlimit");
-    }
-
-    @Override
-    public void apply(Request request) throws OverCapacityException {
-        int permits = costFunction.apply(request);
-        if (!limiter.acquire(permits)) {
-            overlimitCounter.inc();
-            overlimitFunction.apply(request);
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/GuavaRateLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/GuavaRateLimiter.java
deleted file mode 100644
index 3f1909a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/GuavaRateLimiter.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 com.twitter.distributedlog.limiter;
-
-import com.google.common.base.Preconditions;
-
-/**
- * Wrap a guava limiter in a simple interface to make testing easier.
- * Notes:
- * 1. Negative limit translates into (virtually) unlimited.
- * 2. Calling acquire with permits == 0 translates into no acquire.
- */
-public class GuavaRateLimiter implements RateLimiter {
-    com.google.common.util.concurrent.RateLimiter limiter;
-
-    public static RateLimiter of(int limit) {
-        if (limit == 0) {
-            return RateLimiter.REJECT;
-        } else if (limit < 0) {
-            return RateLimiter.ACCEPT;
-        } else {
-            return new GuavaRateLimiter(limit);
-        }
-    }
-
-    public GuavaRateLimiter(int limit) {
-        double effectiveLimit = limit;
-        if (limit < 0) {
-            effectiveLimit = Double.POSITIVE_INFINITY;
-        }
-        this.limiter = com.google.common.util.concurrent.RateLimiter.create(effectiveLimit);
-    }
-
-    @Override
-    public boolean acquire(int permits) {
-        Preconditions.checkState(permits >= 0);
-        if (permits > 0) {
-            return limiter.tryAcquire(permits);
-        } else {
-            return true;
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RateLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RateLimiter.java
deleted file mode 100644
index 0cb1ebe..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RateLimiter.java
+++ /dev/null
@@ -1,49 +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 com.twitter.distributedlog.limiter;
-
-/**
- * Simple interface for a rate limiter used by RequestLimiter.
- */
-public interface RateLimiter {
-
-    public static final RateLimiter REJECT = new RateLimiter() {
-        @Override
-        public boolean acquire(int permits) {
-            return false;
-        }
-    };
-
-    public static final RateLimiter ACCEPT = new RateLimiter() {
-        @Override
-        public boolean acquire(int permits) {
-            return true;
-        }
-    };
-
-    public static abstract class Builder {
-        public abstract RateLimiter build();
-    }
-
-    /**
-     * Try to acquire a certain number of permits.
-     *
-     * @param permits number of permits to acquire
-     */
-    boolean acquire(int permits);
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RequestLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RequestLimiter.java
deleted file mode 100644
index 6c5ad96..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RequestLimiter.java
+++ /dev/null
@@ -1,24 +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 com.twitter.distributedlog.limiter;
-
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-
-public interface RequestLimiter<Request> {
-    public void apply(Request request) throws OverCapacityException;
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/package-info.java
deleted file mode 100644
index d5f61a8..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Rate limiting for distributedlog
- */
-package com.twitter.distributedlog.limiter;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLock.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLock.java
deleted file mode 100644
index fa8bdf0..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLock.java
+++ /dev/null
@@ -1,54 +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 com.twitter.distributedlog.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-/**
- * Interface for distributed locking
- */
-public interface DistributedLock extends AsyncCloseable {
-
-    /**
-     * Asynchronously acquire the lock.
-     *
-     * @return future represents the acquire result.
-     */
-    Future<? extends DistributedLock> asyncAcquire();
-
-    /**
-     * Check if hold lock. If it doesn't, then re-acquire the lock.
-     *
-     * @throws LockingException if the lock attempt fails
-     * @see #checkOwnership()
-     */
-    void checkOwnershipAndReacquire() throws LockingException;
-
-    /**
-     * Check if the lock is held. If not, error out and do not re-acquire.
-     * Use this in cases where there are many waiters by default and re-acquire
-     * is unlikely to succeed.
-     *
-     * @throws LockingException if we lost the ownership
-     * @see #checkOwnershipAndReacquire()
-     */
-    void checkOwnership() throws LockingException;
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLockContext.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLockContext.java
deleted file mode 100644
index 1914793..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLockContext.java
+++ /dev/null
@@ -1,43 +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 com.twitter.distributedlog.lock;
-
-import org.apache.commons.lang3.tuple.Pair;
-
-import java.util.HashSet;
-import java.util.Set;
-
-class DistributedLockContext {
-    private final Set<Pair<String, Long>> lockIds;
-
-    DistributedLockContext() {
-        this.lockIds = new HashSet<Pair<String, Long>>();
-    }
-
-    synchronized void addLockId(Pair<String, Long> lockId) {
-        this.lockIds.add(lockId);
-    }
-
-    synchronized void clearLockIds() {
-        this.lockIds.clear();
-    }
-
-    synchronized boolean hasLockId(Pair<String, Long> lockId) {
-        return this.lockIds.contains(lockId);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/EpochChangedException.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/EpochChangedException.java
deleted file mode 100644
index 032a9cd..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/EpochChangedException.java
+++ /dev/null
@@ -1,33 +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 com.twitter.distributedlog.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-
-/**
- * Exception indicates that epoch already changed when executing a given
- * {@link LockAction}.
- */
-public class EpochChangedException extends LockingException {
-
-    private static final long serialVersionUID = 8775257025963870331L;
-
-    public EpochChangedException(String lockPath, int expectedEpoch, int currentEpoch) {
-        super(lockPath, "lock " + lockPath + " already moved to epoch " + currentEpoch + ", expected " + expectedEpoch);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockAction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockAction.java
deleted file mode 100644
index 46b420d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockAction.java
+++ /dev/null
@@ -1,36 +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 com.twitter.distributedlog.lock;
-
-/**
- * Lock Action
- */
-interface LockAction {
-
-    /**
-     * Execute a lock action
-     */
-    void execute();
-
-    /**
-     * Get lock action name.
-     *
-     * @return lock action name
-     */
-    String getActionName();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockClosedException.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockClosedException.java
deleted file mode 100644
index 5b676bf..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockClosedException.java
+++ /dev/null
@@ -1,38 +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 com.twitter.distributedlog.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.lock.ZKSessionLock.State;
-import org.apache.commons.lang3.tuple.Pair;
-
-/**
- * Exception indicates that the lock was closed (unlocked) before the lock request could complete.
- */
-public class LockClosedException extends LockingException {
-
-    private static final long serialVersionUID = 8775257025963470331L;
-
-    public LockClosedException(String lockPath, String msg) {
-        super(lockPath, msg);
-    }
-
-    public LockClosedException(String lockPath, Pair<String, Long> lockId, State currentState) {
-        super(lockPath, "lock at path " + lockPath + " with id " + lockId + " closed early in state : " + currentState);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockListener.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockListener.java
deleted file mode 100644
index 681c180..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockListener.java
+++ /dev/null
@@ -1,28 +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 com.twitter.distributedlog.lock;
-
-/**
- * Listener on lock state changes
- */
-interface LockListener {
-    /**
-     * Triggered when a lock is changed from CLAIMED to EXPIRED.
-     */
-    void onExpired();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockSessionExpiredException.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockSessionExpiredException.java
deleted file mode 100644
index dac1253..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockSessionExpiredException.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 com.twitter.distributedlog.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.lock.ZKSessionLock.State;
-import org.apache.commons.lang3.tuple.Pair;
-
-/**
- * Exception indicates that the lock's zookeeper session was expired before the lock request could complete.
- */
-public class LockSessionExpiredException extends LockingException {
-
-    private static final long serialVersionUID = 8775253025963470331L;
-
-    public LockSessionExpiredException(String lockPath, Pair<String, Long> lockId, State currentState) {
-        super(lockPath, "lock at path " + lockPath + " with id " + lockId + " expired early in state : " + currentState);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockStateChangedException.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockStateChangedException.java
deleted file mode 100644
index 2b99795..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockStateChangedException.java
+++ /dev/null
@@ -1,36 +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 com.twitter.distributedlog.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.lock.ZKSessionLock.State;
-import org.apache.commons.lang3.tuple.Pair;
-
-/**
- * Exception thrown when lock state changed
- */
-public class LockStateChangedException extends LockingException {
-
-    private static final long serialVersionUID = -3770866789942102262L;
-
-    LockStateChangedException(String lockPath, Pair<String, Long> lockId,
-                              State expectedState, State currentState) {
-        super(lockPath, "Lock state of " + lockId + " for " + lockPath + " has changed : expected "
-                + expectedState + ", but " + currentState);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockTimeoutException.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockTimeoutException.java
deleted file mode 100644
index 3020980..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockTimeoutException.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 com.twitter.distributedlog.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Exception thrown when acquiring lock timeout
- */
-public class LockTimeoutException extends LockingException {
-
-    private static final long serialVersionUID = -3837638877423323820L;
-
-    LockTimeoutException(String lockPath, long timeout, TimeUnit unit) {
-        super(lockPath, "Locking " + lockPath + " timeout in " + timeout + " " + unit);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockWaiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockWaiter.java
deleted file mode 100644
index 73ffabc..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockWaiter.java
+++ /dev/null
@@ -1,95 +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 com.twitter.distributedlog.lock;
-
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import com.twitter.util.Timer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Lock waiter represents the attempt that application tries to lock.
- */
-public class LockWaiter {
-
-    private static final Logger logger = LoggerFactory.getLogger(LockWaiter.class);
-
-    private final String lockId;
-    private final String currentOwner;
-    private final Future<Boolean> acquireFuture;
-
-    public LockWaiter(String lockId,
-                      String currentOwner,
-                      Future<Boolean> acquireFuture) {
-        this.lockId = lockId;
-        this.currentOwner = currentOwner;
-        this.acquireFuture = acquireFuture;
-    }
-
-    /**
-     * Return the lock id of the waiter.
-     *
-     * @return lock id of the waiter
-     */
-    public String getId() {
-        return lockId;
-    }
-
-    /**
-     * Return the owner that observed when waiter is waiting.
-     *
-     * @return the owner that observed when waiter is waiting
-     */
-    public String getCurrentOwner() {
-        return currentOwner;
-    }
-
-    /**
-     * Return the future representing the waiting result.
-     *
-     * <p>If the future is interrupted (e.g. {@link Future#within(Duration, Timer)}),
-     * the waiter will automatically clean up its waiting state.
-     *
-     * @return the future representing the acquire result.
-     */
-    public Future<Boolean> getAcquireFuture() {
-        return acquireFuture;
-    }
-
-    /**
-     * Wait for the acquire result.
-     *
-     * @return true if acquired successfully, otherwise false.
-     */
-    public boolean waitForAcquireQuietly() {
-        boolean success = false;
-        try {
-            success = Await.result(acquireFuture);
-        } catch (InterruptedException ie) {
-            Thread.currentThread().interrupt();
-        } catch (LockTimeoutException lte) {
-            logger.debug("Timeout on lock acquiring", lte);
-        } catch (Exception e) {
-            logger.error("Caught exception waiting for lock acquired", e);
-        }
-        return success;
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/NopDistributedLock.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/NopDistributedLock.java
deleted file mode 100644
index ef6b9ab..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/NopDistributedLock.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 com.twitter.distributedlog.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.util.Future;
-
-/**
- * An implementation of {@link DistributedLock} which does nothing.
- */
-public class NopDistributedLock implements DistributedLock {
-
-    public static final DistributedLock INSTANCE = new NopDistributedLock();
-
-    private NopDistributedLock() {}
-
-    @Override
-    public Future<? extends DistributedLock> asyncAcquire() {
-        return Future.value(this);
-    }
-
-    @Override
-    public void checkOwnershipAndReacquire() throws LockingException {
-        // no-op
-    }
-
-    @Override
-    public void checkOwnership() throws LockingException {
-        // no-op
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        return Future.Void();
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLock.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLock.java
deleted file mode 100644
index 95cd593..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLock.java
+++ /dev/null
@@ -1,126 +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 com.twitter.distributedlog.lock;
-
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.util.Future;
-import scala.runtime.BoxedUnit;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * One time lock.
- * <p>The lock is only alive during a given period. It should
- * be not usable if the lock is expired.
- * <p>Listener could be registered by {@link #setLockListener(LockListener)}
- * to receive state changes of the lock.
- */
-public interface SessionLock {
-
-    /**
-     * Set lock listener for lock state changes.
-     * <p>Typically a listener should be set before try locking.
-     *
-     * @param lockListener
-     *          lock listener for state changes.
-     */
-    SessionLock setLockListener(LockListener lockListener);
-
-    /**
-     * Whether the lock is held or not?
-     *
-     * @return true if the lock is held, otherwise false.
-     */
-    boolean isLockHeld();
-
-    /**
-     * Whether the lock is expired or not?
-     * <p>If a lock is expired, it will not be reusable any more. Because it is an one-time lock.
-     *
-     * @return true if the lock is expired, otherwise false.
-     */
-    boolean isLockExpired();
-
-    /**
-     * Acquire the lock if it is free within given waiting time.
-     * <p>
-     * Calling this method will attempt to acquire the lock. If the lock
-     * is already acquired by others, the caller will wait for <i>timeout</i>
-     * period. If the caller could claim the lock within <i>timeout</i> period,
-     * the caller acquire the lock. Otherwise, it would fail with {@link OwnershipAcquireFailedException}.
-     * <p>
-     * {@link #unlock()} should be called to unlock a claimed lock. The caller
-     * doesn't need to unlock to clean up resources if <i>tryLock</i> fails.
-     * <p>
-     * <i>tryLock</i> here is effectively the combination of following asynchronous calls.
-     * <pre>
-     *     ZKDistributedLock lock = ...;
-     *     Future<LockWaiter> attemptFuture = lock.asyncTryLock(...);
-     *
-     *     boolean acquired = waiter.waitForAcquireQuietly();
-     *     if (acquired) {
-     *         // ...
-     *     } else {
-     *         // ...
-     *     }
-     * </pre>
-     *
-     * @param timeout
-     *          timeout period to wait for claiming ownership
-     * @param unit
-     *          unit of timeout period
-     * @throws OwnershipAcquireFailedException if the lock is already acquired by others
-     * @throws LockingException when encountered other lock related issues.
-     */
-    void tryLock(long timeout, TimeUnit unit)
-            throws OwnershipAcquireFailedException, LockingException;
-
-    /**
-     * Acquire the lock in asynchronous way.
-     * <p>
-     * Calling this method will attempt to place a lock waiter to acquire this lock.
-     * The future returned by this method represents the result of this attempt. It doesn't mean
-     * the caller acquired the lock or not. The application should check {@link LockWaiter#getAcquireFuture()}
-     * to see if it acquired the lock or not.
-     *
-     * @param timeout
-     *          timeout period to wait for claiming ownership
-     * @param unit
-     *          unit of timeout period
-     * @return lock waiter representing this attempt of acquiring lock.
-     * @see #tryLock(long, TimeUnit)
-     */
-    Future<LockWaiter> asyncTryLock(long timeout, TimeUnit unit);
-
-    /**
-     * Release a claimed lock.
-     *
-     * @see #tryLock(long, TimeUnit)
-     */
-    void unlock();
-
-    /**
-     * Release a claimed lock in the asynchronous way.
-     *
-     * @return future representing the result of unlock operation.
-     * @see #unlock()
-     */
-    Future<BoxedUnit> asyncUnlock();
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLockFactory.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLockFactory.java
deleted file mode 100644
index 4334626..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLockFactory.java
+++ /dev/null
@@ -1,38 +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 com.twitter.distributedlog.lock;
-
-import com.twitter.util.Future;
-
-/**
- * Factory to create {@link SessionLock}
- */
-public interface SessionLockFactory {
-
-    /**
-     * Create a lock with lock path.
-     *
-     * @param lockPath
-     *          lock path
-     * @param context
-     *          lock context
-     * @return future represents the creation result.
-     */
-    Future<SessionLock> createLock(String lockPath, DistributedLockContext context);
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKDistributedLock.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKDistributedLock.java
deleted file mode 100644
index 4d0de7f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKDistributedLock.java
+++ /dev/null
@@ -1,537 +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 com.twitter.distributedlog.lock;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Stopwatch;
-import com.twitter.concurrent.AsyncSemaphore;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.FutureUtils.OrderedFutureEventListener;
-import com.twitter.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.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-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.
- * <p/>
- * The lock is vulnerable to timing issues. For example, the process could
- * encounter a really long GC cycle between acquiring the lock, and writing to
- * a ledger. This could have timed out the lock, and another process could have
- * acquired the lock and started writing to bookkeeper. Therefore other
- * mechanisms are required to ensure correctness (i.e. Fencing).
- * <p/>
- * The lock is only allowed to acquire once. If the lock is acquired successfully,
- * the caller holds the ownership until it loses the ownership either because of
- * others already acquired the lock when session expired or explicitly close it.
- * <p>
- * The caller could use {@link #checkOwnership()} or {@link #checkOwnershipAndReacquire()}
- * to check if it still holds the lock. If it doesn't hold the lock, the caller should
- * give up the ownership and close the lock.
- * <h3>Metrics</h3>
- * All the lock related stats are exposed under `lock`.
- * <ul>
- * <li>lock/acquire: opstats. latency spent on acquiring a lock.
- * <li>lock/reacquire: opstats. latency spent on re-acquiring a lock.
- * <li>lock/internalTryRetries: counter. the number of retries on re-creating internal locks.
- * </ul>
- * Other internal lock related stats are also exposed under `lock`. See {@link SessionLock}
- * for details.
- */
-public class ZKDistributedLock implements LockListener, DistributedLock {
-
-    static final Logger LOG = LoggerFactory.getLogger(ZKDistributedLock.class);
-
-    private final SessionLockFactory lockFactory;
-    private final OrderedScheduler lockStateExecutor;
-    private final String lockPath;
-    private final long lockTimeout;
-    private final DistributedLockContext lockContext = new DistributedLockContext();
-
-    private final AsyncSemaphore lockSemaphore = new AsyncSemaphore(1);
-    // 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;
-    // following variable tracking the status of acquire process
-    //   => create (internalLock) => tryLock (tryLockFuture) => waitForAcquire (lockWaiter)
-    private SessionLock internalLock = null;
-    private Future<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;
-
-    // A counter to track how many re-acquires happened during a lock's life cycle.
-    private final AtomicInteger reacquireCount = new AtomicInteger(0);
-    private final StatsLogger lockStatsLogger;
-    private final OpStatsLogger acquireStats;
-    private final OpStatsLogger reacquireStats;
-    private final Counter internalTryRetries;
-
-    public ZKDistributedLock(
-            OrderedScheduler lockStateExecutor,
-            SessionLockFactory lockFactory,
-            String lockPath,
-            long lockTimeout,
-            StatsLogger statsLogger) {
-        this.lockStateExecutor = lockStateExecutor;
-        this.lockPath = lockPath;
-        this.lockTimeout = lockTimeout;
-        this.lockFactory = lockFactory;
-
-        lockStatsLogger = statsLogger.scope("lock");
-        acquireStats = lockStatsLogger.getOpStatsLogger("acquire");
-        reacquireStats = lockStatsLogger.getOpStatsLogger("reacquire");
-        internalTryRetries = lockStatsLogger.getCounter("internalTryRetries");
-    }
-
-    private LockClosedException newLockClosedException() {
-        return new LockClosedException(lockPath, "Lock is already closed");
-    }
-
-    private synchronized void checkLockState() throws LockingException {
-        if (closed) {
-            throw newLockClosedException();
-        }
-        if (null != lockReacquireException) {
-            throw lockReacquireException;
-        }
-    }
-
-    /**
-     * 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() {
-        if (null != lockAcquireFuture) {
-            return Future.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 Stopwatch stopwatch = Stopwatch.createStarted();
-        promise.addEventListener(new FutureEventListener<ZKDistributedLock>() {
-            @Override
-            public void onSuccess(ZKDistributedLock lock) {
-                acquireStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            }
-            @Override
-            public void onFailure(Throwable cause) {
-                acquireStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-                // release the lock if fail to acquire
-                asyncClose();
-            }
-        });
-        this.lockAcquireFuture = promise;
-        lockStateExecutor.submit(lockPath, new Runnable() {
-            @Override
-            public void run() {
-                doAsyncAcquireWithSemaphore(promise, lockTimeout);
-            }
-        });
-        return promise;
-    }
-
-    void doAsyncAcquireWithSemaphore(final Promise<ZKDistributedLock> acquirePromise,
-                                     final long lockTimeout) {
-        lockSemaphore.acquireAndRun(new AbstractFunction0<Future<ZKDistributedLock>>() {
-            @Override
-            public Future<ZKDistributedLock> apply() {
-                doAsyncAcquire(acquirePromise, lockTimeout);
-                return acquirePromise;
-            }
-        });
-    }
-
-    void doAsyncAcquire(final Promise<ZKDistributedLock> acquirePromise,
-                        final long lockTimeout) {
-        LOG.trace("Async Lock Acquire {}", lockPath);
-        try {
-            checkLockState();
-        } catch (IOException ioe) {
-            FutureUtils.setException(acquirePromise, ioe);
-            return;
-        }
-
-        if (haveLock()) {
-            // it already hold the lock
-            FutureUtils.setValue(acquirePromise, this);
-            return;
-        }
-
-        lockFactory.createLock(lockPath, lockContext).addEventListener(OrderedFutureEventListener.of(
-                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());
-                        return;
-                    }
-                }
-                synchronized (ZKDistributedLock.this) {
-                    internalLock = lock;
-                    internalLock.setLockListener(ZKDistributedLock.this);
-                }
-                asyncTryLock(lock, acquirePromise, lockTimeout);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                FutureUtils.setException(acquirePromise, cause);
-            }
-        }, lockStateExecutor, lockPath));
-    }
-
-    void asyncTryLock(SessionLock lock,
-                      final Promise<ZKDistributedLock> acquirePromise,
-                      final long lockTimeout) {
-        if (null != tryLockFuture) {
-            tryLockFuture.cancel();
-        }
-        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 = null;
-                        lockWaiter = waiter;
-                        waitForAcquire(waiter, acquirePromise);
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        FutureUtils.setException(acquirePromise, cause);
-                    }
-                }, lockStateExecutor, 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()));
-                        }
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        FutureUtils.setException(acquirePromise, cause);
-                    }
-                }, lockStateExecutor, lockPath));
-    }
-
-    /**
-     * NOTE: The {@link LockListener#onExpired()} is already executed in lock executor.
-     */
-    @Override
-    public void onExpired() {
-        try {
-            reacquireLock(false);
-        } catch (LockingException le) {
-            // should not happen
-            LOG.error("Locking exception on re-acquiring lock {} : ", lockPath, le);
-        }
-    }
-
-    /**
-     * Check if hold lock, if it doesn't, then re-acquire the lock.
-     *
-     * @throws LockingException     if the lock attempt fails
-     */
-    public synchronized void checkOwnershipAndReacquire() throws LockingException {
-        if (null == lockAcquireFuture || !lockAcquireFuture.isDefined()) {
-            throw new LockingException(lockPath, "check ownership before acquiring");
-        }
-
-        if (haveLock()) {
-            return;
-        }
-
-        // We may have just lost the lock because of a ZK session timeout
-        // not necessarily because someone else acquired the lock.
-        // In such cases just try to reacquire. If that fails, it will throw
-        reacquireLock(true);
-    }
-
-    /**
-     * Check if lock is held.
-     * If not, error out and do not reacquire. Use this in cases where there are many waiters by default
-     * and reacquire is unlikley to succeed.
-     *
-     * @throws LockingException     if the lock attempt fails
-     */
-    public synchronized void checkOwnership() throws LockingException {
-        if (null == lockAcquireFuture || !lockAcquireFuture.isDefined()) {
-            throw new LockingException(lockPath, "check ownership before acquiring");
-        }
-        if (!haveLock()) {
-            throw new LockingException(lockPath, "Lost lock ownership");
-        }
-    }
-
-    @VisibleForTesting
-    int getReacquireCount() {
-        return reacquireCount.get();
-    }
-
-    @VisibleForTesting
-    synchronized Future<ZKDistributedLock> getLockReacquireFuture() {
-        return lockReacquireFuture;
-    }
-
-    @VisibleForTesting
-    synchronized Future<ZKDistributedLock> getLockAcquireFuture() {
-        return lockAcquireFuture;
-    }
-
-    @VisibleForTesting
-    synchronized SessionLock getInternalLock() {
-        return internalLock;
-    }
-
-    @VisibleForTesting
-    LockWaiter getLockWaiter() {
-        return lockWaiter;
-    }
-
-    synchronized boolean haveLock() {
-        return !closed && internalLock != null && internalLock.isLockHeld();
-    }
-
-    void closeWaiter(final LockWaiter waiter,
-                     final Promise<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());
-        }
-    }
-
-    void interruptTryLock(final Future<LockWaiter> tryLockFuture,
-                          final Promise<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);
-        }
-    }
-
-    synchronized void unlockInternalLock(final Promise<Void> closePromise) {
-        if (internalLock == null) {
-            FutureUtils.setValue(closePromise, null);
-        } else {
-            internalLock.asyncUnlock().ensure(new AbstractFunction0<BoxedUnit>() {
-                @Override
-                public BoxedUnit apply() {
-                    FutureUtils.setValue(closePromise, null);
-                    return BoxedUnit.UNIT;
-                }
-            });
-        }
-    }
-
-    @Override
-    public Future<Void> asyncClose() {
-        final Promise<Void> closePromise;
-        synchronized (this) {
-            if (closed) {
-                return closeFuture;
-            }
-            closed = true;
-            closeFuture = closePromise = new Promise<Void>();
-        }
-        final Promise<Void> closeWaiterFuture = new Promise<Void>();
-        closeWaiterFuture.addEventListener(OrderedFutureEventListener.of(new FutureEventListener<Void>() {
-            @Override
-            public void onSuccess(Void value) {
-                complete();
-            }
-            @Override
-            public void onFailure(Throwable cause) {
-                complete();
-            }
-
-            private void complete() {
-                FutureUtils.setValue(closePromise, null);
-            }
-        }, lockStateExecutor, lockPath));
-        lockStateExecutor.submit(lockPath, new Runnable() {
-            @Override
-            public void run() {
-                closeWaiter(lockWaiter, closeWaiterFuture);
-            }
-        });
-        return closePromise;
-    }
-
-    void internalReacquireLock(final AtomicInteger numRetries,
-                               final long lockTimeout,
-                               final Promise<ZKDistributedLock> reacquirePromise) {
-        lockStateExecutor.submit(lockPath, new Runnable() {
-            @Override
-            public void run() {
-                doInternalReacquireLock(numRetries, lockTimeout, reacquirePromise);
-            }
-        });
-    }
-
-    void doInternalReacquireLock(final AtomicInteger numRetries,
-                                 final long lockTimeout,
-                                 final Promise<ZKDistributedLock> reacquirePromise) {
-        internalTryRetries.inc();
-        Promise<ZKDistributedLock> tryPromise = new Promise<ZKDistributedLock>();
-        tryPromise.addEventListener(new FutureEventListener<ZKDistributedLock>() {
-            @Override
-            public void onSuccess(ZKDistributedLock lock) {
-                FutureUtils.setValue(reacquirePromise, lock);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                if (cause instanceof OwnershipAcquireFailedException) {
-                    // the lock has been acquired by others
-                    FutureUtils.setException(reacquirePromise, cause);
-                } else {
-                    if (numRetries.getAndDecrement() > 0 && !closed) {
-                        internalReacquireLock(numRetries, lockTimeout, reacquirePromise);
-                    } else {
-                        FutureUtils.setException(reacquirePromise, cause);
-                    }
-                }
-            }
-        });
-        doAsyncAcquireWithSemaphore(tryPromise, 0);
-    }
-
-    private Future<ZKDistributedLock> reacquireLock(boolean throwLockAcquireException) throws LockingException {
-        final Stopwatch stopwatch = Stopwatch.createStarted();
-        Promise<ZKDistributedLock> lockPromise;
-        synchronized (this) {
-            if (closed) {
-                throw newLockClosedException();
-            }
-            if (null != lockReacquireException) {
-                if (throwLockAcquireException) {
-                    throw lockReacquireException;
-                } else {
-                    return null;
-                }
-            }
-            if (null != lockReacquireFuture) {
-                return lockReacquireFuture;
-            }
-            LOG.info("reacquiring lock at {}", lockPath);
-            lockReacquireFuture = lockPromise = new Promise<ZKDistributedLock>();
-            lockReacquireFuture.addEventListener(new FutureEventListener<ZKDistributedLock>() {
-                @Override
-                public void onSuccess(ZKDistributedLock lock) {
-                    // if re-acquire successfully, clear the state.
-                    synchronized (ZKDistributedLock.this) {
-                        lockReacquireFuture = null;
-                    }
-                    reacquireStats.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-                }
-
-                @Override
-                public void onFailure(Throwable cause) {
-                    synchronized (ZKDistributedLock.this) {
-                        if (cause instanceof LockingException) {
-                            lockReacquireException = (LockingException) cause;
-                        } else {
-                            lockReacquireException = new LockingException(lockPath,
-                                    "Exception on re-acquiring lock", cause);
-                        }
-                    }
-                    reacquireStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-                }
-            });
-        }
-        reacquireCount.incrementAndGet();
-        internalReacquireLock(new AtomicInteger(Integer.MAX_VALUE), 0, lockPromise);
-        return lockPromise;
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLock.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLock.java
deleted file mode 100644
index dc57d55..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLock.java
+++ /dev/null
@@ -1,1363 +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 com.twitter.distributedlog.lock;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.stats.OpStatsListener;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.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;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.util.SafeRunnable;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-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.
- * It is not reusable: if lock failed, if zookeeper session is expired, if #unlock is called,
- * it would be transitioned to expired or closed state.
- *
- * The Locking Procedure is described as below.
- *
- * <p>
- * 0. if it is an immediate lock, it would get lock waiters first. if the lock is already held
- *    by someone. it would fail immediately with {@link com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException}
- *    with current owner. if there is no lock waiters, it would start locking procedure from 1.
- * 1. prepare: create a sequential znode to identify the lock.
- * 2. check lock waiters: get all lock waiters to check after prepare. if it is the first waiter,
- *    claim the ownership; if it is not the first waiter, but first waiter was itself (same client id and same session id)
- *    claim the ownership too; otherwise, it would set watcher on its sibling and wait it to disappared.
- * </p>
- *
- * <pre>
- *                      +-----------------+
- *                      |       INIT      | ------------------------------+
- *                      +--------+--------+                               |
- *                               |                                        |
- *                               |                                        |
- *                      +--------v--------+                               |
- *                      |    PREPARING    |----------------------------+  |
- *                      +--------+--------+                            |  |
- *                               |                                     |  |
- *                               |                                     |  |
- *                      +--------v--------+                            |  |
- *        +-------------|    PREPARED     |--------------+             |  |
- *        |             +-----^---------+-+              |             |  |
- *        |                   |  |      |                |             |  |
- *        |                   |  |      |                |             |  |
- *        |                   |  |      |                |             |  |
- * +------V-----------+       |  |      |       +--------v----------+  |  |
- * |     WAITING      |-------+  |      |       |    CLAIMED        |  |  |
- * +------+-----+-----+          |      |       +--+----------+-----+  |  |
- *        |     |                |      |          |        |          |  |
- *        |     |                |      |          |        |          |  |
- *        |     |                |      |          |        |          |  |
- *        |     |                |    +-v----------v----+   |          |  |
- *        |     +-------------------->|     EXPIRED     |   |          |  |
- *        |                      |    +--+--------------+   |          |  |
- *        |                      |       |                  |          |  |
- *        |                      |       |                  |          |  |
- *        |             +--------V-------V-+                |          |  |
- *        +------------>|     CLOSING      |<---------------+----------+--+
- *                      +------------------+
- *                               |
- *                               |
- *                               |
- *                      +--------V---------+
- *                      |     CLOSED       |
- *                      +------------------+
- * </pre>
- *
- * <h3>Metrics</h3>
- * <ul>
- * <li>tryAcquire: opstats. latency spent on try locking operations. it includes timeouts.
- * <li>tryTimeouts: counter. the number of timeouts on try locking operations
- * <li>unlock: opstats. latency spent on unlock operations.
- * </ul>
- */
-class ZKSessionLock implements SessionLock {
-
-    static final Logger LOG = LoggerFactory.getLogger(ZKSessionLock.class);
-
-    private static final String LOCK_PATH_PREFIX = "/member_";
-    private static final String LOCK_PART_SEP = "_";
-
-    public static String getLockPathPrefixV1(String lockPath) {
-        // member_
-        return lockPath + LOCK_PATH_PREFIX;
-    }
-
-    public static String getLockPathPrefixV2(String lockPath, String clientId) throws UnsupportedEncodingException {
-        // member_<clientid>_
-        return lockPath + LOCK_PATH_PREFIX + URLEncoder.encode(clientId, UTF_8.name()) + LOCK_PART_SEP;
-    }
-
-    public static String getLockPathPrefixV3(String lockPath, String clientId, long sessionOwner) throws UnsupportedEncodingException {
-        // member_<clientid>_s<owner_session>_
-        StringBuilder sb = new StringBuilder();
-        sb.append(lockPath).append(LOCK_PATH_PREFIX).append(URLEncoder.encode(clientId, UTF_8.name())).append(LOCK_PART_SEP)
-                .append("s").append(String.format("%10d", sessionOwner)).append(LOCK_PART_SEP);
-        return sb.toString();
-    }
-
-    public static byte[] serializeClientId(String clientId) {
-        return clientId.getBytes(UTF_8);
-    }
-
-    public static String deserializeClientId(byte[] data) {
-        return new String(data, UTF_8);
-    }
-
-    public static String getLockIdFromPath(String path) {
-        // We only care about our actual id since we want to compare ourselves to siblings.
-        if (path.contains("/")) {
-            return path.substring(path.lastIndexOf("/") + 1);
-        } else {
-            return path;
-        }
-    }
-
-    static final Comparator<String> MEMBER_COMPARATOR = new Comparator<String>() {
-        public int compare(String o1, String o2) {
-            int l1 = parseMemberID(o1);
-            int l2 = parseMemberID(o2);
-            return l1 - l2;
-        }
-    };
-
-    static enum State {
-        INIT,      // initialized state
-        PREPARING, // preparing to lock, but no lock node created
-        PREPARED,  // lock node created
-        CLAIMED,   // claim lock ownership
-        WAITING,   // waiting for the ownership
-        EXPIRED,   // lock is expired
-        CLOSING,   // lock is being closed
-        CLOSED,    // lock is closed
-    }
-
-    /**
-     * Convenience class for state management. Provide debuggability features by tracing unxpected state
-     * transitions.
-     */
-    static class StateManagement {
-
-        static final Logger LOG = LoggerFactory.getLogger(StateManagement.class);
-
-        private volatile State state;
-
-        StateManagement() {
-            this.state = State.INIT;
-        }
-
-        public void transition(State toState) {
-            if (!validTransition(toState)) {
-                LOG.error("Invalid state transition from {} to {} ",
-                        new Object[] { this.state, toState, getStack() });
-            }
-            this.state = toState;
-        }
-
-        private boolean validTransition(State toState) {
-            switch (toState) {
-                case INIT:
-                    return false;
-                case PREPARING:
-                    return inState(State.INIT);
-                case PREPARED:
-                    return inState(State.PREPARING) || inState(State.WAITING);
-                case CLAIMED:
-                    return inState(State.PREPARED);
-                case WAITING:
-                    return inState(State.PREPARED);
-                case EXPIRED:
-                    return isTryingOrClaimed();
-                case CLOSING:
-                    return !inState(State.CLOSED);
-                case CLOSED:
-                    return inState(State.CLOSING) || inState(State.CLOSED);
-                default:
-                    return false;
-            }
-        }
-
-        private State getState() {
-            return state;
-        }
-
-        private boolean isTryingOrClaimed() {
-            return inState(State.PREPARING) || inState(State.PREPARED) ||
-                inState(State.WAITING) || inState(State.CLAIMED);
-        }
-
-        public boolean isExpiredOrClosing() {
-            return inState(State.CLOSED) || inState(State.EXPIRED) || inState(State.CLOSING);
-        }
-
-        public boolean isExpiredOrClosed() {
-            return inState(State.CLOSED) || inState(State.EXPIRED);
-        }
-
-        public boolean isClosed() {
-            return inState(State.CLOSED);
-        }
-
-        private boolean inState(final State state) {
-            return state == this.state;
-        }
-
-        private Exception getStack() {
-            return new Exception();
-        }
-    }
-
-    private final ZooKeeperClient zkClient;
-    private final ZooKeeper zk;
-    private final String lockPath;
-    // Identify a unique lock
-    private final Pair<String, Long> lockId;
-    private StateManagement lockState;
-    private final DistributedLockContext lockContext;
-
-    private final Promise<Boolean> acquireFuture;
-    private String currentId;
-    private String currentNode;
-    private String watchedNode;
-    private LockWatcher watcher;
-    private final AtomicInteger epoch = new AtomicInteger(0);
-    private final OrderedScheduler lockStateExecutor;
-    private LockListener lockListener = null;
-    private final long lockOpTimeout;
-
-    private final OpStatsLogger tryStats;
-    private final Counter tryTimeouts;
-    private final OpStatsLogger unlockStats;
-
-    ZKSessionLock(ZooKeeperClient zkClient,
-                  String lockPath,
-                  String clientId,
-                  OrderedScheduler lockStateExecutor)
-            throws IOException {
-        this(zkClient,
-                lockPath,
-                clientId,
-                lockStateExecutor,
-                DistributedLogConstants.LOCK_OP_TIMEOUT_DEFAULT * 1000, NullStatsLogger.INSTANCE,
-                new DistributedLockContext());
-    }
-
-    /**
-     * Creates a distributed lock using the given {@code zkClient} to coordinate locking.
-     *
-     * @param zkClient The ZooKeeper client to use.
-     * @param lockPath The path used to manage the lock under.
-     * @param clientId client id use for lock.
-     * @param lockStateExecutor executor to execute all lock state changes.
-     * @param lockOpTimeout timeout of lock operations
-     * @param statsLogger stats logger
-     */
-    public ZKSessionLock(ZooKeeperClient zkClient,
-                         String lockPath,
-                         String clientId,
-                         OrderedScheduler lockStateExecutor,
-                         long lockOpTimeout,
-                         StatsLogger statsLogger,
-                         DistributedLockContext lockContext)
-            throws IOException {
-        this.zkClient = zkClient;
-        try {
-            this.zk = zkClient.get();
-        } catch (ZooKeeperClient.ZooKeeperConnectionException zce) {
-            throw new ZKException("Failed to get zookeeper client for lock " + lockPath,
-                    KeeperException.Code.CONNECTIONLOSS);
-        } catch (InterruptedException e) {
-            throw new DLInterruptedException("Interrupted on getting zookeeper client for lock " + lockPath, e);
-        }
-        this.lockPath = lockPath;
-        this.lockId = Pair.of(clientId, this.zk.getSessionId());
-        this.lockContext = lockContext;
-        this.lockStateExecutor = lockStateExecutor;
-        this.lockState = new StateManagement();
-        this.lockOpTimeout = lockOpTimeout;
-
-        this.tryStats = statsLogger.getOpStatsLogger("tryAcquire");
-        this.tryTimeouts = statsLogger.getCounter("tryTimeouts");
-        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 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);
-                // Note re. logging and exceptions: errors are already logged by unlockAsync.
-                return BoxedUnit.UNIT;
-            }
-        });
-    }
-
-    @Override
-    public ZKSessionLock setLockListener(LockListener lockListener) {
-        this.lockListener = lockListener;
-        return this;
-    }
-
-    String getLockPath() {
-        return this.lockPath;
-    }
-
-    @VisibleForTesting
-    AtomicInteger getEpoch() {
-        return epoch;
-    }
-
-    @VisibleForTesting
-    State getLockState() {
-        return lockState.getState();
-    }
-
-    @VisibleForTesting
-    Pair<String, Long> getLockId() {
-        return lockId;
-    }
-
-    public boolean isLockExpired() {
-        return lockState.isExpiredOrClosing();
-    }
-
-    @Override
-    public boolean isLockHeld() {
-        return lockState.inState(State.CLAIMED);
-    }
-
-    /**
-     * Execute a lock action of a given <i>lockEpoch</i> in ordered safe way.
-     *
-     * @param lockEpoch
-     *          lock epoch
-     * @param func
-     *          function to execute a lock action
-     */
-    protected void executeLockAction(final int lockEpoch, final LockAction func) {
-        lockStateExecutor.submit(lockPath, new SafeRunnable() {
-            @Override
-            public void safeRun() {
-                if (ZKSessionLock.this.epoch.get() == lockEpoch) {
-                    if (LOG.isTraceEnabled()) {
-                        LOG.trace("{} executing lock action '{}' under epoch {} for lock {}",
-                                new Object[]{lockId, func.getActionName(), lockEpoch, lockPath});
-                    }
-                    func.execute();
-                    if (LOG.isTraceEnabled()) {
-                        LOG.trace("{} executed lock action '{}' under epoch {} for lock {}",
-                                new Object[]{lockId, func.getActionName(), lockEpoch, lockPath});
-                    }
-                } else {
-                    if (LOG.isTraceEnabled()) {
-                        LOG.trace("{} skipped executing lock action '{}' for lock {}, since epoch is changed from {} to {}.",
-                                new Object[]{lockId, func.getActionName(), lockPath, lockEpoch, ZKSessionLock.this.epoch.get()});
-                    }
-                }
-            }
-        });
-    }
-
-    /**
-     * Execute a lock action of a given <i>lockEpoch</i> in ordered safe way. If the lock action couln't be
-     * executed due to epoch changed, fail the given <i>promise</i> with
-     * {@link EpochChangedException}
-     *
-     * @param lockEpoch
-     *          lock epoch
-     * @param func
-     *          function to execute a lock action
-     * @param promise
-     *          promise
-     */
-    protected <T> void executeLockAction(final int lockEpoch, final LockAction func, final Promise<T> promise) {
-        lockStateExecutor.submit(lockPath, new SafeRunnable() {
-            @Override
-            public void safeRun() {
-                int currentEpoch = ZKSessionLock.this.epoch.get();
-                if (currentEpoch == lockEpoch) {
-                    if (LOG.isTraceEnabled()) {
-                        LOG.trace("{} executed lock action '{}' under epoch {} for lock {}",
-                                new Object[]{lockId, func.getActionName(), lockEpoch, lockPath});
-                    }
-                    func.execute();
-                    if (LOG.isTraceEnabled()) {
-                        LOG.trace("{} executed lock action '{}' under epoch {} for lock {}",
-                                new Object[]{lockId, func.getActionName(), lockEpoch, lockPath});
-                    }
-                } else {
-                    if (LOG.isTraceEnabled()) {
-                        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));
-                }
-            }
-        });
-    }
-
-    /**
-     * Parse member id generated by zookeeper from given <i>nodeName</i>
-     *
-     * @param nodeName
-     *          lock node name
-     * @return member id generated by zookeeper
-     */
-    static int parseMemberID(String nodeName) {
-        int id = -1;
-        String[] parts = nodeName.split("_");
-        if (parts.length > 0) {
-            try {
-                id = Integer.parseInt(parts[parts.length - 1]);
-            } catch (NumberFormatException nfe) {
-                // make it to be MAX_VALUE, so the bad znode will never acquire the lock
-                id = Integer.MAX_VALUE;
-            }
-        }
-        return id;
-    }
-
-    static boolean areLockWaitersInSameSession(String node1, String node2) {
-        String[] parts1 = node1.split("_");
-        String[] parts2 = node2.split("_");
-        if (parts1.length != 4 || parts2.length != 4) {
-            return node1.equals(node2);
-        }
-        if (!parts1[2].startsWith("s") || !parts2[2].startsWith("s")) {
-            return node1.equals(node2);
-        }
-        long sessionOwner1 = Long.parseLong(parts1[2].substring(1));
-        long sessionOwner2 = Long.parseLong(parts2[2].substring(1));
-        if (sessionOwner1 != sessionOwner2) {
-            return false;
-        }
-        String clientId1, clientId2;
-        try {
-            clientId1 = URLDecoder.decode(parts1[1], UTF_8.name());
-            clientId2 = URLDecoder.decode(parts2[1], UTF_8.name());
-            return clientId1.equals(clientId2);
-        } catch (UnsupportedEncodingException e) {
-            // if failed to parse client id, we have to get client id by zookeeper#getData.
-            return node1.equals(node2);
-        }
-    }
-
-    /**
-     * Get client id and its ephemeral owner.
-     *
-     * @param zkClient
-     *          zookeeper client
-     * @param lockPath
-     *          lock path
-     * @param nodeName
-     *          node name
-     * @return client id and its ephemeral owner.
-     */
-    static Future<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")) {
-            long sessionOwner = Long.parseLong(parts[2].substring(1));
-            String clientId;
-            try {
-                clientId = URLDecoder.decode(parts[1], UTF_8.name());
-                return Future.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>>();
-        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)));
-                } else {
-                    promise.setValue(Pair.of(deserializeClientId(data), stat.getEphemeralOwner()));
-                }
-            }
-        }, null);
-        return promise;
-    }
-
-    @Override
-    public Future<LockWaiter> asyncTryLock(final long timeout, final TimeUnit unit) {
-        final Promise<String> result = new Promise<String>();
-        final boolean wait = DistributedLogConstants.LOCK_IMMEDIATE != timeout;
-        if (wait) {
-            asyncTryLock(wait, result);
-        } else {
-            // try to check locks first
-            zk.getChildren(lockPath, null, new AsyncCallback.Children2Callback() {
-                @Override
-                public void processResult(final int rc, String path, Object ctx,
-                                          final List<String> children, Stat stat) {
-                    lockStateExecutor.submit(lockPath, new SafeRunnable() {
-                        @Override
-                        public void safeRun() {
-                            if (!lockState.inState(State.INIT)) {
-                                result.setException(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
-                                return;
-                            }
-                            if (KeeperException.Code.OK.intValue() != rc) {
-                                result.setException(KeeperException.create(KeeperException.Code.get(rc)));
-                                return;
-                            }
-
-                            FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockTryAcquire);
-
-                            Collections.sort(children, MEMBER_COMPARATOR);
-                            if (children.size() > 0) {
-                                asyncParseClientID(zk, lockPath, children.get(0)).addEventListener(
-                                        new FutureEventListener<Pair<String, Long>>() {
-                                            @Override
-                                            public void onSuccess(Pair<String, Long> owner) {
-                                                if (!checkOrClaimLockOwner(owner, result)) {
-                                                    acquireFuture.updateIfEmpty(new Return<Boolean>(false));
-                                                }
-                                            }
-
-                                            @Override
-                                            public void onFailure(final Throwable cause) {
-                                                lockStateExecutor.submit(lockPath, new SafeRunnable() {
-                                                    @Override
-                                                    public void safeRun() {
-                                                        result.setException(cause);
-                                                    }
-                                                });
-                                            }
-                                        });
-                            } else {
-                                asyncTryLock(wait, result);
-                            }
-                        }
-                    });
-                }
-            }, 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>() {
-            @Override
-            public LockWaiter apply(final String currentOwner) {
-                final Exception acquireException = new OwnershipAcquireFailedException(lockPath, currentOwner);
-                FutureUtils.within(
-                        acquireFuture,
-                        timeout,
-                        unit,
-                        acquireException,
-                        lockStateExecutor,
-                        lockPath
-                ).addEventListener(new FutureEventListener<Boolean>() {
-
-                    @Override
-                    public void onSuccess(Boolean acquired) {
-                        completeOrFail(acquireException);
-                    }
-
-                    @Override
-                    public void onFailure(final Throwable acquireCause) {
-                        completeOrFail(acquireException);
-                    }
-
-                    private void completeOrFail(final Throwable acquireCause) {
-                        if (isLockHeld()) {
-                            waiterAcquireFuture.setValue(true);
-                        } else {
-                            asyncUnlock().addEventListener(new FutureEventListener<BoxedUnit>() {
-                                @Override
-                                public void onSuccess(BoxedUnit value) {
-                                    waiterAcquireFuture.setException(acquireCause);
-                                }
-
-                                @Override
-                                public void onFailure(Throwable cause) {
-                                    waiterAcquireFuture.setException(acquireCause);
-                                }
-                            });
-                        }
-                    }
-                });;
-                return new LockWaiter(
-                        lockId.getLeft(),
-                        currentOwner,
-                        waiterAcquireFuture);
-            }
-        });
-    }
-
-    private boolean checkOrClaimLockOwner(final Pair<String, Long> currentOwner,
-                                          final Promise<String> result) {
-        if (lockId.compareTo(currentOwner) != 0 && !lockContext.hasLockId(currentOwner)) {
-            lockStateExecutor.submit(lockPath, new SafeRunnable() {
-                @Override
-                public void safeRun() {
-                    result.setValue(currentOwner.getLeft());
-                }
-            });
-            return false;
-        }
-        // current owner is itself
-        final int curEpoch = epoch.incrementAndGet();
-        executeLockAction(curEpoch, new LockAction() {
-            @Override
-            public void execute() {
-                if (!lockState.inState(State.INIT)) {
-                    result.setException(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
-                    return;
-                }
-                asyncTryLock(false, result);
-            }
-            @Override
-            public String getActionName() {
-                return "claimOwnership(owner=" + currentOwner + ")";
-            }
-        }, result);
-        return true;
-    }
-
-    /**
-     * Try lock. If it failed, it would cleanup its attempt.
-     *
-     * @param wait
-     *          whether to wait for ownership.
-     * @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>() {
-            @Override
-            public void onSuccess(String currentOwner) {
-                result.setValue(currentOwner);
-            }
-
-            @Override
-            public void onFailure(final Throwable lockCause) {
-                // If tryLock failed due to state changed, we don't need to cleanup
-                if (lockCause instanceof LockStateChangedException) {
-                    LOG.info("skipping cleanup for {} at {} after encountering lock " +
-                            "state change exception : ", new Object[] { lockId, lockPath, lockCause });
-                    result.setException(lockCause);
-                    return;
-                }
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("{} is cleaning up its lock state for {} due to : ",
-                            new Object[] { lockId, lockPath, lockCause });
-                }
-
-                // If we encountered any exception we should cleanup
-                Future<BoxedUnit> unlockResult = asyncUnlock();
-                unlockResult.addEventListener(new FutureEventListener<BoxedUnit>() {
-                    @Override
-                    public void onSuccess(BoxedUnit value) {
-                        result.setException(lockCause);
-                    }
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        result.setException(lockCause);
-                    }
-                });
-            }
-        });
-        asyncTryLockWithoutCleanup(wait, lockResult);
-    }
-
-    /**
-     * 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
-     * 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
-     * transformations attached on promise will be executed in order.
-     *
-     * @param wait
-     *          whether to wait for ownership.
-     * @param promise
-     *          promise to satisfy with current lock owner.
-     */
-    private void asyncTryLockWithoutCleanup(final boolean wait, final Promise<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()));
-                    return;
-                }
-                lockState.transition(State.PREPARING);
-
-                final int curEpoch = epoch.incrementAndGet();
-                watcher = new LockWatcher(curEpoch);
-                // register watcher for session expires
-                zkClient.register(watcher);
-                // Encode both client id and session in the lock node
-                String myPath;
-                try {
-                    // member_<clientid>_s<owner_session>_
-                    myPath = getLockPathPrefixV3(lockPath, lockId.getLeft(), lockId.getRight());
-                } catch (UnsupportedEncodingException uee) {
-                    myPath = getLockPathPrefixV1(lockPath);
-                }
-                zk.create(myPath, serializeClientId(lockId.getLeft()), zkClient.getDefaultACL(), CreateMode.EPHEMERAL_SEQUENTIAL,
-                        new AsyncCallback.StringCallback() {
-                            @Override
-                            public void processResult(final int rc, String path, Object ctx, final String name) {
-                                executeLockAction(curEpoch, new LockAction() {
-                                    @Override
-                                    public void execute() {
-                                        if (KeeperException.Code.OK.intValue() != rc) {
-                                            KeeperException ke = KeeperException.create(KeeperException.Code.get(rc));
-                                            promise.setException(ke);
-                                            return;
-                                        }
-
-                                        if (FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockTryCloseRaceCondition)) {
-                                            lockState.transition(State.CLOSING);
-                                            lockState.transition(State.CLOSED);
-                                        }
-
-                                        if (null != currentNode) {
-                                            LOG.error("Current node for {} overwritten current = {} new = {}",
-                                                new Object[] { lockPath, lockId, getLockIdFromPath(currentNode) });
-                                        }
-
-                                        currentNode = name;
-                                        currentId = getLockIdFromPath(currentNode);
-                                        LOG.trace("{} received member id for lock {}", lockId, currentId);
-
-                                        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>();
-                                            deleteLockNode(deletePromise);
-                                            deletePromise.ensure(new Function0<BoxedUnit>() {
-                                                public BoxedUnit apply() {
-                                                    promise.setException(new LockClosedException(lockPath, lockId, lockState.getState()));
-                                                    return BoxedUnit.UNIT;
-                                                }
-                                            });
-                                            return;
-                                        }
-
-                                        lockState.transition(State.PREPARED);
-                                        checkLockOwnerAndWaitIfPossible(watcher, wait, promise);
-                                    }
-
-                                    @Override
-                                    public String getActionName() {
-                                        return "postPrepare(wait=" + wait + ")";
-                                    }
-                                });
-                            }
-                        }, null);
-            }
-            @Override
-            public String getActionName() {
-                return "prepare(wait=" + wait + ")";
-            }
-        }, promise);
-    }
-
-    @Override
-    public void tryLock(long timeout, TimeUnit unit) throws LockingException {
-        final Stopwatch stopwatch = Stopwatch.createStarted();
-        Future<LockWaiter> tryFuture = asyncTryLock(timeout, unit);
-        LockWaiter waiter = waitForTry(stopwatch, tryFuture);
-        boolean acquired = waiter.waitForAcquireQuietly();
-        if (!acquired) {
-            throw new OwnershipAcquireFailedException(lockPath, waiter.getCurrentOwner());
-        }
-    }
-
-    synchronized LockWaiter waitForTry(Stopwatch stopwatch, Future<LockWaiter> tryFuture)
-            throws LockingException {
-        boolean success = false;
-        boolean stateChanged = false;
-        LockWaiter waiter;
-        try {
-            waiter = Await.result(tryFuture, Duration.fromMilliseconds(lockOpTimeout));
-            success = true;
-        } catch (LockStateChangedException ex) {
-            stateChanged = true;
-            throw ex;
-        } catch (LockingException ex) {
-            throw ex;
-        } catch (TimeoutException toe) {
-            tryTimeouts.inc();
-            throw new LockingException(lockPath, "Timeout during try phase of lock acquire", toe);
-        } catch (Exception ex) {
-            String message = getLockId() + " failed to lock " + lockPath;
-            throw new LockingException(lockPath, message, ex);
-        } finally {
-            if (success) {
-                tryStats.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-            } else {
-                tryStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-            }
-            // This can only happen for a Throwable thats not an
-            // Exception, i.e. an Error
-            if (!success && !stateChanged) {
-                unlock();
-            }
-        }
-        return waiter;
-    }
-
-    @Override
-    public Future<BoxedUnit> asyncUnlock() {
-        return asyncUnlock(new LockClosedException(lockPath, lockId, lockState.getState()));
-    }
-
-    Future<BoxedUnit> asyncUnlock(final Throwable cause) {
-        final Promise<BoxedUnit> promise = new Promise<BoxedUnit>();
-
-        // 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
-        // risk of an ABA problem where we delete and recreate a node and then delete it again here.
-        lockStateExecutor.submit(lockPath, new SafeRunnable() {
-            @Override
-            public void safeRun() {
-                acquireFuture.updateIfEmpty(new Throw<Boolean>(cause));
-                unlockInternal(promise);
-                promise.addEventListener(new OpStatsListener<BoxedUnit>(unlockStats));
-            }
-        });
-
-        return promise;
-    }
-
-    @Override
-    public void unlock() {
-        Future<BoxedUnit> unlockResult = asyncUnlock();
-        try {
-            Await.result(unlockResult, Duration.fromMilliseconds(lockOpTimeout));
-        } 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 });
-        } catch (Exception e) {
-            LOG.warn("{} failed to unlock {} : ", new Object[] { lockId, lockPath, e });
-        }
-    }
-
-    // Lock State Changes (all state changes should be executed under a LockAction)
-
-    private void claimOwnership(int lockEpoch) {
-        lockState.transition(State.CLAIMED);
-        // clear previous lock ids
-        lockContext.clearLockIds();
-        // add current lock id
-        lockContext.addLockId(lockId);
-        if (LOG.isDebugEnabled()) {
-            LOG.debug("Notify lock waiters on {} at {} : watcher epoch {}, lock epoch {}",
-                    new Object[] { lockPath, System.currentTimeMillis(),
-                            lockEpoch, ZKSessionLock.this.epoch.get() });
-        }
-        acquireFuture.updateIfEmpty(new Return<Boolean>(true));
-    }
-
-    /**
-     * NOTE: unlockInternal should only after try lock.
-     */
-    private void unlockInternal(final Promise<BoxedUnit> promise) {
-
-        // already closed or expired, nothing to cleanup
-        this.epoch.incrementAndGet();
-        if (null != watcher) {
-            this.zkClient.unregister(watcher);
-        }
-
-        if (lockState.inState(State.CLOSED)) {
-            promise.setValue(BoxedUnit.UNIT);
-            return;
-        }
-
-        LOG.info("Lock {} for {} is closed from state {}.",
-                new Object[] { lockId, lockPath, lockState.getState() });
-
-        final boolean skipCleanup = lockState.inState(State.INIT) || lockState.inState(State.EXPIRED);
-
-        lockState.transition(State.CLOSING);
-
-        if (skipCleanup) {
-            // Nothing to cleanup if INIT (never tried) or EXPIRED (ephemeral node
-            // auto-removed)
-            lockState.transition(State.CLOSED);
-            promise.setValue(BoxedUnit.UNIT);
-            return;
-        }
-
-        // In any other state, we should clean up the member node
-        Promise<BoxedUnit> deletePromise = new Promise<BoxedUnit>();
-        deleteLockNode(deletePromise);
-
-        // Set the state to closed after we've cleaned up
-        deletePromise.addEventListener(new FutureEventListener<BoxedUnit>() {
-            @Override
-            public void onSuccess(BoxedUnit complete) {
-                lockStateExecutor.submit(lockPath, new SafeRunnable() {
-                    @Override
-                    public void safeRun() {
-                        lockState.transition(State.CLOSED);
-                        promise.setValue(BoxedUnit.UNIT);
-                    }
-                });
-            }
-            @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);
-            }
-        });
-    }
-
-    private void deleteLockNode(final Promise<BoxedUnit> promise) {
-        if (null == currentNode) {
-            promise.setValue(BoxedUnit.UNIT);
-            return;
-        }
-
-        zk.delete(currentNode, -1, new AsyncCallback.VoidCallback() {
-            @Override
-            public void processResult(final int rc, final String path, Object ctx) {
-                lockStateExecutor.submit(lockPath, new SafeRunnable() {
-                    @Override
-                    public void safeRun() {
-                        if (KeeperException.Code.OK.intValue() == rc) {
-                            LOG.info("Deleted lock node {} for {} successfully.", path, lockId);
-                        } else if (KeeperException.Code.NONODE.intValue() == rc ||
-                                KeeperException.Code.SESSIONEXPIRED.intValue() == rc) {
-                            LOG.info("Delete node failed. Node already gone for node {} id {}, rc = {}",
-                                    new Object[] { path, lockId, KeeperException.Code.get(rc) });
-                        } else {
-                            LOG.error("Failed on deleting lock node {} for {} : {}",
-                                    new Object[] { path, lockId, KeeperException.Code.get(rc) });
-                        }
-
-                        FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockUnlockCleanup);
-                        promise.setValue(BoxedUnit.UNIT);
-                    }
-                });
-            }
-        }, null);
-    }
-
-    /**
-     * Handle session expired for lock watcher at epoch <i>lockEpoch</i>.
-     *
-     * @param lockEpoch
-     *          lock epoch
-     */
-    private void handleSessionExpired(final int lockEpoch) {
-        executeLockAction(lockEpoch, new LockAction() {
-            @Override
-            public void execute() {
-                if (lockState.inState(State.CLOSED) || lockState.inState(State.CLOSING)) {
-                    // Already fully closed, no need to process expire.
-                    return;
-                }
-
-                boolean shouldNotifyLockListener = lockState.inState(State.CLAIMED);
-
-                lockState.transition(State.EXPIRED);
-
-                // remove the watcher
-                if (null != watcher) {
-                    zkClient.unregister(watcher);
-                }
-
-                // increment epoch to avoid any ongoing locking action
-                ZKSessionLock.this.epoch.incrementAndGet();
-
-                // 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())));
-
-                // session expired, ephemeral node is gone.
-                currentNode = null;
-                currentId = null;
-
-                if (shouldNotifyLockListener) {
-                    // if session expired after claimed, we need to notify the caller to re-lock
-                    if (null != lockListener) {
-                        lockListener.onExpired();
-                    }
-                }
-            }
-
-            @Override
-            public String getActionName() {
-                return "handleSessionExpired(epoch=" + lockEpoch + ")";
-            }
-        });
-    }
-
-    private void handleNodeDelete(int lockEpoch, final WatchedEvent event) {
-        executeLockAction(lockEpoch, new LockAction() {
-            @Override
-            public void execute() {
-                // The lock is either expired or closed
-                if (!lockState.inState(State.WAITING)) {
-                    LOG.info("{} ignore watched node {} deleted event, since lock state has moved to {}.",
-                            new Object[] { lockId, event.getPath(), lockState.getState() });
-                    return;
-                }
-                lockState.transition(State.PREPARED);
-
-                // we don't need to wait and check the result, since:
-                // 1) if it claimed the ownership, it would notify the waiters when claimed ownerships
-                // 2) if it failed, it would also notify the waiters, the waiters would cleanup the state.
-                checkLockOwnerAndWaitIfPossible(watcher, true);
-            }
-
-            @Override
-            public String getActionName() {
-                return "handleNodeDelete(path=" + event.getPath() + ")";
-            }
-        });
-    }
-
-    private Future<String> checkLockOwnerAndWaitIfPossible(final LockWatcher lockWatcher,
-                                                           final boolean wait) {
-        final Promise<String> promise = new Promise<String>();
-        checkLockOwnerAndWaitIfPossible(lockWatcher, wait, promise);
-        return promise;
-    }
-
-    /**
-     * Check Lock Owner Phase 1 : Get all lock waiters.
-     *
-     * @param lockWatcher
-     *          lock watcher.
-     * @param wait
-     *          whether to wait for ownership.
-     * @param promise
-     *          promise to satisfy with current lock owner
-     */
-    private void checkLockOwnerAndWaitIfPossible(final LockWatcher lockWatcher,
-                                                 final boolean wait,
-                                                 final Promise<String> promise) {
-        zk.getChildren(lockPath, false, new AsyncCallback.Children2Callback() {
-            @Override
-            public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
-                processLockWaiters(lockWatcher, wait, rc, children, promise);
-            }
-        }, null);
-    }
-
-    /**
-     * Check Lock Owner Phase 2 : check all lock waiters to get current owner and wait for ownership if necessary.
-     *
-     * @param lockWatcher
-     *          lock watcher.
-     * @param wait
-     *          whether to wait for ownership.
-     * @param getChildrenRc
-     *          result of getting all lock waiters
-     * @param children
-     *          current lock waiters.
-     * @param promise
-     *          promise to satisfy with current lock owner.
-     */
-    private void processLockWaiters(final LockWatcher lockWatcher,
-                                    final boolean wait,
-                                    final int getChildrenRc,
-                                    final List<String> children,
-                                    final Promise<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()));
-                    return;
-                }
-
-                if (KeeperException.Code.OK.intValue() != getChildrenRc) {
-                    promise.setException(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));
-                    return;
-                }
-
-                // sort the children
-                Collections.sort(children, MEMBER_COMPARATOR);
-                final String cid = currentId;
-                final int memberIndex = children.indexOf(cid);
-                if (LOG.isDebugEnabled()) {
-                    LOG.debug("{} is the number {} member in the list.", cid, memberIndex);
-                }
-                // If we hold the lock
-                if (memberIndex == 0) {
-                    LOG.info("{} acquired the lock {}.", cid, lockPath);
-                    claimOwnership(lockWatcher.epoch);
-                    promise.setValue(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>>() {
-                        @Override
-                        public void onSuccess(Pair<String, Long> currentOwner) {
-                            watchLockOwner(lockWatcher, wait,
-                                    cid, children.get(memberIndex - 1), children.get(0), currentOwner, promise);
-                        }
-                        @Override
-                        public void onFailure(final Throwable cause) {
-                            // ensure promise is satisfied in lock thread
-                            executeLockAction(lockWatcher.epoch, new LockAction() {
-                                @Override
-                                public void execute() {
-                                    promise.setException(cause);
-                                }
-
-                                @Override
-                                public String getActionName() {
-                                    return "handleFailureOnParseClientID(lockPath=" + lockPath + ")";
-                                }
-                            }, promise);
-                        }
-                    });
-                } else {
-                    LOG.error("Member {} doesn't exist in the members list {} for lock {}.",
-                            new Object[]{ cid, children, lockPath});
-                    promise.setException(
-                            new UnexpectedException("Member " + cid + " doesn't exist in member list " +
-                                    children + " for lock " + lockPath));
-                }
-            }
-
-            @Override
-            public String getActionName() {
-                return "processLockWaiters(rc=" + getChildrenRc + ", waiters=" + children + ")";
-            }
-        }, promise);
-    }
-
-    /**
-     * Check Lock Owner Phase 3: watch sibling node for lock ownership.
-     *
-     * @param lockWatcher
-     *          lock watcher.
-     * @param wait
-     *          whether to wait for ownership.
-     * @param myNode
-     *          my lock node.
-     * @param siblingNode
-     *          my sibling lock node.
-     * @param ownerNode
-     *          owner lock node.
-     * @param currentOwner
-     *          current owner info.
-     * @param promise
-     *          promise to satisfy with current lock owner.
-     */
-    private void watchLockOwner(final LockWatcher lockWatcher,
-                                final boolean wait,
-                                final String myNode,
-                                final String siblingNode,
-                                final String ownerNode,
-                                final Pair<String, Long> currentOwner,
-                                final Promise<String> promise) {
-        executeLockAction(lockWatcher.epoch, new LockAction() {
-            @Override
-            public void execute() {
-                boolean shouldWatch;
-                final boolean shouldClaimOwnership;
-                if (lockContext.hasLockId(currentOwner) && siblingNode.equals(ownerNode)) {
-                    // if the current owner is the znode left from previous session
-                    // we should watch it and claim ownership
-                    shouldWatch = true;
-                    shouldClaimOwnership = true;
-                    LOG.info("LockWatcher {} for {} found its previous session {} held lock, watch it to claim ownership.",
-                            new Object[] { myNode, lockPath, currentOwner });
-                } else if (lockId.compareTo(currentOwner) == 0 && areLockWaitersInSameSession(siblingNode, ownerNode)) {
-                    // I found that my sibling is the current owner with same lock id (client id & session id)
-                    // It must be left by any race condition from same zookeeper client
-                    shouldWatch = true;
-                    shouldClaimOwnership = true;
-                    LOG.info("LockWatcher {} for {} found itself {} already held lock at sibling node {}, watch it to claim ownership.",
-                            new Object[]{myNode, lockPath, lockId, siblingNode});
-                } else {
-                    shouldWatch = wait;
-                    if (wait) {
-                        if (LOG.isDebugEnabled()) {
-                            LOG.debug("Current LockWatcher for {} with ephemeral node {}, is waiting for {} to release lock at {}.",
-                                    new Object[]{lockPath, myNode, siblingNode, System.currentTimeMillis()});
-                        }
-                    }
-                    shouldClaimOwnership = false;
-                }
-
-                // watch sibling for lock ownership
-                if (shouldWatch) {
-                    watchedNode = String.format("%s/%s", lockPath, siblingNode);
-                    zk.exists(watchedNode, lockWatcher, new AsyncCallback.StatCallback() {
-                        @Override
-                        public void processResult(final int rc, String path, Object ctx, final Stat stat) {
-                            executeLockAction(lockWatcher.epoch, new LockAction() {
-                                @Override
-                                public void execute() {
-                                    if (!lockState.inState(State.PREPARED)) {
-                                        promise.setException(new LockStateChangedException(lockPath, lockId, State.PREPARED, lockState.getState()));
-                                        return;
-                                    }
-
-                                    if (KeeperException.Code.OK.intValue() == rc) {
-                                        if (shouldClaimOwnership) {
-                                            // watch owner successfully
-                                            LOG.info("LockWatcher {} claimed ownership for {} after set watcher on {}.",
-                                                    new Object[]{ myNode, lockPath, ownerNode });
-                                            claimOwnership(lockWatcher.epoch);
-                                            promise.setValue(currentOwner.getLeft());
-                                        } else {
-                                            // watch sibling successfully
-                                            lockState.transition(State.WAITING);
-                                            promise.setValue(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)));
-                                    }
-                                }
-
-                                @Override
-                                public String getActionName() {
-                                    StringBuilder sb = new StringBuilder();
-                                    sb.append("postWatchLockOwner(myNode=").append(myNode).append(", siblingNode=")
-                                            .append(siblingNode).append(", ownerNode=").append(ownerNode).append(")");
-                                    return sb.toString();
-                                }
-                            }, promise);
-                        }
-                    }, null);
-                } else {
-                    promise.setValue(currentOwner.getLeft());
-                }
-            }
-
-            @Override
-            public String getActionName() {
-                StringBuilder sb = new StringBuilder();
-                sb.append("watchLockOwner(myNode=").append(myNode).append(", siblingNode=")
-                        .append(siblingNode).append(", ownerNode=").append(ownerNode).append(")");
-                return sb.toString();
-            }
-        }, promise);
-    }
-
-    class LockWatcher implements Watcher {
-
-        // Enforce a epoch number to avoid a race on canceling attempt
-        final int epoch;
-
-        LockWatcher(int epoch) {
-            this.epoch = epoch;
-        }
-
-        @Override
-        public void process(WatchedEvent event) {
-            LOG.debug("Received event {} from lock {} at {} : watcher epoch {}, lock epoch {}.",
-                    new Object[] { event, lockPath, System.currentTimeMillis(), epoch, ZKSessionLock.this.epoch.get() });
-            if (event.getType() == Watcher.Event.EventType.None) {
-                switch (event.getState()) {
-                    case SyncConnected:
-                        break;
-                    case Expired:
-                        LOG.info("Session {} is expired for lock {} at {} : watcher epoch {}, lock epoch {}.",
-                                new Object[] { lockId.getRight(), lockPath, System.currentTimeMillis(),
-                                        epoch, ZKSessionLock.this.epoch.get() });
-                        handleSessionExpired(epoch);
-                        break;
-                    default:
-                        break;
-                }
-            } else if (event.getType() == Event.EventType.NodeDeleted) {
-                // this handles the case where we have aborted a lock and deleted ourselves but still have a
-                // watch on the nextLowestNode. This is a workaround since ZK doesn't support unsub.
-                if (!event.getPath().equals(watchedNode)) {
-                    LOG.warn("{} (watching {}) ignored watched event from {} ",
-                            new Object[] { lockId, watchedNode, event.getPath() });
-                    return;
-                }
-                handleNodeDelete(epoch, event);
-            } else {
-                LOG.warn("Unexpected ZK event: {}", event.getType().name());
-            }
-        }
-
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLockFactory.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLockFactory.java
deleted file mode 100644
index bb98e07..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLockFactory.java
+++ /dev/null
@@ -1,133 +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 com.twitter.distributedlog.lock;
-
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.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.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * Factory to create zookeeper based locks.
- */
-public class ZKSessionLockFactory implements SessionLockFactory {
-
-    private final ZooKeeperClient zkc;
-    private final String clientId;
-    private final OrderedScheduler lockStateExecutor;
-    private final long lockOpTimeout;
-    private final int lockCreationRetries;
-    private final long zkRetryBackoffMs;
-
-    // Stats
-    private final StatsLogger lockStatsLogger;
-
-    public ZKSessionLockFactory(ZooKeeperClient zkc,
-                                String clientId,
-                                OrderedScheduler lockStateExecutor,
-                                int lockCreationRetries,
-                                long lockOpTimeout,
-                                long zkRetryBackoffMs,
-                                StatsLogger statsLogger) {
-        this.zkc = zkc;
-        this.clientId = clientId;
-        this.lockStateExecutor = lockStateExecutor;
-        this.lockCreationRetries = lockCreationRetries;
-        this.lockOpTimeout = lockOpTimeout;
-        this.zkRetryBackoffMs = zkRetryBackoffMs;
-
-        this.lockStatsLogger = statsLogger.scope("lock");
-    }
-
-    @Override
-    public Future<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;
-            }
-        });
-        createLock(
-                lockPath,
-                context,
-                interruptedException,
-                numRetries,
-                createPromise,
-                0L);
-        return createPromise;
-    }
-
-    void createLock(final String lockPath,
-                    final DistributedLockContext context,
-                    final AtomicReference<Throwable> interruptedException,
-                    final AtomicInteger numRetries,
-                    final Promise<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()));
-                    return;
-                }
-                try {
-                    SessionLock lock = new ZKSessionLock(
-                            zkc,
-                            lockPath,
-                            clientId,
-                            lockStateExecutor,
-                            lockOpTimeout,
-                            lockStatsLogger,
-                            context);
-                    createPromise.updateIfEmpty(new Return<SessionLock>(lock));
-                } catch (DLInterruptedException dlie) {
-                    // if the creation is interrupted, throw the exception without retrie.
-                    createPromise.updateIfEmpty(new Throw<SessionLock>(dlie));
-                    return;
-                } catch (IOException e) {
-                    if (numRetries.getAndDecrement() < 0) {
-                        createPromise.updateIfEmpty(new Throw<SessionLock>(e));
-                        return;
-                    }
-                    createLock(
-                            lockPath,
-                            context,
-                            interruptedException,
-                            numRetries,
-                            createPromise,
-                            zkRetryBackoffMs);
-                }
-            }
-        }, delayMs, TimeUnit.MILLISECONDS);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/package-info.java
deleted file mode 100644
index 02d905d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Distributed locking mechanism in distributedlog
- */
-package com.twitter.distributedlog.lock;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryReader.java
deleted file mode 100644
index 81eb5ed..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryReader.java
+++ /dev/null
@@ -1,113 +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 com.twitter.distributedlog.logsegment;
-
-import com.google.common.annotations.Beta;
-import com.twitter.distributedlog.Entry;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.util.List;
-
-/**
- * An interface class to read the enveloped entry (serialized bytes of
- * {@link com.twitter.distributedlog.Entry}) from a log segment
- */
-@Beta
-public interface LogSegmentEntryReader extends AsyncCloseable {
-
-    interface StateChangeListener {
-
-        /**
-         * Notify when caught up on inprogress.
-         */
-        void onCaughtupOnInprogress();
-
-    }
-
-    /**
-     * Start the reader. The method to signal the implementation
-     * to start preparing the data for consumption {@link #readNext(int)}
-     */
-    void start();
-
-    /**
-     * Register the state change listener
-     *
-     * @param listener register the state change listener
-     * @return entry reader
-     */
-    LogSegmentEntryReader registerListener(StateChangeListener listener);
-
-    /**
-     * Unregister the state change listener
-     *
-     * @param listener register the state change listener
-     * @return entry reader
-     */
-    LogSegmentEntryReader unregisterListener(StateChangeListener listener);
-
-    /**
-     * Return the log segment metadata for this reader.
-     *
-     * @return the log segment metadata
-     */
-    LogSegmentMetadata getSegment();
-
-    /**
-     * Update the log segment each time when the metadata has changed.
-     *
-     * @param segment new metadata of the log segment.
-     */
-    void onLogSegmentMetadataUpdated(LogSegmentMetadata segment);
-
-    /**
-     * Read next <i>numEntries</i> entries from current log segment.
-     * <p>
-     * <i>numEntries</i> will be best-effort.
-     *
-     * @param numEntries num entries to read from current log segment
-     * @return A promise that when satisified will contain a non-empty list of entries with their content.
-     * @throws {@link com.twitter.distributedlog.exceptions.EndOfLogSegmentException} when
-     *          read entries beyond the end of a <i>closed</i> log segment.
-     */
-    Future<List<Entry.Reader>> readNext(int numEntries);
-
-    /**
-     * Return the last add confirmed entry id (LAC).
-     *
-     * @return the last add confirmed entry id.
-     */
-    long getLastAddConfirmed();
-
-    /**
-     * Is the reader reading beyond last add confirmed.
-     *
-     * @return true if the reader is reading beyond last add confirmed
-     */
-    boolean isBeyondLastAddConfirmed();
-
-    /**
-     * Has the log segment reader caught up with the inprogress log segment.
-     *
-     * @return true only if the log segment is inprogress and it is caught up, otherwise return false.
-     */
-    boolean hasCaughtUpOnInprogress();
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryStore.java
deleted file mode 100644
index bcf8129..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryStore.java
+++ /dev/null
@@ -1,73 +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 com.twitter.distributedlog.logsegment;
-
-import com.google.common.annotations.Beta;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.util.Allocator;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.util.Future;
-
-import java.io.IOException;
-
-/**
- * Log Segment Store to read log segments
- */
-@Beta
-public interface LogSegmentEntryStore {
-
-    /**
-     * Delete the actual log segment from the entry store.
-     *
-     * @param segment log segment metadata
-     * @return future represent the delete result
-     */
-    Future<LogSegmentMetadata> deleteLogSegment(LogSegmentMetadata segment);
-
-    /**
-     * Create a new log segment allocator for allocating log segment entry writers.
-     *
-     * @param metadata the metadata for the log stream
-     * @return future represent the log segment allocator
-     */
-    Allocator<LogSegmentEntryWriter, Object> newLogSegmentAllocator(
-            LogMetadataForWriter metadata,
-            DynamicDistributedLogConfiguration dynConf) throws IOException;
-
-    /**
-     * Open the reader for reading data to the log <i>segment</i>.
-     *
-     * @param segment the log <i>segment</i> to read data from
-     * @param startEntryId the start entry id
-     * @return future represent the opened reader
-     */
-    Future<LogSegmentEntryReader> openReader(LogSegmentMetadata segment,
-                                             long startEntryId);
-
-    /**
-     * Open the reader for reading entries from a random access log <i>segment</i>.
-     *
-     * @param segment the log <i>segment</i> to read entries from
-     * @param fence the flag to fence log segment
-     * @return future represent the opened random access reader
-     */
-    Future<LogSegmentRandomAccessEntryReader> openRandomAccessReader(LogSegmentMetadata segment,
-                                                                     boolean fence);
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryWriter.java
deleted file mode 100644
index 8b7d9b2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryWriter.java
+++ /dev/null
@@ -1,72 +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 com.twitter.distributedlog.logsegment;
-
-import com.google.common.annotations.Beta;
-import com.twitter.distributedlog.Entry;
-import com.twitter.distributedlog.util.Sizable;
-import org.apache.bookkeeper.client.AsyncCallback;
-
-/**
- * An interface class to write the enveloped entry (serialized bytes of
- * {@link Entry} into the log segment.
- *
- * <p>It is typically used by {@link LogSegmentWriter}.
- *
- * @see LogSegmentWriter
- *
- * TODO: The interface is leveraging bookkeeper's callback and status code now
- *       Consider making it more generic.
- */
-@Beta
-public interface LogSegmentEntryWriter extends Sizable {
-
-    /**
-     * Get the log segment id.
-     *
-     * @return log segment id.
-     */
-    long getLogSegmentId();
-
-    /**
-     * Close the entry writer.
-     */
-    void asyncClose(AsyncCallback.CloseCallback callback, Object ctx);
-
-    /**
-     * Async add entry to the log segment.
-     * <p>The implementation semantic follows
-     * {@link org.apache.bookkeeper.client.LedgerHandle#asyncAddEntry(
-     * byte[], int, int, AsyncCallback.AddCallback, Object)}
-     *
-     * @param data
-     *          data to add
-     * @param offset
-     *          offset in the data
-     * @param length
-     *          length of the data
-     * @param callback
-     *          callback
-     * @param ctx
-     *          ctx
-     * @see org.apache.bookkeeper.client.LedgerHandle#asyncAddEntry(
-     * byte[], int, int, AsyncCallback.AddCallback, Object)
-     */
-    void asyncAddEntry(byte[] data, int offset, int length,
-                       AsyncCallback.AddCallback callback, Object ctx);
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentFilter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentFilter.java
deleted file mode 100644
index f8bf183..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentFilter.java
+++ /dev/null
@@ -1,42 +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 com.twitter.distributedlog.logsegment;
-
-import java.util.Collection;
-
-/**
- * Filter to filter log segments
- */
-public interface LogSegmentFilter {
-
-    public static final LogSegmentFilter DEFAULT_FILTER = new LogSegmentFilter() {
-        @Override
-        public Collection<String> filter(Collection<String> fullList) {
-            return fullList;
-        }
-    };
-
-    /**
-     * Filter the log segments from the full log segment list.
-     *
-     * @param fullList
-     *          full list of log segment names.
-     * @return filtered log segment names
-     */
-    Collection<String> filter(Collection<String> fullList);
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataCache.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataCache.java
deleted file mode 100644
index d4ca3ea..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataCache.java
+++ /dev/null
@@ -1,98 +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 com.twitter.distributedlog.logsegment;
-
-import com.google.common.base.Ticker;
-import com.google.common.cache.Cache;
-import com.google.common.cache.CacheBuilder;
-import com.google.common.cache.RemovalListener;
-import com.google.common.cache.RemovalNotification;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.TimeUnit;
-
-/**
- * Cache the log segment metadata
- */
-public class LogSegmentMetadataCache implements RemovalListener<String, LogSegmentMetadata> {
-
-    private static final Logger logger = LoggerFactory.getLogger(LogSegmentMetadataCache.class);
-
-    private final Cache<String, LogSegmentMetadata> cache;
-    private final boolean isCacheEnabled;
-
-    public LogSegmentMetadataCache(DistributedLogConfiguration conf,
-                                   Ticker ticker) {
-        cache = CacheBuilder.newBuilder()
-                .concurrencyLevel(conf.getNumWorkerThreads())
-                .initialCapacity(1024)
-                .expireAfterAccess(conf.getLogSegmentCacheTTLMs(), TimeUnit.MILLISECONDS)
-                .maximumSize(conf.getLogSegmentCacheMaxSize())
-                .removalListener(this)
-                .ticker(ticker)
-                .recordStats()
-                .build();
-        this.isCacheEnabled = conf.isLogSegmentCacheEnabled();
-        logger.info("Log segment cache is enabled = {}", this.isCacheEnabled);
-    }
-
-    /**
-     * Add the log <i>segment</i> of <i>path</i> to the cache.
-     *
-     * @param path the path of the log segment
-     * @param segment log segment metadata
-     */
-    public void put(String path, LogSegmentMetadata segment) {
-        if (isCacheEnabled) {
-            cache.put(path, segment);
-        }
-    }
-
-    /**
-     * Invalid the cache entry associated with <i>path</i>.
-     *
-     * @param path the path of the log segment
-     */
-    public void invalidate(String path) {
-        if (isCacheEnabled) {
-            cache.invalidate(path);
-        }
-    }
-
-    /**
-     * Retrieve the log segment of <i>path</i> from the cache.
-     *
-     * @param path the path of the log segment.
-     * @return log segment metadata if exists, otherwise null.
-     */
-    public LogSegmentMetadata get(String path) {
-        return cache.getIfPresent(path);
-    }
-
-    @Override
-    public void onRemoval(RemovalNotification<String, LogSegmentMetadata> notification) {
-        if (notification.wasEvicted()) {
-            if (logger.isDebugEnabled()) {
-                logger.debug("Log segment of {} was evicted.", notification.getKey());
-            }
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataStore.java
deleted file mode 100644
index dda76e5..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataStore.java
+++ /dev/null
@@ -1,163 +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 com.twitter.distributedlog.logsegment;
-
-import com.google.common.annotations.Beta;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.callback.LogSegmentNamesListener;
-import com.twitter.distributedlog.metadata.LogMetadata;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.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;
-
-/**
- * Interface for log segment metadata store. All operations that modify log segments should
- * be executed under a {@link Transaction}.
- */
-@Beta
-public interface LogSegmentMetadataStore extends Closeable {
-
-    /**
-     * Start the transaction on changing log segment metadata store.
-     *
-     * @return transaction of the log segment metadata store.
-     */
-    Transaction<Object> transaction();
-
-    // The reason to keep storing log segment sequence number & log record transaction id
-    // in this log segment metadata store interface is to share the transaction that used
-    // to start/complete log segment. It is a bit hard to separate them out right now.
-
-    /**
-     * Store the maximum log segment sequence number on <code>path</code>.
-     *
-     * @param txn
-     *          transaction to execute for storing log segment sequence number.
-     * @param logMetadata
-     *          metadata of the log stream
-     * @param sequenceNumber
-     *          log segment sequence number to store
-     * @param listener
-     *          listener on the result to this operation
-     */
-    void storeMaxLogSegmentSequenceNumber(Transaction<Object> txn,
-                                          LogMetadata logMetadata,
-                                          Versioned<Long> sequenceNumber,
-                                          OpListener<Version> listener);
-
-    /**
-     * Store the maximum transaction id for <code>path</code>
-     *
-     * @param txn
-     *          transaction to execute for storing transaction id
-     * @param logMetadata
-     *          metadata of the log stream
-     * @param transactionId
-     *          transaction id to store
-     * @param listener
-     *          listener on the result to this operation
-     */
-    void storeMaxTxnId(Transaction<Object> txn,
-                       LogMetadataForWriter logMetadata,
-                       Versioned<Long> transactionId,
-                       OpListener<Version> listener);
-
-    /**
-     * Create a log segment <code>segment</code> under transaction <code>txn</code>.
-     *
-     * NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
-     *       immediately. the operation should be executed via {@link Transaction#execute()}
-     *
-     * @param txn
-     *          transaction to execute for this operation
-     * @param segment
-     *          segment to create
-     * @param opListener
-     *          the listener on the operation result
-     */
-    void createLogSegment(Transaction<Object> txn,
-                          LogSegmentMetadata segment,
-                          OpListener<Void> opListener);
-
-    /**
-     * Delete a log segment <code>segment</code> under transaction <code>txn</code>.
-     *
-     * NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
-     *       immediately. the operation should be executed via {@link Transaction#execute()}
-     *
-     * @param txn
-     *          transaction to execute for this operation
-     * @param segment
-     *          segment to delete
-     */
-    void deleteLogSegment(Transaction<Object> txn,
-                          LogSegmentMetadata segment,
-                          OpListener<Void> opListener);
-
-    /**
-     * Update a log segment <code>segment</code> under transaction <code>txn</code>.
-     *
-     * NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
-     *       immediately. the operation should be executed via {@link Transaction#execute()}
-     *
-     * @param txn
-     *          transaction to execute for this operation
-     * @param segment
-     *          segment to update
-     */
-    void updateLogSegment(Transaction<Object> txn, LogSegmentMetadata segment);
-
-    /**
-     * Retrieve the log segment associated <code>path</code>.
-     *
-     * @param logSegmentPath
-     *          path to store log segment metadata
-     * @return future of the retrieved log segment metadata
-     */
-    Future<LogSegmentMetadata> getLogSegment(String logSegmentPath);
-
-    /**
-     * Retrieve the list of log segments under <code>logSegmentsPath</code> and register a <i>listener</i>
-     * for subsequent changes for the list of log segments.
-     *
-     * @param logSegmentsPath
-     *          path to store list of log segments
-     * @param listener
-     *          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);
-
-    /**
-     * Unregister a log segment <code>listener</code> on log segment changes under <code>logSegmentsPath</code>.
-     *
-     * @param logSegmentsPath
-     *          log segments path
-     * @param listener
-     *          log segment listener on log segment changes
-     */
-    void unregisterLogSegmentListener(String logSegmentsPath,
-                                      LogSegmentNamesListener listener);
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java
deleted file mode 100644
index 70472ca..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java
+++ /dev/null
@@ -1,47 +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 com.twitter.distributedlog.logsegment;
-
-import com.twitter.distributedlog.Entry;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.util.List;
-
-/**
- * An interface class to read entries {@link com.twitter.distributedlog.Entry}
- * from a random access log segment.
- */
-public interface LogSegmentRandomAccessEntryReader extends AsyncCloseable {
-
-    /**
-     * Read entries [startEntryId, endEntryId] from a random access log segment.
-     *
-     * @param startEntryId start entry id
-     * @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);
-
-    /**
-     * Return the last add confirmed entry id (LAC).
-     *
-     * @return the last add confirmed entry id.
-     */
-    long getLastAddConfirmed();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentWriter.java
deleted file mode 100644
index a0b4610..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentWriter.java
+++ /dev/null
@@ -1,87 +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 com.twitter.distributedlog.logsegment;
-
-import com.google.common.annotations.Beta;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.io.AsyncAbortable;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.util.Future;
-
-import java.io.IOException;
-
-/**
- * An interface class to write log records into a log segment.
- */
-@Beta
-public interface LogSegmentWriter extends AsyncCloseable, AsyncAbortable {
-
-    /**
-     * Get the unique log segment id.
-     *
-     * @return log segment id.
-     */
-    public long getLogSegmentId();
-
-    /**
-     * Write a log record to a log segment.
-     *
-     * @param record single log record
-     * @return a future representing write result. A {@link DLSN} is returned if write succeeds,
-     *         otherwise, exceptions are returned.
-     * @throws com.twitter.distributedlog.exceptions.LogRecordTooLongException if log record is too long
-     * @throws com.twitter.distributedlog.exceptions.InvalidEnvelopedEntryException on invalid enveloped entry
-     * @throws LockingException if failed to acquire lock for the writer
-     * @throws BKTransmitException if failed to transmit data to bk
-     * @throws com.twitter.distributedlog.exceptions.WriteException if failed to write to bk
-     */
-    public Future<DLSN> asyncWrite(LogRecord record);
-
-    /**
-     * This isn't a simple synchronous version of {@code asyncWrite}. It has different semantic.
-     * This method only writes data to the buffer and flushes buffer if needed.
-     *
-     * TODO: we should remove this method. when we rewrite synchronous writer based on asynchronous writer,
-     *       since this is the semantic needed to be provided in higher level but just calling write & flush.
-     *
-     * @param record single log record
-     * @throws IOException when tried to flush the buffer.
-     * @see LogSegmentWriter#asyncWrite(LogRecord)
-     */
-    public void write(LogRecord record) throws IOException;
-
-    /**
-     * Transmit the buffered data and wait for it being persisted and return the last acknowledged
-     * transaction id.
-     *
-     * @return future representing the transmit result with last acknowledged transaction id.
-     */
-    public Future<Long> flush();
-
-    /**
-     * Commit the current acknowledged data. It is the consequent operation of {@link #flush()},
-     * which makes all the acknowledged data visible to
-     *
-     * @return future representing the commit result.
-     */
-    public Future<Long> commit();
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/PerStreamLogSegmentCache.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/PerStreamLogSegmentCache.java
deleted file mode 100644
index 5f88c5a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/PerStreamLogSegmentCache.java
+++ /dev/null
@@ -1,243 +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 com.twitter.distributedlog.logsegment;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Managing log segments in local cache.
- *
- * <p>
- * Caching of log segment metadata assumes that the data contained in the ZNodes for individual
- * log segments is never updated after creation i.e we never call setData. A log segment
- * is finalized by creating a new ZNode and deleting the in progress node. This code will have
- * to change if we change the behavior
- * </p>
- */
-public class PerStreamLogSegmentCache {
-
-    static final Logger LOG = LoggerFactory.getLogger(PerStreamLogSegmentCache.class);
-
-    protected final String streamName;
-    protected final boolean validateLogSegmentSequenceNumber;
-    protected final Map<String, LogSegmentMetadata> logSegments =
-            new HashMap<String, LogSegmentMetadata>();
-    protected final ConcurrentMap<Long, LogSegmentMetadata> lid2LogSegments =
-            new ConcurrentHashMap<Long, LogSegmentMetadata>();
-
-    @VisibleForTesting
-    PerStreamLogSegmentCache(String streamName) {
-        this(streamName, true);
-    }
-
-    public PerStreamLogSegmentCache(String streamName,
-                                    boolean validateLogSegmentSequenceNumber) {
-        this.streamName = streamName;
-        this.validateLogSegmentSequenceNumber = validateLogSegmentSequenceNumber;
-    }
-
-    /**
-     * Retrieve log segments from the cache.
-     *
-     * - first sort the log segments in ascending order
-     * - do validation and assign corresponding sequence id
-     * - apply comparator after validation
-     *
-     * @param comparator
-     *          comparator to sort the returned log segments.
-     * @return list of sorted and filtered log segments.
-     * @throws UnexpectedException if unexpected condition detected (e.g. ledger sequence number gap)
-     */
-    public List<LogSegmentMetadata> getLogSegments(Comparator<LogSegmentMetadata> comparator)
-        throws UnexpectedException {
-        List<LogSegmentMetadata> segmentsToReturn;
-        synchronized (logSegments) {
-            segmentsToReturn = new ArrayList<LogSegmentMetadata>(logSegments.size());
-            segmentsToReturn.addAll(logSegments.values());
-        }
-        Collections.sort(segmentsToReturn, LogSegmentMetadata.COMPARATOR);
-
-        LogSegmentMetadata prevSegment = null;
-        if (validateLogSegmentSequenceNumber) {
-            // validation ledger sequence number to ensure the log segments are unique.
-            for (int i = 0; i < segmentsToReturn.size(); i++) {
-                LogSegmentMetadata segment = segmentsToReturn.get(i);
-
-                if (null != prevSegment
-                        && prevSegment.getVersion() >= LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value
-                        && segment.getVersion() >= LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value
-                        && prevSegment.getLogSegmentSequenceNumber() + 1 != segment.getLogSegmentSequenceNumber()) {
-                    LOG.error("{} found ledger sequence number gap between log segment {} and {}",
-                            new Object[] { streamName, prevSegment, segment });
-                    throw new UnexpectedException(streamName + " found ledger sequence number gap between log segment "
-                            + prevSegment.getLogSegmentSequenceNumber() + " and " + segment.getLogSegmentSequenceNumber());
-                }
-                prevSegment = segment;
-            }
-        }
-
-        prevSegment = null;
-        long startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
-        for (int i = 0; i < segmentsToReturn.size(); i++) {
-                LogSegmentMetadata segment = segmentsToReturn.get(i);
-            // assign sequence id
-            if (!segment.isInProgress()) {
-                if (segment.supportsSequenceId()) {
-                    startSequenceId = segment.getStartSequenceId() + segment.getRecordCount();
-                    if (null != prevSegment && prevSegment.supportsSequenceId()
-                            && prevSegment.getStartSequenceId() > segment.getStartSequenceId()) {
-                        LOG.warn("{} found decreasing start sequence id in log segment {}, previous is {}",
-                                new Object[] { streamName, segment, prevSegment });
-                    }
-                } else {
-                    startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
-                }
-            } else {
-                if (segment.supportsSequenceId()) {
-                    LogSegmentMetadata newSegment = segment.mutator()
-                            .setStartSequenceId(startSequenceId == DistributedLogConstants.UNASSIGNED_SEQUENCE_ID ? 0L : startSequenceId)
-                            .build();
-                    segmentsToReturn.set(i, newSegment);
-                }
-
-                break;
-            }
-            prevSegment = segment;
-        }
-        if (comparator != LogSegmentMetadata.COMPARATOR) {
-            Collections.sort(segmentsToReturn, comparator);
-        }
-        return segmentsToReturn;
-    }
-
-    /**
-     * Add the segment <i>metadata</i> for <i>name</i> in the cache.
-     *
-     * @param name
-     *          segment name.
-     * @param metadata
-     *          segment metadata.
-     */
-    public void add(String name, LogSegmentMetadata metadata) {
-        synchronized (logSegments) {
-            if (!logSegments.containsKey(name)) {
-                logSegments.put(name, metadata);
-                LOG.info("{} added log segment ({} : {}) to cache.",
-                        new Object[]{ streamName, name, metadata });
-            }
-            LogSegmentMetadata oldMetadata = lid2LogSegments.remove(metadata.getLogSegmentId());
-            if (null == oldMetadata) {
-                lid2LogSegments.put(metadata.getLogSegmentId(), metadata);
-            } else {
-                if (oldMetadata.isInProgress() && !metadata.isInProgress()) {
-                    lid2LogSegments.put(metadata.getLogSegmentId(), metadata);
-                } else {
-                    lid2LogSegments.put(oldMetadata.getLogSegmentId(), oldMetadata);
-                }
-            }
-        }
-    }
-
-    /**
-     * Retrieve log segment <code>name</code> from the cache.
-     *
-     * @param name
-     *          name of the log segment.
-     * @return log segment metadata
-     */
-    public LogSegmentMetadata get(String name) {
-        synchronized (logSegments) {
-            return logSegments.get(name);
-        }
-    }
-
-    /**
-     * Update the log segment cache with removed/added segments.
-     *
-     * @param segmentsRemoved
-     *          segments that removed
-     * @param segmentsAdded
-     *          segments that added
-     */
-    public void update(Set<String> segmentsRemoved,
-                       Map<String, LogSegmentMetadata> segmentsAdded) {
-        synchronized (logSegments) {
-            for (Map.Entry<String, LogSegmentMetadata> entry : segmentsAdded.entrySet()) {
-                add(entry.getKey(), entry.getValue());
-            }
-            for (String segment : segmentsRemoved) {
-                remove(segment);
-            }
-        }
-    }
-
-    /**
-     * Diff with new received segment list <code>segmentReceived</code>.
-     *
-     * @param segmentsReceived
-     *          new received segment list
-     * @return segments added (left) and removed (right).
-     */
-    public Pair<Set<String>, Set<String>> diff(Set<String> segmentsReceived) {
-        Set<String> segmentsAdded;
-        Set<String> segmentsRemoved;
-        synchronized (logSegments) {
-            Set<String> segmentsCached = logSegments.keySet();
-            segmentsAdded = Sets.difference(segmentsReceived, segmentsCached).immutableCopy();
-            segmentsRemoved = Sets.difference(segmentsCached, segmentsReceived).immutableCopy();
-        }
-        return Pair.of(segmentsAdded, segmentsRemoved);
-    }
-
-    /**
-     * Remove log segment <code>name</code> from the cache.
-     *
-     * @param name
-     *          name of the log segment.
-     * @return log segment metadata.
-     */
-    public LogSegmentMetadata remove(String name) {
-        synchronized (logSegments) {
-            LogSegmentMetadata metadata = logSegments.remove(name);
-            if (null != metadata) {
-                lid2LogSegments.remove(metadata.getLogSegmentId(), metadata);
-                LOG.debug("Removed log segment ({} : {}) from cache.", name, metadata);
-            }
-            return metadata;
-        }
-    }
-
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/RollingPolicy.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/RollingPolicy.java
deleted file mode 100644
index 0101bff..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/RollingPolicy.java
+++ /dev/null
@@ -1,33 +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 com.twitter.distributedlog.logsegment;
-
-import com.twitter.distributedlog.util.Sizable;
-
-public interface RollingPolicy {
-    /**
-     * Determines if a rollover may be appropriate at this time.
-     *
-     * @param sizable
-     *          Any object that is sizable.
-     * @param lastRolloverTimeMs
-     *          last rolling time in millis.
-     * @return true if a rollover is required. otherwise, false.
-     */
-    boolean shouldRollover(Sizable sizable, long lastRolloverTimeMs);
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/SizeBasedRollingPolicy.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/SizeBasedRollingPolicy.java
deleted file mode 100644
index 8b1fa0f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/SizeBasedRollingPolicy.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 com.twitter.distributedlog.logsegment;
-
-import com.twitter.distributedlog.util.Sizable;
-
-public class SizeBasedRollingPolicy implements RollingPolicy {
-
-    final long maxSize;
-
-    public SizeBasedRollingPolicy(long maxSize) {
-        this.maxSize = maxSize;
-    }
-
-    @Override
-    public boolean shouldRollover(Sizable sizable, long lastRolloverTimeMs) {
-        return sizable.size() > maxSize;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/TimeBasedRollingPolicy.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/TimeBasedRollingPolicy.java
deleted file mode 100644
index bc88720..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/TimeBasedRollingPolicy.java
+++ /dev/null
@@ -1,46 +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 com.twitter.distributedlog.logsegment;
-
-import com.twitter.distributedlog.util.Sizable;
-import com.twitter.distributedlog.util.Utils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TimeBasedRollingPolicy implements RollingPolicy {
-
-    final static Logger LOG = LoggerFactory.getLogger(TimeBasedRollingPolicy.class);
-
-    final long rollingIntervalMs;
-
-    public TimeBasedRollingPolicy(long rollingIntervalMs) {
-        this.rollingIntervalMs = rollingIntervalMs;
-    }
-
-    @Override
-    public boolean shouldRollover(Sizable sizable, long lastRolloverTimeMs) {
-        long elapsedMs = Utils.elapsedMSec(lastRolloverTimeMs);
-        boolean shouldSwitch = elapsedMs > rollingIntervalMs;
-        if (shouldSwitch) {
-            LOG.debug("Last Finalize Time: {} elapsed time (MSec): {}", lastRolloverTimeMs,
-                      elapsedMs);
-        }
-        return shouldSwitch;
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/package-info.java
deleted file mode 100644
index 0f5b877..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Log Segment Management
- */
-package com.twitter.distributedlog.logsegment;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLConfig.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLConfig.java
deleted file mode 100644
index 178074a..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLConfig.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 com.twitter.distributedlog.metadata;
-
-import java.io.IOException;
-
-/**
- * Specific config of a given implementation of DL
- */
-public interface DLConfig {
-    /**
-     * Serialize the dl config into a string.
-     */
-    public String serialize();
-
-    /**
-     * Deserialize the dl config from a readable stream.
-     *
-     * @param data
-     *          bytes to desrialize dl config.
-     * @throws IOException if fail to deserialize the dl config string representation.
-     */
-    public void deserialize(byte[] data) throws IOException;
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLMetadata.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLMetadata.java
deleted file mode 100644
index c0b5fb7..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLMetadata.java
+++ /dev/null
@@ -1,227 +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 com.twitter.distributedlog.metadata;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.BufferedReader;
-import java.io.IOException;
-import java.io.StringReader;
-import java.net.URI;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Metadata of a given DL instance.
- */
-public class DLMetadata {
-
-    static final Logger LOG = LoggerFactory.getLogger(DLMetadata.class);
-
-    static final String LINE_SPLITTER = "\n";
-    static final String BK_DL_TYPE = "BKDL";
-    static final int METADATA_FORMAT_VERSION = 1;
-
-    // metadata format version
-    private int metadataFormatVersion = 0;
-    // underlying dl type
-    private String dlType;
-    // underlying dl config
-    private DLConfig dlConfig;
-
-    public DLMetadata(String dlType, DLConfig dlConfig) {
-        this(dlType, dlConfig, METADATA_FORMAT_VERSION);
-    }
-
-    DLMetadata(String dlType, DLConfig dlConfig, int metadataFormatVersion) {
-        this.dlType = dlType;
-        this.dlConfig = dlConfig;
-        this.metadataFormatVersion = metadataFormatVersion;
-    }
-
-    /**
-     * @return DL type
-     */
-    public String getDLType() {
-        return dlType;
-    }
-
-    /**
-     * @return DL Config
-     */
-    public DLConfig getDLConfig() {
-        return dlConfig;
-    }
-
-    /**
-     * Serialize the DL metadata into bytes array.
-     *
-     * @return bytes of DL metadata.
-     */
-    public byte[] serialize() {
-        StringBuilder sb = new StringBuilder();
-        sb.append(metadataFormatVersion).append(LINE_SPLITTER);
-        sb.append(dlType).append(LINE_SPLITTER);
-        sb.append(dlConfig.serialize());
-        LOG.debug("Serialized dl metadata {}.", sb);
-        return sb.toString().getBytes(UTF_8);
-    }
-
-    @Override
-    public int hashCode() {
-        return dlType.hashCode() * 13 + dlConfig.hashCode();
-    }
-
-    @Override
-    public String toString() {
-        return new String(serialize(), UTF_8);
-    }
-
-    public void update(URI uri) throws IOException {
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
-                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                .retryThreadCount(conf.getZKClientNumberRetryThreads())
-                .requestRateLimit(conf.getZKRequestRateLimit())
-                .zkAclId(conf.getZkAclId())
-                .uri(uri)
-                .build();
-        byte[] data = serialize();
-        try {
-            zkc.get().setData(uri.getPath(), data, -1);
-        } catch (KeeperException e) {
-            throw new IOException("Fail to update dl metadata " + new String(data, UTF_8)
-                    + " to uri " + uri, e);
-        } catch (InterruptedException e) {
-            throw new IOException("Interrupted when updating dl metadata "
-                    + new String(data, UTF_8) + " to uri " + uri, e);
-        } finally {
-            zkc.close();
-        }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof DLMetadata)) {
-            return false;
-        }
-        DLMetadata other = (DLMetadata) o;
-        return dlType.equals(other.dlType) && dlConfig.equals(other.dlConfig);
-    }
-
-    public void create(URI uri) throws IOException {
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
-                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                .retryThreadCount(conf.getZKClientNumberRetryThreads())
-                .requestRateLimit(conf.getZKRequestRateLimit())
-                .zkAclId(conf.getZkAclId())
-                .uri(uri)
-                .build();
-        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);
-        } finally {
-            zkc.close();
-        }
-    }
-
-    public static void unbind(URI uri) throws IOException {
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
-                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                .retryThreadCount(conf.getZKClientNumberRetryThreads())
-                .requestRateLimit(conf.getZKRequestRateLimit())
-                .zkAclId(conf.getZkAclId())
-                .uri(uri)
-                .build();
-        byte[] data = new byte[0];
-        try {
-            zkc.get().setData(uri.getPath(), data, -1);
-        } catch (KeeperException ke) {
-            throw new IOException("Fail to unbound dl metadata on uri " + uri, ke);
-        } catch (InterruptedException ie) {
-            throw new IOException("Interrupted when unbinding dl metadata on uri " + uri, ie);
-        } finally {
-            zkc.close();
-        }
-    }
-
-    /**
-     * Deserialize dl metadata of given <i>uri</i> from a given bytes array.
-     *
-     * @param uri
-     *          uri that stored dl metadata bindings
-     * @param data
-     *          bytes of dl metadata
-     * @return dl metadata
-     * @throws IOException if failed to parse the bytes array
-     */
-    public static DLMetadata deserialize(URI uri, byte[] data) throws IOException {
-        String metadata = new String(data, UTF_8);
-        LOG.debug("Parsing dl metadata {}.", metadata);
-        BufferedReader br = new BufferedReader(new StringReader(metadata));
-        String versionLine = br.readLine();
-        if (null == versionLine) {
-            throw new IOException("Empty DL Metadata.");
-        }
-        int version;
-        try {
-            version = Integer.parseInt(versionLine);
-        } catch (NumberFormatException nfe) {
-            version = -1;
-        }
-        if (METADATA_FORMAT_VERSION != version) {
-            throw new IOException("Metadata version not compatible. Expected "
-                    + METADATA_FORMAT_VERSION + " but got " + version);
-        }
-        String type = br.readLine();
-        if (!BK_DL_TYPE.equals(type)) {
-            throw new IOException("Invalid DL type : " + type);
-        }
-        DLConfig dlConfig = new BKDLConfig(uri);
-        StringBuilder sb = new StringBuilder();
-        String line;
-        while (null != (line = br.readLine())) {
-            sb.append(line);
-        }
-        dlConfig.deserialize(sb.toString().getBytes(UTF_8));
-        return new DLMetadata(type, dlConfig, version);
-    }
-
-    public static DLMetadata create(BKDLConfig bkdlConfig) {
-        return new DLMetadata(BK_DL_TYPE, bkdlConfig);
-    }
-
-}
-
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
deleted file mode 100644
index b2a417e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.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 com.twitter.distributedlog.metadata;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.util.Future;
-
-public class DryrunLogSegmentMetadataStoreUpdater extends LogSegmentMetadataStoreUpdater {
-
-    public DryrunLogSegmentMetadataStoreUpdater(DistributedLogConfiguration conf,
-                                                LogSegmentMetadataStore metadataStore) {
-        super(conf, metadataStore);
-    }
-
-    @Override
-    public Transaction<Object> transaction() {
-        return new Transaction<Object>() {
-            @Override
-            public void addOp(Op<Object> operation) {
-                // no-op
-            }
-
-            @Override
-            public Future<Void> execute() {
-                return Future.Void();
-            }
-
-            @Override
-            public void abort(Throwable reason) {
-                // no-op
-            }
-        };
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadata.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadata.java
deleted file mode 100644
index c878d68..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadata.java
+++ /dev/null
@@ -1,175 +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 com.twitter.distributedlog.metadata;
-
-import java.net.URI;
-
-/**
- * Class to represent the layout and metadata of the zookeeper-based log metadata
- */
-public class LogMetadata {
-
-    protected static String getLogComponentPath(URI uri, String logName, String logIdentifier, String component) {
-        return String.format("%s/%s/%s%s", uri.getPath(), logName, logIdentifier, component);
-    }
-
-    /**
-     * Get the top stream path for a given log.
-     *
-     * @param uri namespace to store the log
-     * @param logName name of the log
-     * @return top stream path
-     */
-    public static String getLogStreamPath(URI uri, String logName) {
-        return String.format("%s/%s", uri.getPath(), logName);
-    }
-
-    /**
-     * Get the log root path for a given log.
-     *
-     * @param uri
-     *          namespace to store the log
-     * @param logName
-     *          name of the log
-     * @param logIdentifier
-     *          identifier of the log
-     * @return log root path
-     */
-    public static String getLogRootPath(URI uri, String logName, String logIdentifier) {
-        return getLogComponentPath(uri, logName, logIdentifier, "");
-    }
-
-    /**
-     * Get the logsegments root path for a given log.
-     *
-     * @param uri
-     *          namespace to store the log
-     * @param logName
-     *          name of the log
-     * @param logIdentifier
-     *          identifier of the log
-     * @return logsegments root path
-     */
-    public static String getLogSegmentsPath(URI uri, String logName, String logIdentifier) {
-        return getLogComponentPath(uri, logName, logIdentifier, LOGSEGMENTS_PATH);
-    }
-
-    public static final int LAYOUT_VERSION = -1;
-    public final static String LOGSEGMENTS_PATH = "/ledgers";
-    public final static String VERSION_PATH = "/version";
-    // writer znodes
-    public final static String MAX_TXID_PATH = "/maxtxid";
-    public final static String LOCK_PATH = "/lock";
-    public final static String ALLOCATION_PATH = "/allocation";
-    // reader znodes
-    public final static String READ_LOCK_PATH = "/readLock";
-
-    protected final URI uri;
-    protected final String logName;
-    protected final String logIdentifier;
-
-    // Root path of the log
-    protected final String logRootPath;
-    // Components
-    protected final String logSegmentsPath;
-    protected final String lockPath;
-    protected final String maxTxIdPath;
-    protected final String allocationPath;
-
-    /**
-     * metadata representation of a log
-     *
-     * @param uri
-     *          namespace to store the log
-     * @param logName
-     *          name of the log
-     * @param logIdentifier
-     *          identifier of the log
-     */
-    protected LogMetadata(URI uri,
-                          String logName,
-                          String logIdentifier) {
-        this.uri = uri;
-        this.logName = logName;
-        this.logIdentifier = logIdentifier;
-        this.logRootPath = getLogRootPath(uri, logName, logIdentifier);
-        this.logSegmentsPath = logRootPath + LOGSEGMENTS_PATH;
-        this.lockPath = logRootPath + LOCK_PATH;
-        this.maxTxIdPath = logRootPath + MAX_TXID_PATH;
-        this.allocationPath = logRootPath + ALLOCATION_PATH;
-    }
-
-    public URI getUri() {
-        return uri;
-    }
-
-    public String getLogName() {
-        return logName;
-    }
-
-    /**
-     * Get the root path of the log.
-     *
-     * @return root path of the log.
-     */
-    public String getLogRootPath() {
-        return logRootPath;
-    }
-
-    /**
-     * Get the root path for log segments.
-     *
-     * @return root path for log segments
-     */
-    public String getLogSegmentsPath() {
-        return this.logSegmentsPath;
-    }
-
-    /**
-     * Get the path for a log segment of the log.
-     *
-     * @param segmentName
-     *          segment name
-     * @return path for the log segment
-     */
-    public String getLogSegmentPath(String segmentName) {
-        return this.logSegmentsPath + "/" + segmentName;
-    }
-
-    public String getLockPath() {
-        return lockPath;
-    }
-
-    public String getMaxTxIdPath() {
-        return maxTxIdPath;
-    }
-
-    public String getAllocationPath() {
-        return allocationPath;
-    }
-
-    /**
-     * Get the fully qualified name of the log.
-     *
-     * @return fully qualified name
-     */
-    public String getFullyQualifiedName() {
-        return String.format("%s:%s", logName, logIdentifier);
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForReader.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForReader.java
deleted file mode 100644
index ff6bfca..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForReader.java
+++ /dev/null
@@ -1,103 +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 com.twitter.distributedlog.metadata;
-
-import com.google.common.base.Optional;
-
-import java.net.URI;
-
-/**
- * Log Metadata for Reader
- */
-public class LogMetadataForReader extends LogMetadata {
-
-    /**
-     * Get the root path to store subscription infos of a log.
-     *
-     * @param uri
-     *          namespace of the log
-     * @param logName
-     *          name of the log
-     * @param logIdentifier
-     *          identifier of the log
-     * @return subscribers root path
-     */
-    public static String getSubscribersPath(URI uri, String logName, String logIdentifier) {
-        return getLogComponentPath(uri, logName, logIdentifier, SUBSCRIBERS_PATH);
-    }
-
-    /**
-     * Get the path that stores subscription info for a <code>subscriberId</code> for a <code>log</code>.
-     *
-     * @param uri
-     *          namespace of the log
-     * @param logName
-     *          name of the log
-     * @param logIdentifier
-     *          identifier of the log
-     * @param subscriberId
-     *          subscriber id of the log
-     * @return subscriber's path
-     */
-    public static String getSubscriberPath(URI uri, String logName, String logIdentifier, String subscriberId) {
-        return String.format("%s/%s", getSubscribersPath(uri, logName, logIdentifier), subscriberId);
-    }
-
-    /**
-     * Create a metadata representation of a log for reader.
-     *
-     * @param uri
-     *          namespace to store the log
-     * @param logName
-     *          name of the log
-     * @param logIdentifier
-     *          identifier of the log
-     * @return metadata representation of a log for reader
-     */
-    public static LogMetadataForReader of(URI uri, String logName, String logIdentifier) {
-        return new LogMetadataForReader(uri, logName, logIdentifier);
-    }
-
-    final static String SUBSCRIBERS_PATH = "/subscribers";
-
-    /**
-     * metadata representation of a log
-     *
-     * @param uri           namespace to store the log
-     * @param logName       name of the log
-     * @param logIdentifier identifier of the log
-     */
-    private LogMetadataForReader(URI uri, String logName, String logIdentifier) {
-        super(uri, logName, logIdentifier);
-    }
-
-    /**
-     * Get the readlock path for the log or a subscriber of the log.
-     *
-     * @param subscriberId
-     *          subscriber id. it is optional.
-     * @return read lock path
-     */
-    public String getReadLockPath(Optional<String> subscriberId) {
-        if (subscriberId.isPresent()) {
-            return logRootPath + SUBSCRIBERS_PATH + "/" + subscriberId.get() + READ_LOCK_PATH;
-        } else {
-            return logRootPath + READ_LOCK_PATH;
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForWriter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForWriter.java
deleted file mode 100644
index 2284cbb..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForWriter.java
+++ /dev/null
@@ -1,64 +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 com.twitter.distributedlog.metadata;
-
-import org.apache.bookkeeper.versioning.Versioned;
-
-import java.net.URI;
-
-/**
- * Log Metadata for writer
- */
-public class LogMetadataForWriter extends LogMetadata {
-
-    private final Versioned<byte[]> maxLSSNData;
-    private final Versioned<byte[]> maxTxIdData;
-    private final Versioned<byte[]> allocationData;
-
-    /**
-     * metadata representation of a log
-     *
-     * @param uri           namespace to store the log
-     * @param logName       name of the log
-     * @param logIdentifier identifier of the log
-     */
-    public LogMetadataForWriter(URI uri,
-                                String logName,
-                                String logIdentifier,
-                                Versioned<byte[]> maxLSSNData,
-                                Versioned<byte[]> maxTxIdData,
-                                Versioned<byte[]> allocationData) {
-        super(uri, logName, logIdentifier);
-        this.maxLSSNData = maxLSSNData;
-        this.maxTxIdData = maxTxIdData;
-        this.allocationData = allocationData;
-    }
-
-    public Versioned<byte[]> getMaxLSSNData() {
-        return maxLSSNData;
-    }
-
-    public Versioned<byte[]> getMaxTxIdData() {
-        return maxTxIdData;
-    }
-
-    public Versioned<byte[]> getAllocationData() {
-        return allocationData;
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataStore.java
deleted file mode 100644
index 01dccb7..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataStore.java
+++ /dev/null
@@ -1,66 +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 com.twitter.distributedlog.metadata;
-
-import com.google.common.annotations.Beta;
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.util.Future;
-
-import java.net.URI;
-import java.util.Iterator;
-
-/**
- * Interface for log metadata store.
- */
-@Beta
-public interface LogMetadataStore {
-
-    /**
-     * Create a stream and return it is namespace location.
-     *
-     * @param logName
-     *          name of the log
-     * @return namespace location that stores this stream.
-     */
-    Future<URI> createLog(String logName);
-
-    /**
-     * Get the location of the log.
-     *
-     * @param logName
-     *          name of the log
-     * @return namespace location that stores this stream.
-     */
-    Future<Optional<URI>> getLogLocation(String logName);
-
-    /**
-     * Retrieves logs from the namespace.
-     *
-     * @return iterator of logs of the namespace.
-     */
-    Future<Iterator<String>> getLogs();
-
-    /**
-     * Register a namespace listener on streams changes.
-     *
-     * @param listener
-     *          namespace listener
-     */
-    void registerNamespaceListener(NamespaceListener listener);
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
deleted file mode 100644
index 0e5e6d4..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
+++ /dev/null
@@ -1,184 +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 com.twitter.distributedlog.metadata;
-
-import com.google.common.base.Preconditions;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.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 {
-
-    static final Logger LOG = LoggerFactory.getLogger(LogSegmentMetadataStoreUpdater.class);
-
-    public static MetadataUpdater createMetadataUpdater(DistributedLogConfiguration conf,
-                                                        LogSegmentMetadataStore metadataStore) {
-        return new LogSegmentMetadataStoreUpdater(conf, metadataStore);
-    }
-
-    protected final LogSegmentMetadataStore metadataStore;
-    protected final LogSegmentMetadata.LogSegmentMetadataVersion metadataVersion;
-
-    protected LogSegmentMetadataStoreUpdater(DistributedLogConfiguration conf,
-                                             LogSegmentMetadataStore metadataStore) {
-        this.metadataStore = metadataStore;
-        this.metadataVersion = LogSegmentMetadata.LogSegmentMetadataVersion.of(conf.getDLLedgerMetadataLayoutVersion());
-    }
-
-    private String formatLogSegmentSequenceNumber(long logSegmentSeqNo) {
-        return String.format("%018d", logSegmentSeqNo);
-    }
-
-    @Override
-    public Transaction<Object> transaction() {
-        return metadataStore.transaction();
-    }
-
-    @Override
-    public Future<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.");
-        Preconditions.checkArgument(segment.isDLSNinThisSegment(dlsn),
-                "DLSN " + dlsn + " doesn't belong to segment " + segment);
-        final LogSegmentMetadata newSegment = segment.mutator()
-                .setLastDLSN(dlsn)
-                .setLastTxId(record.getTransactionId())
-                .setRecordCount(record)
-                .build();
-        return updateSegmentMetadata(newSegment);
-    }
-
-    @Override
-    public Future<LogSegmentMetadata> changeSequenceNumber(LogSegmentMetadata segment,
-                                                           long logSegmentSeqNo) {
-        String newZkPath = segment.getZkPath()
-                .replace(formatLogSegmentSequenceNumber(segment.getLogSegmentSequenceNumber()),
-                        formatLogSegmentSequenceNumber(logSegmentSeqNo));
-        final LogSegmentMetadata newSegment = segment.mutator()
-                .setLogSegmentSequenceNumber(logSegmentSeqNo)
-                .setZkPath(newZkPath)
-                .build();
-        return addNewSegmentAndDeleteOldSegment(newSegment, segment);
-    }
-
-    /**
-     * Change the truncation status of a <i>log segment</i> to be active
-     *
-     * @param segment log segment to change truncation status to active.
-     * @return new log segment
-     */
-    @Override
-    public Future<LogSegmentMetadata> setLogSegmentActive(LogSegmentMetadata segment) {
-        final LogSegmentMetadata newSegment = segment.mutator()
-            .setTruncationStatus(LogSegmentMetadata.TruncationStatus.ACTIVE)
-            .build();
-        return addNewSegmentAndDeleteOldSegment(newSegment, segment);
-    }
-
-    /**
-     * Change the truncation status of a <i>log segment</i> to truncated
-     *
-     * @param segment log segment to change truncation status to truncated.
-     * @return new log segment
-     */
-    @Override
-    public Future<LogSegmentMetadata> setLogSegmentTruncated(LogSegmentMetadata segment) {
-        final LogSegmentMetadata newSegment = segment.mutator()
-            .setTruncationStatus(LogSegmentMetadata.TruncationStatus.TRUNCATED)
-            .build();
-        return addNewSegmentAndDeleteOldSegment(newSegment, segment);
-    }
-
-    @Override
-    public LogSegmentMetadata setLogSegmentTruncated(Transaction<Object> txn, LogSegmentMetadata segment) {
-        final LogSegmentMetadata newSegment = segment.mutator()
-            .setTruncationStatus(LogSegmentMetadata.TruncationStatus.TRUNCATED)
-            .build();
-        addNewSegmentAndDeleteOldSegment(txn, newSegment, segment);
-        return newSegment;
-    }
-
-    /**
-     * Change the truncation status of a <i>log segment</i> to partially truncated
-     *
-     * @param segment log segment to change sequence number.
-     * @param minActiveDLSN DLSN within the log segment before which log has been truncated
-     * @return new log segment
-     */
-    @Override
-    public Future<LogSegmentMetadata> setLogSegmentPartiallyTruncated(LogSegmentMetadata segment, DLSN minActiveDLSN) {
-        final LogSegmentMetadata newSegment = segment.mutator()
-            .setTruncationStatus(LogSegmentMetadata.TruncationStatus.PARTIALLY_TRUNCATED)
-            .setMinActiveDLSN(minActiveDLSN)
-            .build();
-        return addNewSegmentAndDeleteOldSegment(newSegment, segment);
-    }
-
-    @Override
-    public LogSegmentMetadata setLogSegmentPartiallyTruncated(Transaction<Object> txn,
-                                                              LogSegmentMetadata segment,
-                                                              DLSN minActiveDLSN) {
-        final LogSegmentMetadata newSegment = segment.mutator()
-                .setTruncationStatus(LogSegmentMetadata.TruncationStatus.PARTIALLY_TRUNCATED)
-                .setMinActiveDLSN(minActiveDLSN)
-                .build();
-        addNewSegmentAndDeleteOldSegment(txn, newSegment, segment);
-        return newSegment;
-    }
-
-    protected Future<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;
-            }
-        });
-    }
-
-    protected Future<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;
-            }
-        });
-    }
-
-    protected void addNewSegmentAndDeleteOldSegment(Transaction<Object> txn,
-                                                    LogSegmentMetadata newSegment,
-                                                    LogSegmentMetadata oldSegment) {
-        metadataStore.deleteLogSegment(txn, oldSegment, null);
-        metadataStore.createLogSegment(txn, newSegment, null);
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogStreamMetadataStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogStreamMetadataStore.java
deleted file mode 100644
index 7242a5e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogStreamMetadataStore.java
+++ /dev/null
@@ -1,114 +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 com.twitter.distributedlog.metadata;
-
-import com.google.common.annotations.Beta;
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.lock.DistributedLock;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.util.PermitManager;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.util.Future;
-
-import java.io.Closeable;
-import java.net.URI;
-
-/**
- * The interface to manage the log stream metadata. The implementation is responsible
- * for creating the metadata layout.
- */
-@Beta
-public interface LogStreamMetadataStore extends Closeable {
-
-    /**
-     * Create a transaction for the metadata operations happening in the metadata store.
-     *
-     * @return transaction for the metadata operations
-     */
-    Transaction<Object> newTransaction();
-
-    /**
-     * Ensure the existence of a log stream
-     *
-     * @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 com.twitter.distributedlog.LogNotFoundException}
-     *         is thrown if the log doesn't exist
-     */
-    Future<Void> logExists(URI uri, String logName);
-
-    /**
-     * Create the read lock for the log stream.
-     *
-     * @param metadata the metadata for a log stream
-     * @param readerId the reader id used for lock
-     * @return the read lock
-     */
-    Future<DistributedLock> createReadLock(LogMetadataForReader metadata,
-                                           Optional<String> readerId);
-
-    /**
-     * Create the write lock for the log stream.
-     *
-     * @param metadata the metadata for a log stream
-     * @return the write lock
-     */
-    DistributedLock createWriteLock(LogMetadataForWriter metadata);
-
-    /**
-     * Create the metadata of a log.
-     *
-     * @param uri the location to store the metadata of the log
-     * @param streamName the name of the log stream
-     * @param ownAllocator whether to use its own allocator or external allocator
-     * @param createIfNotExists flag to create the stream if it doesn't exist
-     * @return the metadata of the log
-     */
-    Future<LogMetadataForWriter> getLog(URI uri,
-                                        String streamName,
-                                        boolean ownAllocator,
-                                        boolean createIfNotExists);
-
-    /**
-     * Delete the metadata of a log.
-     *
-     * @param uri the location to store the metadata of the log
-     * @param streamName the name of the log stream
-     * @return future represents the result of the deletion.
-     */
-    Future<Void> deleteLog(URI uri, String streamName);
-
-    /**
-     * Get the log segment metadata store.
-     *
-     * @return the log segment metadata store.
-     */
-    LogSegmentMetadataStore getLogSegmentMetadataStore();
-
-    /**
-     * Get the permit manager for this metadata store. It can be used for limiting the concurrent
-     * metadata operations. The implementation can disable handing over the permits when the metadata
-     * store is unavailable (for example zookeeper session expired).
-     *
-     * @return the permit manager
-     */
-    PermitManager getPermitManager();
-
-
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataResolver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataResolver.java
deleted file mode 100644
index 417cab8..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataResolver.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 com.twitter.distributedlog.metadata;
-
-import java.io.IOException;
-import java.net.URI;
-
-/**
- * Resolver to resolve the metadata used to instantiate a DL instance.
- *
- * <p>
- * E.g. we stored a common dl config under /messaging/distributedlog to use
- * bookkeeper cluster x. so all the distributedlog instances under this path
- * inherit this dl config. if a dl D is allocated under /messaging/distributedlog,
- * but use a different cluster y, so its metadata is stored /messaging/distributedlog/D.
- * The resolver resolve the URI
- * </p>
- *
- * <p>
- * The resolver looks up the uri path and tries to interpret the path segments from
- * bottom-to-top to see if there is a DL metadata bound. It stops when it found valid
- * dl metadata.
- * </p>
- */
-public interface MetadataResolver {
-
-    /**
-     * Resolve the path to get the DL metadata.
-     *
-     * @param uri
-     *          dl uri
-     * @return dl metadata.
-     * @throws IOException
-     */
-    public DLMetadata resolve(URI uri) throws IOException;
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataUpdater.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataUpdater.java
deleted file mode 100644
index b98f168..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataUpdater.java
+++ /dev/null
@@ -1,120 +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 com.twitter.distributedlog.metadata;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.util.Future;
-
-/**
- * An updater to update metadata. It contains utility functions on mutating metadata.
- */
-public interface MetadataUpdater {
-
-    /**
-     * Start a transaction on metadata updates
-     *
-     * @return transaction
-     */
-    Transaction<Object> transaction();
-
-    /**
-     * Update the log segment metadata with correct last <i>record</i>.
-     *
-     * @param segment
-     *          log segment to update last dlsn.
-     * @param record
-     *          correct last record.
-     * @return new log segment
-     */
-    Future<LogSegmentMetadata> updateLastRecord(LogSegmentMetadata segment,
-                                                LogRecordWithDLSN record);
-
-    /**
-     * Change ledger sequence number of <i>segment</i> to given <i>logSegmentSeqNo</i>.
-     *
-     * @param segment
-     *          log segment to change sequence number.
-     * @param logSegmentSeqNo
-     *          ledger sequence number to change.
-     * @return new log segment
-     */
-    Future<LogSegmentMetadata> changeSequenceNumber(LogSegmentMetadata segment,
-                                                    long logSegmentSeqNo);
-
-    /**
-     * Change the truncation status of a <i>log segment</i> to be active
-     *
-     * @param segment
-     *          log segment to change truncation status to active.
-     * @return new log segment
-     */
-    Future<LogSegmentMetadata> setLogSegmentActive(LogSegmentMetadata segment);
-
-    /**
-     * Change the truncation status of a <i>log segment</i> to truncated
-     *
-     * @param segment
-     *          log segment to change truncation status to truncated.
-     * @return new log segment
-     */
-    Future<LogSegmentMetadata> setLogSegmentTruncated(LogSegmentMetadata segment);
-
-    /**
-     * Change the truncation status of a <i>log segment</i> to truncated. The operation won't be executed
-     * immediately. The update only happens after {@link Transaction#execute()}.
-     *
-     * @param txn
-     *          transaction used to set the log segment status
-     * @param segment
-     *          segment to set truncation status to truncated
-     * @return log segment that truncation status is set to truncated.
-     */
-    LogSegmentMetadata setLogSegmentTruncated(Transaction<Object> txn, LogSegmentMetadata segment);
-
-    /**
-     * Change the truncation status of a <i>log segment</i> to partially truncated
-     *
-     * @param segment
-     *          log segment to change sequence number.
-     * @param minActiveDLSN
-     *          DLSN within the log segment before which log has been truncated
-     * @return new log segment
-     */
-    Future<LogSegmentMetadata> setLogSegmentPartiallyTruncated(LogSegmentMetadata segment,
-                                                               DLSN minActiveDLSN);
-
-    /**
-     * Change the truncation status of a <i>log segment</i> to partially truncated. The operation won't be
-     * executed until {@link Transaction#execute()}.
-     *
-     * @param txn
-     *          transaction used to set the log segment status
-     * @param segment
-     *          segment to set truncation status to partially truncated
-     * @param minActiveDLSN
-     *          DLSN within the log segment before which log has been truncated
-     * @return log segment that truncation status has been set to partially truncated
-     */
-    LogSegmentMetadata setLogSegmentPartiallyTruncated(Transaction<Object> txn,
-                                                       LogSegmentMetadata segment,
-                                                       DLSN minActiveDLSN);
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/package-info.java
deleted file mode 100644
index f740c77..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Metadata management for distributedlog.
- */
-package com.twitter.distributedlog.metadata;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespace.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespace.java
deleted file mode 100644
index 5d1d888..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespace.java
+++ /dev/null
@@ -1,190 +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 com.twitter.distributedlog.namespace;
-
-import com.google.common.annotations.Beta;
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * A namespace is the basic unit for managing a set of distributedlogs.
- *
- * <h4>Namespace Interface</h4>
- *
- * <P>
- * The <code>DistributedLogNamespace</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,
- *         watch the changes of logs.
- *     <LI>Access Control -- manage the access controls for logs in the namespace.
- * </OL>
- * </P>
- *
- * <h4>Namespace Location</h4>
- *
- * 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.
- *
- * 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.
- *
- * @see DistributedLogManager
- * @since 0.3.32
- */
-@Beta
-public interface DistributedLogNamespace {
-
-    /**
-     * Get the namespace driver used by this namespace.
-     *
-     * @return namespace driver
-     */
-    NamespaceDriver getNamespaceDriver();
-
-    //
-    // Method to operate logs
-    //
-
-    /**
-     * Create a log named <i>logName</i>.
-     *
-     * @param logName
-     *          name of the log
-     * @throws InvalidStreamNameException if log name is invalid.
-     * @throws IOException when encountered issues with backend.
-     */
-    void createLog(String logName)
-            throws InvalidStreamNameException, IOException;
-
-    /**
-     * Delete a log named <i>logName</i>.
-     *
-     * @param logName
-     *          name of the log
-     * @throws InvalidStreamNameException if log name is invalid
-     * @throws LogNotFoundException if log doesn't exist
-     * @throws IOException when encountered issues with backend
-     */
-    void deleteLog(String logName)
-            throws InvalidStreamNameException, LogNotFoundException, IOException;
-
-    /**
-     * Open a log named <i>logName</i>.
-     * A distributedlog manager is returned to access log <i>logName</i>.
-     *
-     * @param logName
-     *          name of the log
-     * @return distributedlog manager instance.
-     * @throws InvalidStreamNameException if log name is invalid.
-     * @throws IOException when encountered issues with backend.
-     */
-    DistributedLogManager openLog(String logName)
-            throws InvalidStreamNameException, IOException;
-
-    /**
-     * Open a log named <i>logName</i> with specific log configurations.
-     *
-     * <p>This method allows the caller to override global configuration settings by
-     * supplying log configuration overrides. Log config overrides come in two flavors,
-     * static and dynamic. Static config never changes in the lifecyle of <code>DistributedLogManager</code>,
-     * dynamic config changes by reloading periodically and safe to access from any context.</p>
-     *
-     * @param logName
-     *          name of the log
-     * @param logConf
-     *          static log configuration
-     * @param dynamicLogConf
-     *          dynamic log configuration
-     * @return distributedlog manager instance.
-     * @throws InvalidStreamNameException if log name is invalid.
-     * @throws IOException when encountered issues with backend.
-     */
-    DistributedLogManager openLog(String logName,
-                                  Optional<DistributedLogConfiguration> logConf,
-                                  Optional<DynamicDistributedLogConfiguration> dynamicLogConf,
-                                  Optional<StatsLogger> perStreamStatsLogger)
-            throws InvalidStreamNameException, IOException;
-
-    /**
-     * Check whether the log <i>logName</i> exist.
-     *
-     * @param logName
-     *          name of the log
-     * @return <code>true</code> if the log exists, otherwise <code>false</code>.
-     * @throws IOException when encountered exceptions on checking
-     */
-    boolean logExists(String logName)
-            throws IOException;
-
-    /**
-     * Retrieve the logs under the namespace.
-     *
-     * @return iterator of the logs under the namespace.
-     * @throws IOException when encountered issues with backend.
-     */
-    Iterator<String> getLogs()
-            throws IOException;
-
-    //
-    // Methods for namespace
-    //
-
-    /**
-     * Register namespace listener on stream updates under the namespace.
-     *
-     * @param listener
-     *          listener to receive stream updates under the namespace
-     */
-    void registerNamespaceListener(NamespaceListener listener);
-
-    /**
-     * Create an access control manager to manage/check acl for logs.
-     *
-     * @return access control manager for logs under the namespace.
-     * @throws IOException
-     */
-    AccessControlManager createAccessControlManager()
-            throws IOException;
-
-    /**
-     * Close the namespace.
-     */
-    void close();
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespaceBuilder.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespaceBuilder.java
deleted file mode 100644
index 07b3848..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespaceBuilder.java
+++ /dev/null
@@ -1,278 +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 com.twitter.distributedlog.namespace;
-
-import com.google.common.base.Preconditions;
-import com.twitter.distributedlog.BKDistributedLogNamespace;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.feature.CoreFeatureKeys;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.injector.AsyncRandomFailureInjector;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.PermitLimiter;
-import com.twitter.distributedlog.util.SimplePermitLimiter;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.SettableFeatureProvider;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-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 {
-
-    private static final Logger logger = LoggerFactory.getLogger(DistributedLogNamespaceBuilder.class);
-
-    public static DistributedLogNamespaceBuilder newBuilder() {
-        return new DistributedLogNamespaceBuilder();
-    }
-
-    private DistributedLogConfiguration _conf = null;
-    private DynamicDistributedLogConfiguration _dynConf = null;
-    private URI _uri = null;
-    private StatsLogger _statsLogger = NullStatsLogger.INSTANCE;
-    private StatsLogger _perLogStatsLogger = NullStatsLogger.INSTANCE;
-    private FeatureProvider _featureProvider = null;
-    private String _clientId = DistributedLogConstants.UNKNOWN_CLIENT_ID;
-    private int _regionId = DistributedLogConstants.LOCAL_REGION_ID;
-
-    // private constructor
-    private DistributedLogNamespaceBuilder() {}
-
-    /**
-     * DistributedLog Configuration used for the namespace.
-     *
-     * @param conf
-     *          distributedlog configuration
-     * @return namespace builder.
-     */
-    public DistributedLogNamespaceBuilder conf(DistributedLogConfiguration conf) {
-        this._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._dynConf = dynConf;
-        return this;
-    }
-
-    /**
-     * Namespace Location.
-     *
-     * @param uri
-     *          namespace location uri.
-     * @see DistributedLogNamespace
-     * @return namespace builder.
-     */
-    public DistributedLogNamespaceBuilder uri(URI uri) {
-        this._uri = uri;
-        return this;
-    }
-
-    /**
-     * Stats Logger used for stats collection
-     *
-     * @param statsLogger
-     *          stats logger
-     * @return namespace builder.
-     */
-    public DistributedLogNamespaceBuilder statsLogger(StatsLogger statsLogger) {
-        this._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._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._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._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._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 {
-        // Check arguments
-        Preconditions.checkNotNull(_conf, "No DistributedLog Configuration.");
-        Preconditions.checkNotNull(_uri, "No DistributedLog URI");
-
-        // validate the configuration
-        _conf.validate();
-        if (null == _dynConf) {
-            _dynConf = ConfUtils.getConstDynConf(_conf);
-        }
-
-        // retrieve the namespace driver
-        NamespaceDriver driver = NamespaceDriverManager.getDriver(_uri);
-        URI normalizedUri = DLUtils.normalizeURI(_uri);
-
-        // build the feature provider
-        FeatureProvider featureProvider;
-        if (null == _featureProvider) {
-            featureProvider = new SettableFeatureProvider("", 0);
-            logger.info("No feature provider is set. All features are disabled now.");
-        } else {
-            featureProvider = _featureProvider;
-        }
-
-        // build the failure injector
-        AsyncFailureInjector failureInjector = AsyncRandomFailureInjector.newBuilder()
-                .injectDelays(_conf.getEIInjectReadAheadDelay(),
-                              _conf.getEIInjectReadAheadDelayPercent(),
-                              _conf.getEIInjectMaxReadAheadDelayMs())
-                .injectErrors(false, 10)
-                .injectStops(_conf.getEIInjectReadAheadStall(), 10)
-                .injectCorruption(_conf.getEIInjectReadAheadBrokenEntries())
-                .build();
-
-        // normalize the per log stats logger
-        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
-        driver.initialize(
-                _conf,
-                _dynConf,
-                normalizedUri,
-                scheduler,
-                featureProvider,
-                failureInjector,
-                _statsLogger,
-                perLogStatsLogger,
-                DLUtils.normalizeClientId(_clientId),
-                _regionId);
-
-        // initialize the write limiter
-        PermitLimiter writeLimiter;
-        if (_conf.getGlobalOutstandingWriteLimit() < 0) {
-            writeLimiter = PermitLimiter.NULL_PERMIT_LIMITER;
-        } else {
-            Feature disableWriteLimitFeature = featureProvider.getFeature(
-                CoreFeatureKeys.DISABLE_WRITE_LIMIT.name().toLowerCase());
-            writeLimiter = new SimplePermitLimiter(
-                _conf.getOutstandingWriteLimitDarkmode(),
-                _conf.getGlobalOutstandingWriteLimit(),
-                _statsLogger.scope("writeLimiter"),
-                true /* singleton */,
-                disableWriteLimitFeature);
-        }
-
-        return new BKDistributedLogNamespace(
-                _conf,
-                normalizedUri,
-                driver,
-                scheduler,
-                featureProvider,
-                writeLimiter,
-                failureInjector,
-                _statsLogger,
-                perLogStatsLogger,
-                DLUtils.normalizeClientId(_clientId),
-                _regionId);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriver.java
deleted file mode 100644
index 738f124..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriver.java
+++ /dev/null
@@ -1,138 +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 com.twitter.distributedlog.namespace;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.MetadataAccessor;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.metadata.LogMetadataStore;
-import com.twitter.distributedlog.metadata.LogStreamMetadataStore;
-import com.twitter.distributedlog.subscription.SubscriptionsStore;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.net.URI;
-
-/**
- * Manager to manage all the stores required by a namespace.
- */
-public interface NamespaceDriver extends Closeable {
-
-    enum Role {
-        WRITER,
-        READER
-    }
-
-    /**
-     * Initialize the namespace manager.
-     *
-     * @param conf distributedlog configuration
-     * @param dynConf dynamic distributedlog configuration
-     * @param namespace root uri of the namespace
-     * @param scheduler ordered scheduler
-     * @param featureProvider feature provider
-     * @param statsLogger stats logger
-     * @param perLogStatsLogger per log stream stats logger
-     * @param clientId client id
-     * @return namespace manager
-     * @throws IOException when failed to initialize the namespace manager
-     */
-    NamespaceDriver initialize(DistributedLogConfiguration conf,
-                               DynamicDistributedLogConfiguration dynConf,
-                               URI namespace,
-                               OrderedScheduler scheduler,
-                               FeatureProvider featureProvider,
-                               AsyncFailureInjector failureInjector,
-                               StatsLogger statsLogger,
-                               StatsLogger perLogStatsLogger,
-                               String clientId,
-                               int regionId) throws IOException;
-
-    /**
-     * Get the scheme of the namespace driver.
-     *
-     * @return the scheme of the namespace driver.
-     */
-    String getScheme();
-
-    /**
-     * Get the root uri of the namespace driver.
-     *
-     * @return the root uri of the namespace driver.
-     */
-    URI getUri();
-
-    /**
-     * Retrieve the log {@code metadata store} used by the namespace.
-     *
-     * @return the log metadata store
-     */
-    LogMetadataStore getLogMetadataStore();
-
-    /**
-     * Retrieve the log stream {@code metadata store} used by the namespace.
-     *
-     * @param role the role to retrieve the log stream metadata store.
-     * @return the log stream metadata store
-     */
-    LogStreamMetadataStore getLogStreamMetadataStore(Role role);
-
-    /**
-     * Retrieve the log segment {@code entry store} used by the namespace.
-     *
-     * @param role the role to retrieve the log segment entry store.
-     * @return the log segment entry store.
-     * @throws IOException when failed to open log segment entry store.
-     */
-    LogSegmentEntryStore getLogSegmentEntryStore(Role role);
-
-    /**
-     * Create an access control manager to manage/check acl for logs.
-     *
-     * @return access control manager for logs under the namespace.
-     * @throws IOException
-     */
-    AccessControlManager getAccessControlManager()
-            throws IOException;
-
-    /**
-     * Retrieve the metadata accessor for log stream {@code streamName}.
-     * (TODO: it is a legacy interface. should remove it if we have metadata of stream.)
-     *
-     * @param streamName name of log stream.
-     * @return metadata accessor for log stream {@code streamName}.
-     */
-    MetadataAccessor getMetadataAccessor(String streamName)
-            throws InvalidStreamNameException, IOException;
-
-    /**
-     * Retrieve the subscriptions store for log stream {@code streamName}.
-     *
-     * @return the subscriptions store for log stream {@code streamName}
-     */
-    SubscriptionsStore getSubscriptionsStore(String streamName);
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriverManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriverManager.java
deleted file mode 100644
index 79945ad..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriverManager.java
+++ /dev/null
@@ -1,180 +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 com.twitter.distributedlog.namespace;
-
-import com.google.common.base.Objects;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import static com.google.common.base.Preconditions.*;
-
-/**
- * The basic service for managing a set of namespace drivers.
- */
-public class NamespaceDriverManager {
-
-    private static final Logger logger = LoggerFactory.getLogger(NamespaceDriverManager.class);
-
-    static class NamespaceDriverInfo {
-
-        final Class<? extends NamespaceDriver> driverClass;
-        final String driverClassName;
-
-        NamespaceDriverInfo(Class<? extends NamespaceDriver> driverClass) {
-            this.driverClass = driverClass;
-            this.driverClassName = this.driverClass.getName();
-        }
-
-        @Override
-        public String toString() {
-            StringBuilder sb = new StringBuilder();
-            sb.append("driver[")
-                    .append(driverClassName)
-                    .append("]");
-            return sb.toString();
-        }
-    }
-
-    private static final ConcurrentMap<String, NamespaceDriverInfo> drivers;
-    private static boolean initialized = false;
-
-    static {
-        drivers = new ConcurrentHashMap<String, NamespaceDriverInfo>();
-        initialize();
-    }
-
-    static void initialize() {
-        if (initialized) {
-            return;
-        }
-        loadInitialDrivers();
-        initialized = true;
-        logger.info("DistributedLog NamespaceDriverManager initialized");
-    }
-
-    private static void loadInitialDrivers() {
-        Set<String> driverList = Sets.newHashSet();
-        // add default bookkeeper based driver
-        driverList.add(BKNamespaceDriver.class.getName());
-        // load drivers from system property
-        String driversStr = System.getProperty("distributedlog.namespace.drivers");
-        if (null != driversStr) {
-            String[] driversArray = StringUtils.split(driversStr, ':');
-            for (String driver : driversArray) {
-                driverList.add(driver);
-            }
-        }
-        // initialize the drivers
-        for (String driverClsName : driverList) {
-            try {
-                NamespaceDriver driver =
-                        ReflectionUtils.newInstance(driverClsName, NamespaceDriver.class);
-                NamespaceDriverInfo driverInfo = new NamespaceDriverInfo(driver.getClass());
-                drivers.put(driver.getScheme().toLowerCase(), driverInfo);
-            } catch (Exception ex) {
-                logger.warn("Failed to load namespace driver {} : ", driverClsName, ex);
-            }
-        }
-    }
-
-    /**
-     * Prevent the NamespaceDriverManager class from being instantiated.
-     */
-    private NamespaceDriverManager() {}
-
-    /**
-     * Register the namespace {@code driver}.
-     *
-     * @param driver the namespace driver
-     * @return the namespace driver manager
-     */
-    public static void registerDriver(String backend, Class<? extends NamespaceDriver> driver) {
-        if (!initialized) {
-            initialize();
-        }
-
-        String scheme = backend.toLowerCase();
-        NamespaceDriverInfo oldDriverInfo = drivers.get(scheme);
-        if (null != oldDriverInfo) {
-            return;
-        }
-        NamespaceDriverInfo newDriverInfo = new NamespaceDriverInfo(driver);
-        oldDriverInfo = drivers.putIfAbsent(scheme, newDriverInfo);
-        if (null != oldDriverInfo) {
-            logger.debug("Driver for {} is already there.", scheme);
-        }
-    }
-
-    /**
-     * Retrieve the namespace driver for {@code scheme}.
-     *
-     * @param scheme the scheme for the namespace driver
-     * @return the namespace driver
-     * @throws NullPointerException when scheme is null
-     */
-    public static NamespaceDriver getDriver(String scheme) {
-        checkNotNull(scheme, "Driver Scheme is null");
-        if (!initialized) {
-            initialize();
-        }
-        NamespaceDriverInfo driverInfo = drivers.get(scheme.toLowerCase());
-        if (null == driverInfo) {
-            throw new IllegalArgumentException("Unknown backend " + scheme);
-        }
-        return ReflectionUtils.newInstance(driverInfo.driverClass);
-    }
-
-    /**
-     * Retrieve the namespace driver for {@code uri}.
-     *
-     * @param uri the distributedlog uri
-     * @return the namespace driver for {@code uri}
-     * @throws NullPointerException if the distributedlog {@code uri} is null or doesn't have scheme
-     *          or there is no namespace driver registered for the scheme
-     * @throws IllegalArgumentException if the distributedlog {@code uri} scheme is illegal
-     */
-    public static NamespaceDriver getDriver(URI uri) {
-        // Validate the uri and load the backend according to scheme
-        checkNotNull(uri, "DistributedLog uri is null");
-        String scheme = uri.getScheme();
-        checkNotNull(scheme, "Invalid distributedlog uri : " + uri);
-        scheme = scheme.toLowerCase();
-        String[] schemeParts = StringUtils.split(scheme, '-');
-        checkArgument(schemeParts.length > 0,
-                "Invalid distributedlog scheme found : " + uri);
-        checkArgument(Objects.equal(DistributedLogConstants.SCHEME_PREFIX, schemeParts[0].toLowerCase()),
-                "Unknown distributedlog scheme found : " + uri);
-        // bookkeeper is the default backend
-        String backend = DistributedLogConstants.BACKEND_BK;
-        if (schemeParts.length > 1) {
-            backend = schemeParts[1];
-        }
-        return getDriver(backend);
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceWatcher.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceWatcher.java
deleted file mode 100644
index f836520..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceWatcher.java
+++ /dev/null
@@ -1,61 +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 com.twitter.distributedlog.namespace;
-
-import com.twitter.distributedlog.callback.NamespaceListener;
-
-import java.util.concurrent.CopyOnWriteArraySet;
-
-/**
- * Namespace Watcher watching namespace changes.
- */
-public abstract class NamespaceWatcher {
-
-    protected final CopyOnWriteArraySet<NamespaceListener> listeners =
-            new CopyOnWriteArraySet<NamespaceListener>();
-
-    /**
-     * Register listener for namespace changes.
-     *
-     * @param listener
-     *          listener to add
-     */
-    public void registerListener(NamespaceListener listener) {
-        if (listeners.add(listener)) {
-            watchNamespaceChanges();
-        }
-    }
-
-    /**
-     * Unregister listener from the namespace watcher.
-     *
-     * @param listener
-     *          listener to remove from namespace watcher
-     */
-    public void unregisterListener(NamespaceListener listener) {
-        listeners.remove(listener);
-    }
-
-    /**
-     * Watch the namespace changes. It would be triggered each time
-     * a namspace listener is added. The implementation should handle
-     * this.
-     */
-    protected abstract void watchNamespaceChanges();
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/package-info.java
deleted file mode 100644
index d659f44..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Namespace
- */
-package com.twitter.distributedlog.namespace;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolver.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolver.java
deleted file mode 100644
index 2298faf..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolver.java
+++ /dev/null
@@ -1,113 +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 com.twitter.distributedlog.net;
-
-import org.apache.bookkeeper.net.DNSToSwitchMapping;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * Abstract DNS resolver for bookkeeper ensemble placement.
- */
-public abstract class DNSResolver implements DNSToSwitchMapping {
-    static final Logger LOG = LoggerFactory.getLogger(DNSResolver.class);
-
-    protected final ConcurrentMap<String, String> domainNameToNetworkLocation =
-            new ConcurrentHashMap<String, String>();
-
-    protected final ConcurrentMap<String, String> hostNameToRegion =
-        new ConcurrentHashMap<String, String>();
-
-    /**
-     * Construct the default dns resolver without host-region overrides.
-     */
-    public DNSResolver() {
-        this("");
-    }
-
-    /**
-     * Construct the dns resolver with host-region overrides.
-     * <p>
-     * <i>hostRegionOverrides</i> is a string of pairs of host-region mapping
-     * (host:region) separated by ';'. during dns resolution, the host will be resolved
-     * to override region. example: <i>host1:region1;host2:region2;...</i>
-     *
-     * @param hostRegionOverrides
-     *          pairs of host-region mapping separated by ';'
-     */
-    public DNSResolver(String hostRegionOverrides) {
-        if (StringUtils.isNotBlank(hostRegionOverrides)) {
-            // Host Region Overrides are of the form
-            // HN1:R1;HN2:R2;...
-            String[] overrides = hostRegionOverrides.split(";");
-
-            for (String override : overrides) {
-                String[] parts = override.split(":");
-                if (parts.length != 2) {
-                    LOG.warn("Incorrect override specified", override);
-                } else {
-                    hostNameToRegion.putIfAbsent(parts[0], parts[1]);
-                }
-            }
-        } // otherwise, no overrides were specified
-    }
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public List<String> resolve(List<String> names) {
-        List<String> networkLocations = new ArrayList<String>(names.size());
-        for (String name : names) {
-            networkLocations.add(resolve(name));
-        }
-        return networkLocations;
-    }
-
-    private String resolve(String domainName) {
-        String networkLocation = domainNameToNetworkLocation.get(domainName);
-        if (null == networkLocation) {
-            networkLocation = resolveToNetworkLocation(domainName);
-            domainNameToNetworkLocation.put(domainName, networkLocation);
-        }
-        return networkLocation;
-    }
-
-    /**
-     * Resolve the <code>domainName</code> to its network location.
-     *
-     * @param domainName
-     *          domain name
-     * @return the network location of <i>domainName</i>
-     */
-    protected abstract String resolveToNetworkLocation(String domainName);
-
-    /**
-     * {@inheritDoc}
-     */
-    @Override
-    public void reloadCachedMappings() {
-        domainNameToNetworkLocation.clear();
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRacks.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRacks.java
deleted file mode 100644
index a0298d0..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRacks.java
+++ /dev/null
@@ -1,66 +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 com.twitter.distributedlog.net;
-
-/**
- * Resolve the dns by racks.
- * <p>
- * It resolves domain name like `(region)-(rack)-xxx-xxx.*` to network location
- * `/(region)/(rack)`. If resolution failed, it returns `/default-region/default-rack`.
- * <p>
- * region could be override in <code>hostRegionOverrides</code>. for example, if the
- * host name is <i>regionA-rack1-xx-yyy</i>, it would be resolved to `/regionA/rack1`
- * without any overrides. If the specified overrides is <i>regionA-rack1-xx-yyy:regionB</i>,
- * the resolved network location would be <i>/regionB/rack1</i>.
- * <p>
- * Region overrides provide optimization hits to bookkeeper if two `logical` regions are
- * in same or close locations.
- *
- * @see DNSResolver#DNSResolver(String)
- */
-public class DNSResolverForRacks extends DNSResolver {
-    static final String DEFAULT_RACK = "/default-region/default-rack";
-
-    public DNSResolverForRacks() {
-    }
-
-    public DNSResolverForRacks(String hostRegionOverrides) {
-        super(hostRegionOverrides);
-    }
-
-    @Override
-    protected String resolveToNetworkLocation(String domainName) {
-        String[] parts = domainName.split("\\.");
-        if (parts.length <= 0) {
-            return DEFAULT_RACK;
-        }
-
-        String hostName = parts[0];
-        String[] labels = hostName.split("-");
-        if (labels.length != 4) {
-            return DEFAULT_RACK;
-        }
-
-        String region = hostNameToRegion.get(hostName);
-        if (null == region) {
-            region = labels[0];
-        }
-
-        return String.format("/%s/%s", region, labels[1]);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRows.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRows.java
deleted file mode 100644
index f585640..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRows.java
+++ /dev/null
@@ -1,72 +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 com.twitter.distributedlog.net;
-
-/**
- * Resolve the dns by rows.
- * <p>
- * It resolves domain name like `(region)-(row)xx-xxx-xxx.*` to network location
- * `/(region)/(row)`. If resolution failed, it returns `/default-region/default-row`.
- * <p>
- * region could be override in <code>hostRegionOverrides</code>. for example, if the
- * host name is <i>regionA-row1-xx-yyy</i>, it would be resolved to `/regionA/row1`
- * without any overrides. If the specified overrides is <i>regionA-row1-xx-yyy:regionB</i>,
- * the resolved network location would be <i>/regionB/row1</i>.
- * <p>
- * Region overrides provide optimization hits to bookkeeper if two `logical` regions are
- * in same or close locations.
- *
- * @see DNSResolver#DNSResolver(String)
- */
-public class DNSResolverForRows extends DNSResolver {
-    static final String DEFAULT_ROW = "/default-region/default-row";
-
-    public DNSResolverForRows() {
-    }
-
-    public DNSResolverForRows(String hostRegionOverrides) {
-        super(hostRegionOverrides);
-    }
-
-    @Override
-    protected String resolveToNetworkLocation(String domainName) {
-        String[] parts = domainName.split("\\.");
-        if (parts.length <= 0) {
-            return DEFAULT_ROW;
-        }
-        String hostName = parts[0];
-        String[] labels = hostName.split("-");
-        if (labels.length != 4) {
-            return DEFAULT_ROW;
-        }
-
-        String region = hostNameToRegion.get(hostName);
-        if (null == region) {
-            region = labels[0];
-        }
-
-        final String rack = labels[1];
-
-        if (rack.length() < 2) {
-            // Default to rack name if the rack name format cannot be recognized
-            return String.format("/%s/%s", region, rack);
-        } else {
-            return String.format("/%s/%s", region, rack.substring(0, 2));
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/NetUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/net/NetUtils.java
deleted file mode 100644
index ce0d360..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/NetUtils.java
+++ /dev/null
@@ -1,74 +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 com.twitter.distributedlog.net;
-
-import org.apache.bookkeeper.net.DNSToSwitchMapping;
-
-import java.lang.reflect.Constructor;
-import java.lang.reflect.InvocationTargetException;
-
-/**
- * Utils about network
- */
-public class NetUtils {
-
-    /**
-     * Get the dns resolver from class <code>resolverClassName</code> with optional
-     * <code>hostRegionOverrides</code>.
-     * <p>
-     * It would try to load the class with the constructor with <code>hostRegionOverrides</code>.
-     * If it fails, it would fall back to load the class with default empty constructor.
-     * The interpretion of <code>hostRegionOverrides</code> is up to the implementation.
-     *
-     * @param resolverCls
-     *          resolver class
-     * @param hostRegionOverrides
-     *          host region overrides
-     * @return dns resolver
-     */
-    public static DNSToSwitchMapping getDNSResolver(Class<? extends DNSToSwitchMapping> resolverCls,
-                                                    String hostRegionOverrides) {
-        // first try to construct the dns resolver with overrides
-        Constructor<? extends DNSToSwitchMapping> constructor;
-        Object[] parameters;
-        try {
-            constructor = resolverCls.getDeclaredConstructor(String.class);
-            parameters = new Object[] { hostRegionOverrides };
-        } catch (NoSuchMethodException nsme) {
-            // no constructor with overrides
-            try {
-                constructor = resolverCls.getDeclaredConstructor();
-                parameters = new Object[0];
-            } catch (NoSuchMethodException nsme1) {
-                throw new RuntimeException("Unable to find constructor for dns resolver "
-                        + resolverCls, nsme1);
-            }
-        }
-        constructor.setAccessible(true);
-        try {
-            return constructor.newInstance(parameters);
-        } catch (InstantiationException ie) {
-            throw new RuntimeException("Unable to instantiate dns resolver " + resolverCls, ie);
-        } catch (IllegalAccessException iae) {
-            throw new RuntimeException("Illegal access to dns resolver " + resolverCls, iae);
-        } catch (InvocationTargetException ite) {
-            throw new RuntimeException("Unable to construct dns resolver " + resolverCls, ite);
-        }
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/net/package-info.java
deleted file mode 100644
index 9093fef..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/net/package-info.java
+++ /dev/null
@@ -1,33 +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.
- */
-/**
- * This package contains all the utilities of network.
- *
- * <h2>DNSResolver</h2>
- *
- * DNS resolver is the utility to resolve host name to a string which represents this host's network location.
- * BookKeeper will use such network locations to place ensemble to ensure rack or region diversity to ensure
- * data availability in the case of switch/router/region is down.
- * <p>
- * Available dns resolvers:
- * <ul>
- * <li>{@link com.twitter.distributedlog.net.DNSResolverForRacks}
- * <li>{@link com.twitter.distributedlog.net.DNSResolverForRows}
- * </ul>
- */
-package com.twitter.distributedlog.net;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/package-info.java
deleted file mode 100644
index 4c1fe57..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Protocol & Core of DistributedLog
- */
-package com.twitter.distributedlog;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRate.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRate.java
deleted file mode 100644
index 98eae00..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRate.java
+++ /dev/null
@@ -1,24 +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 com.twitter.distributedlog.rate;
-
-public interface MovingAverageRate {
-    double get();
-    void add(long amount);
-    void inc();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRateFactory.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRateFactory.java
deleted file mode 100644
index a77f753..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRateFactory.java
+++ /dev/null
@@ -1,65 +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 com.twitter.distributedlog.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;
-
-public class MovingAverageRateFactory {
-
-    private static final int DEFAULT_INTERVAL_SECS = 1;
-
-    private final Timer timer;
-    private final TimerTask timerTask;
-    private final CopyOnWriteArrayList<SampledMovingAverageRate> avgs;
-
-    public MovingAverageRateFactory(Timer timer) {
-        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);
-    }
-
-    public MovingAverageRate create(int intervalSecs) {
-        SampledMovingAverageRate avg = new SampledMovingAverageRate(intervalSecs);
-        avgs.add(avg);
-        return avg;
-    }
-
-    public void close() {
-        timerTask.cancel();
-        avgs.clear();
-    }
-
-    private void sampleAll() {
-        for (SampledMovingAverageRate avg : avgs) {
-            avg.sample();
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/SampledMovingAverageRate.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/SampledMovingAverageRate.java
deleted file mode 100644
index a616324..0000000
--- a/distributedlog-core/src/main/java/com/twitter/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 com.twitter.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/com/twitter/distributedlog/readahead/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/readahead/package-info.java
deleted file mode 100644
index 4945133..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/readahead/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * ReadAhead Mechanism for distributedlog streaming reads
- */
-package com.twitter.distributedlog.readahead;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstDLSNNotLessThanSelector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstDLSNNotLessThanSelector.java
deleted file mode 100644
index 443c503..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstDLSNNotLessThanSelector.java
+++ /dev/null
@@ -1,46 +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 com.twitter.distributedlog.selector;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-
-/**
- * Save the first record with a dlsn not less than the dlsn provided.
- */
-public class FirstDLSNNotLessThanSelector implements LogRecordSelector {
-
-    LogRecordWithDLSN result;
-    final DLSN dlsn;
-
-    public FirstDLSNNotLessThanSelector(DLSN dlsn) {
-        this.dlsn = dlsn;
-    }
-
-    @Override
-    public void process(LogRecordWithDLSN record) {
-        if ((record.getDlsn().compareTo(dlsn) >= 0) && (null == result)) {
-            this.result = record;
-        }
-    }
-
-    @Override
-    public LogRecordWithDLSN result() {
-        return this.result;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstRecordSelector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstRecordSelector.java
deleted file mode 100644
index a51210f..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstRecordSelector.java
+++ /dev/null
@@ -1,46 +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 com.twitter.distributedlog.selector;
-
-import com.twitter.distributedlog.LogRecordWithDLSN;
-
-/**
- * Save the first record processed
- */
-public class FirstRecordSelector implements LogRecordSelector {
-
-    final boolean includeControl;
-    LogRecordWithDLSN firstRecord;
-
-    public FirstRecordSelector(boolean includeControl) {
-        this.includeControl = includeControl;
-    }
-
-    @Override
-    public void process(LogRecordWithDLSN record) {
-        if (null == this.firstRecord
-                && (includeControl || !record.isControl())) {
-            this.firstRecord = record;
-        }
-    }
-
-    @Override
-    public LogRecordWithDLSN result() {
-        return this.firstRecord;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstTxIdNotLessThanSelector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstTxIdNotLessThanSelector.java
deleted file mode 100644
index 03c2cbb..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstTxIdNotLessThanSelector.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 com.twitter.distributedlog.selector;
-
-import com.twitter.distributedlog.LogRecordWithDLSN;
-
-/**
- * Save the first record with transaction id not less than the provided transaction id.
- * If all records' transaction id is less than provided transaction id, save the last record.
- */
-public class FirstTxIdNotLessThanSelector implements LogRecordSelector {
-
-    LogRecordWithDLSN result;
-    final long txId;
-    boolean found = false;
-
-    public FirstTxIdNotLessThanSelector(long txId) {
-        this.txId = txId;
-    }
-
-    @Override
-    public void process(LogRecordWithDLSN record) {
-        if (found) {
-            return;
-        }
-        this.result = record;
-        if (record.getTransactionId() >= txId) {
-            found = true;
-        }
-    }
-
-    @Override
-    public LogRecordWithDLSN result() {
-        return this.result;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LastRecordSelector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LastRecordSelector.java
deleted file mode 100644
index 191342c..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LastRecordSelector.java
+++ /dev/null
@@ -1,38 +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 com.twitter.distributedlog.selector;
-
-import com.twitter.distributedlog.LogRecordWithDLSN;
-
-/**
- * Save the last record processed.
- */
-public class LastRecordSelector implements LogRecordSelector {
-
-    LogRecordWithDLSN lastRecord;
-
-    @Override
-    public void process(LogRecordWithDLSN record) {
-        lastRecord = record;
-    }
-
-    @Override
-    public LogRecordWithDLSN result() {
-        return lastRecord;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LogRecordSelector.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LogRecordSelector.java
deleted file mode 100644
index 45d1c49..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LogRecordSelector.java
+++ /dev/null
@@ -1,40 +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 com.twitter.distributedlog.selector;
-
-import com.twitter.distributedlog.LogRecordWithDLSN;
-
-/**
- * Visitor interface to process a set of records, and return some result.
- */
-public interface LogRecordSelector {
-    /**
-     * Process a given <code>record</code>.
-     *
-     * @param record
-     *          log record to process
-     */
-    void process(LogRecordWithDLSN record);
-
-    /**
-     * Returned the selected log record after processing a set of records.
-     *
-     * @return the selected log record.
-     */
-    LogRecordWithDLSN result();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BKExceptionStatsLogger.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BKExceptionStatsLogger.java
deleted file mode 100644
index be71aef..0000000
--- a/distributedlog-core/src/main/java/com/twitter/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 com.twitter.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/com/twitter/distributedlog/stats/BroadCastStatsLogger.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BroadCastStatsLogger.java
deleted file mode 100644
index 10a7011..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BroadCastStatsLogger.java
+++ /dev/null
@@ -1,194 +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 com.twitter.distributedlog.stats;
-
-import com.google.common.base.Preconditions;
-
-import org.apache.bookkeeper.stats.CachingStatsLogger;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.OpStatsData;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Stats Loggers that broadcast stats to multiple {@link StatsLogger}.
- */
-public class BroadCastStatsLogger {
-
-    /**
-     * Create a broadcast stats logger of two stats loggers `<code>first</code>` and
-     * `<code>second</code>`. The returned stats logger doesn't allow registering any
-     * {@link Gauge}.
-     *
-     * @param first
-     *          first stats logger
-     * @param second
-     *          second stats logger
-     * @return broadcast stats logger
-     */
-    public static StatsLogger two(StatsLogger first, StatsLogger second) {
-        return new CachingStatsLogger(new Two(first, second));
-    }
-
-    static class Two implements StatsLogger {
-        protected final StatsLogger first;
-        protected final StatsLogger second;
-
-        private Two(StatsLogger first, StatsLogger second) {
-            super();
-            Preconditions.checkNotNull(first);
-            Preconditions.checkNotNull(second);
-            this.first = first;
-            this.second = second;
-        }
-
-        @Override
-        public OpStatsLogger getOpStatsLogger(final String statName) {
-            final OpStatsLogger firstLogger = first.getOpStatsLogger(statName);
-            final OpStatsLogger secondLogger = second.getOpStatsLogger(statName);
-            return new OpStatsLogger() {
-                @Override
-                public void registerFailedEvent(long l) {
-                    firstLogger.registerFailedEvent(l);
-                    secondLogger.registerFailedEvent(l);
-                }
-
-                @Override
-                public void registerSuccessfulEvent(long l) {
-                    firstLogger.registerSuccessfulEvent(l);
-                    secondLogger.registerSuccessfulEvent(l);
-                }
-
-                @Override
-                public OpStatsData toOpStatsData() {
-                    // Eventually consistent.
-                    return firstLogger.toOpStatsData();
-                }
-
-                @Override
-                public void clear() {
-                    firstLogger.clear();
-                    secondLogger.clear();
-                }
-            };
-        }
-
-        @Override
-        public Counter getCounter(final String statName) {
-            final Counter firstCounter = first.getCounter(statName);
-            final Counter secondCounter = second.getCounter(statName);
-            return new Counter() {
-                @Override
-                public void clear() {
-                    firstCounter.clear();
-                    secondCounter.clear();
-                }
-
-                @Override
-                public void inc() {
-                    firstCounter.inc();
-                    secondCounter.inc();
-                }
-
-                @Override
-                public void dec() {
-                    firstCounter.dec();
-                    secondCounter.dec();
-                }
-
-                @Override
-                public void add(long l) {
-                    firstCounter.add(l);
-                    secondCounter.add(l);
-                }
-
-                @Override
-                public Long get() {
-                    // Eventually consistent.
-                    return firstCounter.get();
-                }
-            };
-        }
-
-        @Override
-        public <T extends Number> void registerGauge(String statName, Gauge<T> gauge) {
-            // Different underlying stats loggers have different semantics wrt. gauge registration.
-            throw new RuntimeException("Cannot register a gauge on BroadCastStatsLogger.Two");
-        }
-
-        @Override
-        public <T extends Number> void unregisterGauge(String statName, Gauge<T> gauge) {
-            // no-op
-        }
-
-        @Override
-        public StatsLogger scope(final String scope) {
-            return new Two(first.scope(scope), second.scope(scope));
-        }
-
-        @Override
-        public void removeScope(String scope, StatsLogger statsLogger) {
-            if (!(statsLogger instanceof Two)) {
-                return;
-            }
-
-            Two another = (Two) statsLogger;
-
-            first.removeScope(scope, another.first);
-            second.removeScope(scope, another.second);
-        }
-    }
-
-    /**
-     * Create a broadcast stats logger of two stats loggers <code>master</code> and <code>slave</code>.
-     * It is similar as {@link #two(StatsLogger, StatsLogger)}, but it allows registering {@link Gauge}s.
-     * The {@link Gauge} will be registered under master.
-     *
-     * @param master
-     *          master stats logger to receive {@link Counter}, {@link OpStatsLogger} and {@link Gauge}.
-     * @param slave
-     *          slave stats logger to receive only {@link Counter} and {@link OpStatsLogger}.
-     * @return broadcast stats logger
-     */
-    public static StatsLogger masterslave(StatsLogger master, StatsLogger slave) {
-        return new CachingStatsLogger(new MasterSlave(master, slave));
-    }
-
-    static class MasterSlave extends Two {
-
-        private MasterSlave(StatsLogger master, StatsLogger slave) {
-            super(master, slave);
-        }
-
-        @Override
-        public <T extends Number> void registerGauge(String statName, Gauge<T> gauge) {
-            first.registerGauge(statName, gauge);
-        }
-
-        @Override
-        public <T extends Number> void unregisterGauge(String statName, Gauge<T> gauge) {
-            first.unregisterGauge(statName, gauge);
-        }
-
-        @Override
-        public StatsLogger scope(String scope) {
-            return new MasterSlave(first.scope(scope), second.scope(scope));
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/OpStatsListener.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/OpStatsListener.java
deleted file mode 100644
index 0432706..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/OpStatsListener.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 com.twitter.distributedlog.stats;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.util.FutureEventListener;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import java.util.concurrent.TimeUnit;
-
-public class OpStatsListener<T> implements FutureEventListener<T> {
-    OpStatsLogger opStatsLogger;
-    Stopwatch stopwatch;
-
-    public OpStatsListener(OpStatsLogger opStatsLogger, Stopwatch stopwatch) {
-        this.opStatsLogger = opStatsLogger;
-        if (null == stopwatch) {
-            this.stopwatch = Stopwatch.createStarted();
-        } else {
-            this.stopwatch = stopwatch;
-        }
-    }
-
-    public OpStatsListener(OpStatsLogger opStatsLogger) {
-        this(opStatsLogger, null);
-    }
-
-    @Override
-    public void onSuccess(T value) {
-        opStatsLogger.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-    }
-
-    @Override
-    public void onFailure(Throwable cause) {
-        opStatsLogger.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-    }
-}
\ No newline at end of file
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionStateStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionStateStore.java
deleted file mode 100644
index 9e4c4f2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionStateStore.java
+++ /dev/null
@@ -1,42 +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 com.twitter.distributedlog.subscription;
-
-import java.io.Closeable;
-
-import scala.runtime.BoxedUnit;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.util.Future;
-
-public interface SubscriptionStateStore extends Closeable {
-    /**
-     * Get the last committed position stored for this subscription
-     *
-     * @return future represents the last commit position
-     */
-    public Future<DLSN> getLastCommitPosition();
-
-    /**
-     * Advances the position associated with the subscriber
-     *
-     * @param newPosition - new commit position
-     * @return future represents the advance result
-     */
-    public Future<BoxedUnit> advanceCommitPosition(DLSN newPosition);
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionsStore.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionsStore.java
deleted file mode 100644
index 27d5c1d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionsStore.java
+++ /dev/null
@@ -1,69 +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 com.twitter.distributedlog.subscription;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.util.Future;
-import scala.runtime.BoxedUnit;
-
-import java.io.Closeable;
-import java.util.Map;
-
-/**
- * Store to manage subscriptions
- */
-public interface SubscriptionsStore extends Closeable {
-
-    /**
-     * Get the last committed position stored for <i>subscriberId</i>.
-     *
-     * @param subscriberId
-     *          subscriber id
-     * @return future representing last committed position.
-     */
-    public 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();
-
-    /**
-     * Advance the last committed position for <i>subscriberId</i>.
-     *
-     * @param subscriberId
-     *          subscriber id.
-     * @param newPosition
-     *          new committed position.
-     * @return future representing advancing result.
-     */
-    public Future<BoxedUnit> advanceCommitPosition(String subscriberId, DLSN newPosition);
-
-    /**
-     * Delete the subscriber <i>subscriberId</i> permanently. Once the subscriber is deleted, all the
-     * data stored under this subscriber will be lost.
-     * @param subscriberId subscriber id
-     * @return future represent success or failure.
-     * 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);
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java
deleted file mode 100644
index 03d70bd..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java
+++ /dev/null
@@ -1,2873 +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 com.twitter.distributedlog.tools;
-
-import java.io.BufferedReader;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileNotFoundException;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.InputStreamReader;
-import java.io.OutputStreamWriter;
-import java.io.PrintWriter;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Enumeration;
-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.SortedMap;
-import java.util.TreeMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-import java.util.regex.Matcher;
-import java.util.regex.Pattern;
-
-import com.google.common.base.Preconditions;
-import com.twitter.distributedlog.BKDistributedLogNamespace;
-import com.twitter.distributedlog.Entry;
-import com.twitter.distributedlog.MetadataAccessor;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.namespace.NamespaceDriver;
-import com.twitter.distributedlog.util.Utils;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.BookKeeperAccessor;
-import org.apache.bookkeeper.client.BookKeeperAdmin;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.client.LedgerMetadata;
-import org.apache.bookkeeper.client.LedgerReader;
-import org.apache.bookkeeper.net.BookieSocketAddress;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
-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.commons.codec.binary.Hex;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.google.common.util.concurrent.RateLimiter;
-import com.twitter.distributedlog.AsyncLogReader;
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.BookKeeperClientBuilder;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.LogReader;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.auditor.DLAuditor;
-import com.twitter.distributedlog.bk.LedgerAllocator;
-import com.twitter.distributedlog.bk.LedgerAllocatorUtils;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.MetadataUpdater;
-import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.util.Await;
-import com.twitter.util.FutureEventListener;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-public class DistributedLogTool extends Tool {
-
-    static final Logger logger = LoggerFactory.getLogger(DistributedLogTool.class);
-
-    static final List<String> EMPTY_LIST = Lists.newArrayList();
-
-    static int compareByCompletionTime(long time1, long time2) {
-        return time1 > time2 ? 1 : (time1 < time2 ? -1 : 0);
-    }
-
-    static final Comparator<LogSegmentMetadata> LOGSEGMENT_COMPARATOR_BY_TIME = new Comparator<LogSegmentMetadata>() {
-        @Override
-        public int compare(LogSegmentMetadata o1, LogSegmentMetadata o2) {
-            if (o1.isInProgress() && o2.isInProgress()) {
-                return compareByCompletionTime(o1.getFirstTxId(), o2.getFirstTxId());
-            } else if (!o1.isInProgress() && !o2.isInProgress()) {
-                return compareByCompletionTime(o1.getCompletionTime(), o2.getCompletionTime());
-            } else if (o1.isInProgress() && !o2.isInProgress()) {
-                return compareByCompletionTime(o1.getFirstTxId(), o2.getCompletionTime());
-            } else {
-                return compareByCompletionTime(o1.getCompletionTime(), o2.getFirstTxId());
-            }
-        }
-    };
-
-    static DLSN parseDLSN(String dlsnStr) throws ParseException {
-        if (dlsnStr.equals("InitialDLSN")) {
-            return DLSN.InitialDLSN;
-        }
-        String[] parts = dlsnStr.split(",");
-        if (parts.length != 3) {
-            throw new ParseException("Invalid dlsn : " + dlsnStr);
-        }
-        try {
-            return new DLSN(Long.parseLong(parts[0]), Long.parseLong(parts[1]), Long.parseLong(parts[2]));
-        } catch (Exception nfe) {
-            throw new ParseException("Invalid dlsn : " + dlsnStr);
-        }
-    }
-
-    /**
-     * Per DL Command, which parses basic options. e.g. uri.
-     */
-    protected abstract static class PerDLCommand extends OptsCommand {
-
-        protected Options options = new Options();
-        protected final DistributedLogConfiguration dlConf;
-        protected URI uri;
-        protected String zkAclId = null;
-        protected boolean force = false;
-        protected DistributedLogNamespace namespace = null;
-
-        protected PerDLCommand(String name, String description) {
-            super(name, description);
-            dlConf = new DistributedLogConfiguration();
-            // Tools are allowed to read old metadata as long as they can interpret it
-            dlConf.setDLLedgerMetadataSkipMinVersionCheck(true);
-            options.addOption("u", "uri", true, "DistributedLog URI");
-            options.addOption("c", "conf", true, "DistributedLog Configuration File");
-            options.addOption("a", "zk-acl-id", true, "Zookeeper ACL ID");
-            options.addOption("f", "force", false, "Force command (no warnings or prompts)");
-        }
-
-        @Override
-        protected int runCmd(CommandLine commandLine) throws Exception {
-            try {
-                parseCommandLine(commandLine);
-            } catch (ParseException pe) {
-                System.err.println("ERROR: failed to parse commandline : '" + pe.getMessage() + "'");
-                printUsage();
-                return -1;
-            }
-            try {
-                return runCmd();
-            } finally {
-                if (null != namespace) {
-                    namespace.close();
-                }
-            }
-        }
-
-        protected abstract int runCmd() throws Exception;
-
-        @Override
-        protected Options getOptions() {
-            return options;
-        }
-
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            if (!cmdline.hasOption("u")) {
-                throw new ParseException("No distributedlog uri provided.");
-            }
-            uri = URI.create(cmdline.getOptionValue("u"));
-            if (cmdline.hasOption("c")) {
-                String configFile = cmdline.getOptionValue("c");
-                try {
-                    dlConf.loadConf(new File(configFile).toURI().toURL());
-                } catch (ConfigurationException e) {
-                    throw new ParseException("Failed to load distributedlog configuration from " + configFile + ".");
-                } catch (MalformedURLException e) {
-                    throw new ParseException("Failed to load distributedlog configuration from " + configFile + ": malformed uri.");
-                }
-            }
-            if (cmdline.hasOption("a")) {
-                zkAclId = cmdline.getOptionValue("a");
-            }
-            if (cmdline.hasOption("f")) {
-                force = true;
-            }
-        }
-
-        protected DistributedLogConfiguration getConf() {
-            return dlConf;
-        }
-
-        protected URI getUri() {
-            return uri;
-        }
-
-        protected void setUri(URI uri) {
-            this.uri = uri;
-        }
-
-        protected String getZkAclId() {
-            return zkAclId;
-        }
-
-        protected void setZkAclId(String zkAclId) {
-            this.zkAclId = zkAclId;
-        }
-
-        protected boolean getForce() {
-            return force;
-        }
-
-        protected void setForce(boolean force) {
-            this.force = force;
-        }
-
-        protected DistributedLogNamespace getNamespace() throws IOException {
-            if (null == this.namespace) {
-                this.namespace = DistributedLogNamespaceBuilder.newBuilder()
-                        .uri(getUri())
-                        .conf(getConf())
-                        .build();
-            }
-            return this.namespace;
-        }
-
-        protected LogSegmentMetadataStore getLogSegmentMetadataStore() throws IOException {
-            return getNamespace()
-                    .getNamespaceDriver()
-                    .getLogStreamMetadataStore(NamespaceDriver.Role.READER)
-                    .getLogSegmentMetadataStore();
-        }
-
-        protected ZooKeeperClient getZooKeeperClient() throws IOException {
-            NamespaceDriver driver = getNamespace().getNamespaceDriver();
-            assert(driver instanceof BKNamespaceDriver);
-            return ((BKNamespaceDriver) driver).getWriterZKC();
-        }
-
-        protected BookKeeperClient getBookKeeperClient() throws IOException {
-            NamespaceDriver driver = getNamespace().getNamespaceDriver();
-            assert(driver instanceof BKNamespaceDriver);
-            return ((BKNamespaceDriver) driver).getReaderBKC();
-        }
-    }
-
-    /**
-     * Base class for simple command with no resource setup requirements.
-     */
-    public abstract static class SimpleCommand extends OptsCommand {
-
-        protected final Options options = new Options();
-
-        SimpleCommand(String name, String description) {
-            super(name, description);
-        }
-
-        @Override
-        protected int runCmd(CommandLine commandLine) throws Exception {
-            try {
-                parseCommandLine(commandLine);
-            } catch (ParseException pe) {
-                System.err.println("ERROR: failed to parse commandline : '" + pe.getMessage() + "'");
-                printUsage();
-                return -1;
-            }
-            return runSimpleCmd();
-        }
-
-        abstract protected int runSimpleCmd() throws Exception;
-
-        abstract protected void parseCommandLine(CommandLine cmdline) throws ParseException;
-
-        @Override
-        protected Options getOptions() {
-            return options;
-        }
-    }
-
-    /**
-     * Per Stream Command, which parse common options for per stream. e.g. stream name.
-     */
-    abstract static class PerStreamCommand extends PerDLCommand {
-
-        protected String streamName;
-
-        protected PerStreamCommand(String name, String description) {
-            super(name, description);
-            options.addOption("s", "stream", true, "Stream Name");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("s")) {
-                throw new ParseException("No stream name provided.");
-            }
-            streamName = cmdline.getOptionValue("s");
-        }
-
-        protected String getStreamName() {
-            return streamName;
-        }
-
-        protected void setStreamName(String streamName) {
-            this.streamName = streamName;
-        }
-    }
-
-    /**
-     * NOTE: we might consider adding a command to 'delete' namespace. The implementation of the namespace
-     *       driver should implement the 'delete' operation.
-     */
-    protected static class DeleteAllocatorPoolCommand extends PerDLCommand {
-
-        int concurrency = 1;
-        String allocationPoolPath = DistributedLogConstants.ALLOCATION_POOL_NODE;
-
-        DeleteAllocatorPoolCommand() {
-            super("delete_allocator_pool", "Delete allocator pool for a given distributedlog instance");
-            options.addOption("t", "concurrency", true, "Concurrency on deleting allocator pool.");
-            options.addOption("ap", "allocation-pool-path", true, "Ledger Allocation Pool Path");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (cmdline.hasOption("t")) {
-                concurrency = Integer.parseInt(cmdline.getOptionValue("t"));
-                if (concurrency <= 0) {
-                    throw new ParseException("Invalid concurrency value : " + concurrency + ": it must be greater or equal to 0.");
-                }
-            }
-            if (cmdline.hasOption("ap")) {
-                allocationPoolPath = cmdline.getOptionValue("ap");
-                if (!allocationPoolPath.startsWith(".") || !allocationPoolPath.contains("allocation")) {
-                    throw new ParseException("Invalid allocation pool path : " + allocationPoolPath + ": it must starts with a '.' and must contains 'allocation'");
-                }
-            }
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            String rootPath = getUri().getPath() + "/" + allocationPoolPath;
-            final ScheduledExecutorService allocationExecutor = Executors.newSingleThreadScheduledExecutor();
-            ExecutorService executorService = Executors.newFixedThreadPool(concurrency);
-            Preconditions.checkArgument(getNamespace() instanceof BKDistributedLogNamespace);
-            BKDistributedLogNamespace bkns = (BKDistributedLogNamespace) getNamespace();
-            final ZooKeeperClient zkc = ((BKNamespaceDriver) bkns.getNamespaceDriver()).getWriterZKC();
-            final BookKeeperClient bkc = ((BKNamespaceDriver) bkns.getNamespaceDriver()).getReaderBKC();
-            try {
-                List<String> pools = zkc.get().getChildren(rootPath, false);
-                final LinkedBlockingQueue<String> poolsToDelete = new LinkedBlockingQueue<String>();
-                if (getForce() || IOUtils.confirmPrompt("Are you sure you want to delete allocator pools : " + pools)) {
-                    for (String pool : pools) {
-                        poolsToDelete.add(rootPath + "/" + pool);
-                    }
-                    final CountDownLatch doneLatch = new CountDownLatch(concurrency);
-                    for (int i = 0; i < concurrency; i++) {
-                        final int tid = i;
-                        executorService.submit(new Runnable() {
-                            @Override
-                            public void run() {
-                                while (!poolsToDelete.isEmpty()) {
-                                    String poolPath = poolsToDelete.poll();
-                                    if (null == poolPath) {
-                                        break;
-                                    }
-                                    try {
-                                        LedgerAllocator allocator =
-                                                LedgerAllocatorUtils.createLedgerAllocatorPool(poolPath, 0, getConf(),
-                                                        zkc, bkc,
-                                                        allocationExecutor);
-                                        allocator.delete();
-                                        System.out.println("Deleted allocator pool : " + poolPath + " .");
-                                    } catch (IOException ioe) {
-                                        System.err.println("Failed to delete allocator pool " + poolPath + " : " + ioe.getMessage());
-                                    }
-                                }
-                                doneLatch.countDown();
-                                System.out.println("Thread " + tid + " is done.");
-                            }
-                        });
-                    }
-                    doneLatch.await();
-                }
-            } finally {
-                executorService.shutdown();
-                allocationExecutor.shutdown();
-            }
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "delete_allocator_pool";
-        }
-    }
-
-    public static class ListCommand extends PerDLCommand {
-
-        boolean printMetadata = false;
-        boolean printHex = false;
-
-        ListCommand() {
-            super("list", "list streams of a given distributedlog instance");
-            options.addOption("m", "meta", false, "Print metadata associated with each stream");
-            options.addOption("x", "hex", false, "Print metadata in hex format");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            printMetadata = cmdline.hasOption("m");
-            printHex = cmdline.hasOption("x");
-        }
-
-        @Override
-        protected String getUsage() {
-            return "list [options]";
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            printStreams(getNamespace());
-            return 0;
-        }
-
-        protected void printStreams(DistributedLogNamespace namespace) throws Exception {
-            Iterator<String> streams = namespace.getLogs();
-            System.out.println("Streams under " + getUri() + " : ");
-            System.out.println("--------------------------------");
-            while (streams.hasNext()) {
-                String streamName = streams.next();
-                System.out.println(streamName);
-                if (!printMetadata) {
-                    continue;
-                }
-                MetadataAccessor accessor =
-                        namespace.getNamespaceDriver().getMetadataAccessor(streamName);
-                byte[] metadata = accessor.getMetadata();
-                if (null == metadata || metadata.length == 0) {
-                    continue;
-                }
-                if (printHex) {
-                    System.out.println(Hex.encodeHexString(metadata));
-                } else {
-                    System.out.println(new String(metadata, UTF_8));
-                }
-                System.out.println("");
-            }
-            System.out.println("--------------------------------");
-        }
-    }
-
-    public static class WatchNamespaceCommand extends PerDLCommand implements NamespaceListener {
-        private Set<String> currentSet = Sets.<String>newHashSet();
-        private CountDownLatch doneLatch = new CountDownLatch(1);
-
-        WatchNamespaceCommand() {
-            super("watch", "watch and report changes for a dl namespace");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-        }
-
-        @Override
-        protected String getUsage() {
-            return "watch [options]";
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            watchAndReportChanges(getNamespace());
-            doneLatch.await();
-            return 0;
-        }
-
-        @Override
-        public synchronized void onStreamsChanged(Iterator<String> streams) {
-            Set<String> updatedSet = Sets.newHashSet(streams);
-            Set<String> oldStreams = Sets.difference(currentSet, updatedSet);
-            Set<String> newStreams = Sets.difference(updatedSet, currentSet);
-            currentSet = updatedSet;
-
-            System.out.println("Old streams : ");
-            for (String stream : oldStreams) {
-                System.out.println(stream);
-            }
-
-            System.out.println("New streams : ");
-            for (String stream : newStreams) {
-                System.out.println(stream);
-            }
-
-            System.out.println("");
-        }
-
-        protected void watchAndReportChanges(DistributedLogNamespace namespace) throws Exception {
-            namespace.registerNamespaceListener(this);
-        }
-    }
-
-    protected static class InspectCommand extends PerDLCommand {
-
-        int numThreads = 1;
-        String streamPrefix = null;
-        boolean printInprogressOnly = false;
-        boolean dumpEntries = false;
-        boolean orderByTime = false;
-        boolean printStreamsOnly = false;
-        boolean checkInprogressOnly = false;
-
-        InspectCommand() {
-            super("inspect", "Inspect streams under a given dl uri to find any potential corruptions");
-            options.addOption("t", "threads", true, "Number threads to do inspection.");
-            options.addOption("ft", "filter", true, "Stream filter by prefix");
-            options.addOption("i", "inprogress", false, "Print inprogress log segments only");
-            options.addOption("d", "dump", false, "Dump entries of inprogress log segments");
-            options.addOption("ot", "orderbytime", false, "Order the log segments by completion time");
-            options.addOption("pso", "print-stream-only", false, "Print streams only");
-            options.addOption("cio", "check-inprogress-only", false, "Check duplicated inprogress only");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (cmdline.hasOption("t")) {
-                numThreads = Integer.parseInt(cmdline.getOptionValue("t"));
-            }
-            if (cmdline.hasOption("ft")) {
-                streamPrefix = cmdline.getOptionValue("ft");
-            }
-            printInprogressOnly = cmdline.hasOption("i");
-            dumpEntries = cmdline.hasOption("d");
-            orderByTime = cmdline.hasOption("ot");
-            printStreamsOnly = cmdline.hasOption("pso");
-            checkInprogressOnly = cmdline.hasOption("cio");
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            SortedMap<String, List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates =
-                    new TreeMap<String, List<Pair<LogSegmentMetadata, List<String>>>>();
-            inspectStreams(corruptedCandidates);
-            System.out.println("Corrupted Candidates : ");
-            if (printStreamsOnly) {
-                System.out.println(corruptedCandidates.keySet());
-                return 0;
-            }
-            for (Map.Entry<String, List<Pair<LogSegmentMetadata, List<String>>>> entry : corruptedCandidates.entrySet()) {
-                System.out.println(entry.getKey() + " : \n");
-                for (Pair<LogSegmentMetadata, List<String>> pair : entry.getValue()) {
-                    System.out.println("\t - " + pair.getLeft());
-                    if (printInprogressOnly && dumpEntries) {
-                        int i = 0;
-                        for (String entryData : pair.getRight()) {
-                            System.out.println("\t" + i + "\t: " + entryData);
-                            ++i;
-                        }
-                    }
-                }
-                System.out.println();
-            }
-            return 0;
-        }
-
-        private void inspectStreams(final SortedMap<String, List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates)
-                throws Exception {
-            Iterator<String> streamCollection = getNamespace().getLogs();
-            final List<String> streams = new ArrayList<String>();
-            while (streamCollection.hasNext()) {
-                String s = streamCollection.next();
-                if (null != streamPrefix) {
-                    if (s.startsWith(streamPrefix)) {
-                        streams.add(s);
-                    }
-                } else {
-                    streams.add(s);
-                }
-            }
-            if (0 == streams.size()) {
-                return;
-            }
-            println("Streams : " + streams);
-            if (!getForce() && !IOUtils.confirmPrompt("Are you sure you want to inspect " + streams.size() + " streams")) {
-                return;
-            }
-            numThreads = Math.min(streams.size(), numThreads);
-            final int numStreamsPerThreads = streams.size() / numThreads;
-            Thread[] threads = new Thread[numThreads];
-            for (int i = 0; i < numThreads; i++) {
-                final int tid = i;
-                threads[i] = new Thread("Inspect-" + i) {
-                    @Override
-                    public void run() {
-                        try {
-                            inspectStreams(streams, tid, numStreamsPerThreads, corruptedCandidates);
-                            System.out.println("Thread " + tid + " finished.");
-                        } catch (Exception e) {
-                            System.err.println("Thread " + tid + " quits with exception : " + e.getMessage());
-                        }
-                    }
-                };
-                threads[i].start();
-            }
-            for (int i = 0; i < numThreads; i++) {
-                threads[i].join();
-            }
-        }
-
-        private void inspectStreams(List<String> streams,
-                                    int tid,
-                                    int numStreamsPerThreads,
-                                    SortedMap<String, List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates)
-                throws Exception {
-            int startIdx = tid * numStreamsPerThreads;
-            int endIdx = Math.min(streams.size(), (tid + 1) * numStreamsPerThreads);
-            for (int i = startIdx; i < endIdx; i++) {
-                String s = streams.get(i);
-                BookKeeperClient bkc = getBookKeeperClient();
-                DistributedLogManager dlm = getNamespace().openLog(s);
-                try {
-                    List<LogSegmentMetadata> segments = dlm.getLogSegments();
-                    if (segments.size() <= 1) {
-                        continue;
-                    }
-                    boolean isCandidate = false;
-                    if (checkInprogressOnly) {
-                        Set<Long> inprogressSeqNos = new HashSet<Long>();
-                        for (LogSegmentMetadata segment : segments) {
-                            if (segment.isInProgress()) {
-                                inprogressSeqNos.add(segment.getLogSegmentSequenceNumber());
-                            }
-                        }
-                        for (LogSegmentMetadata segment : segments) {
-                            if (!segment.isInProgress() && inprogressSeqNos.contains(segment.getLogSegmentSequenceNumber())) {
-                                isCandidate = true;
-                            }
-                        }
-                    } else {
-                        LogSegmentMetadata firstSegment = segments.get(0);
-                        long lastSeqNo = firstSegment.getLogSegmentSequenceNumber();
-
-                        for (int j = 1; j < segments.size(); j++) {
-                            LogSegmentMetadata nextSegment = segments.get(j);
-                            if (lastSeqNo + 1 != nextSegment.getLogSegmentSequenceNumber()) {
-                                isCandidate = true;
-                                break;
-                            }
-                            ++lastSeqNo;
-                        }
-                    }
-                    if (isCandidate) {
-                        if (orderByTime) {
-                            Collections.sort(segments, LOGSEGMENT_COMPARATOR_BY_TIME);
-                        }
-                        List<Pair<LogSegmentMetadata, List<String>>> ledgers =
-                                new ArrayList<Pair<LogSegmentMetadata, List<String>>>();
-                        for (LogSegmentMetadata seg : segments) {
-                            LogSegmentMetadata segment = seg;
-                            List<String> dumpedEntries = new ArrayList<String>();
-                            if (segment.isInProgress()) {
-                                LedgerHandle lh = bkc.get().openLedgerNoRecovery(segment.getLogSegmentId(), BookKeeper.DigestType.CRC32,
-                                                                                 dlConf.getBKDigestPW().getBytes(UTF_8));
-                                try {
-                                    long lac = lh.readLastConfirmed();
-                                    segment = segment.mutator().setLastEntryId(lac).build();
-                                    if (printInprogressOnly && dumpEntries && lac >= 0) {
-                                        Enumeration<LedgerEntry> entries = lh.readEntries(0L, lac);
-                                        while (entries.hasMoreElements()) {
-                                            LedgerEntry entry = entries.nextElement();
-                                            dumpedEntries.add(new String(entry.getEntry(), UTF_8));
-                                        }
-                                    }
-                                } finally {
-                                    lh.close();
-                                }
-                            }
-                            if (printInprogressOnly) {
-                                if (segment.isInProgress()) {
-                                    ledgers.add(Pair.of(segment, dumpedEntries));
-                                }
-                            } else {
-                                ledgers.add(Pair.of(segment, EMPTY_LIST));
-                            }
-                        }
-                        synchronized (corruptedCandidates) {
-                            corruptedCandidates.put(s, ledgers);
-                        }
-                    }
-                } finally {
-                    dlm.close();
-                }
-            }
-        }
-
-        @Override
-        protected String getUsage() {
-            return "inspect [options]";
-        }
-    }
-
-    protected static class TruncateCommand extends PerDLCommand {
-
-        int numThreads = 1;
-        String streamPrefix = null;
-        boolean deleteStream = false;
-
-        TruncateCommand() {
-            super("truncate", "truncate streams under a given dl uri");
-            options.addOption("t", "threads", true, "Number threads to do truncation");
-            options.addOption("ft", "filter", true, "Stream filter by prefix");
-            options.addOption("d", "delete", false, "Delete Stream");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (cmdline.hasOption("t")) {
-                numThreads = Integer.parseInt(cmdline.getOptionValue("t"));
-            }
-            if (cmdline.hasOption("ft")) {
-                streamPrefix = cmdline.getOptionValue("ft");
-            }
-            if (cmdline.hasOption("d")) {
-                deleteStream = true;
-            }
-        }
-
-        @Override
-        protected String getUsage() {
-            return "truncate [options]";
-        }
-
-        protected void setFilter(String filter) {
-            this.streamPrefix = filter;
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            getConf().setZkAclId(getZkAclId());
-            return truncateStreams(getNamespace());
-        }
-
-        private int truncateStreams(final DistributedLogNamespace namespace) throws Exception {
-            Iterator<String> streamCollection = namespace.getLogs();
-            final List<String> streams = new ArrayList<String>();
-            while (streamCollection.hasNext()) {
-                String s = streamCollection.next();
-                if (null != streamPrefix) {
-                    if (s.startsWith(streamPrefix)) {
-                        streams.add(s);
-                    }
-                } else {
-                    streams.add(s);
-                }
-            }
-            if (0 == streams.size()) {
-                return 0;
-            }
-            System.out.println("Streams : " + streams);
-            if (!getForce() && !IOUtils.confirmPrompt("Do you want to truncate " + streams.size() + " streams ?")) {
-                return 0;
-            }
-            numThreads = Math.min(streams.size(), numThreads);
-            final int numStreamsPerThreads = streams.size() / numThreads + 1;
-            Thread[] threads = new Thread[numThreads];
-            for (int i = 0; i < numThreads; i++) {
-                final int tid = i;
-                threads[i] = new Thread("Truncate-" + i) {
-                    @Override
-                    public void run() {
-                        try {
-                            truncateStreams(namespace, streams, tid, numStreamsPerThreads);
-                            System.out.println("Thread " + tid + " finished.");
-                        } catch (IOException e) {
-                            System.err.println("Thread " + tid + " quits with exception : " + e.getMessage());
-                        }
-                    }
-                };
-                threads[i].start();
-            }
-            for (int i = 0; i < numThreads; i++) {
-                threads[i].join();
-            }
-            return 0;
-        }
-
-        private void truncateStreams(DistributedLogNamespace namespace, List<String> streams,
-                                     int tid, int numStreamsPerThreads) throws IOException {
-            int startIdx = tid * numStreamsPerThreads;
-            int endIdx = Math.min(streams.size(), (tid + 1) * numStreamsPerThreads);
-            for (int i = startIdx; i < endIdx; i++) {
-                String s = streams.get(i);
-                DistributedLogManager dlm = namespace.openLog(s);
-                try {
-                    if (deleteStream) {
-                        dlm.delete();
-                    } else {
-                        dlm.purgeLogsOlderThan(Long.MAX_VALUE);
-                    }
-                } finally {
-                    dlm.close();
-                }
-            }
-        }
-    }
-
-    public static class SimpleBookKeeperClient {
-        BookKeeperClient bkc;
-        ZooKeeperClient zkc;
-
-        public SimpleBookKeeperClient(DistributedLogConfiguration conf, URI uri) {
-            try {
-                zkc = ZooKeeperClientBuilder.newBuilder()
-                    .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                    .zkAclId(conf.getZkAclId())
-                    .uri(uri)
-                    .build();
-                BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
-                BKDLConfig.propagateConfiguration(bkdlConfig, conf);
-                bkc = BookKeeperClientBuilder.newBuilder()
-                        .zkc(zkc)
-                        .dlConfig(conf)
-                        .ledgersPath(bkdlConfig.getBkLedgersPath())
-                        .name("dlog")
-                        .build();
-            } catch (Exception e) {
-                close();
-            }
-        }
-        public BookKeeperClient client() {
-            return bkc;
-        }
-        public void close() {
-            if (null != bkc) {
-                bkc.close();
-            }
-            if (null != zkc) {
-                zkc.close();
-            }
-        }
-    }
-
-    protected static class ShowCommand extends PerStreamCommand {
-
-        SimpleBookKeeperClient bkc = null;
-        boolean listSegments = true;
-        boolean listEppStats = false;
-        long firstLid = 0;
-        long lastLid = -1;
-
-        ShowCommand() {
-            super("show", "show metadata of a given stream and list segments");
-            options.addOption("ns", "no-log-segments", false, "Do not list log segment metadata");
-            options.addOption("lp", "placement-stats", false, "Show ensemble placement stats");
-            options.addOption("fl", "first-ledger", true, "First log sement no");
-            options.addOption("ll", "last-ledger", true, "Last log sement no");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (cmdline.hasOption("fl")) {
-                try {
-                    firstLid = Long.parseLong(cmdline.getOptionValue("fl"));
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid ledger id " + cmdline.getOptionValue("fl"));
-                }
-            }
-            if (firstLid < 0) {
-                throw new IllegalArgumentException("Invalid ledger id " + firstLid);
-            }
-            if (cmdline.hasOption("ll")) {
-                try {
-                    lastLid = Long.parseLong(cmdline.getOptionValue("ll"));
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid ledger id " + cmdline.getOptionValue("ll"));
-                }
-            }
-            if (lastLid != -1 && firstLid > lastLid) {
-                throw new IllegalArgumentException("Invalid ledger ids " + firstLid + " " + lastLid);
-            }
-            listSegments = !cmdline.hasOption("ns");
-            listEppStats = cmdline.hasOption("lp");
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
-            try {
-                if (listEppStats) {
-                    bkc = new SimpleBookKeeperClient(getConf(), getUri());
-                }
-                printMetadata(dlm);
-            } finally {
-                dlm.close();
-                if (null != bkc) {
-                    bkc.close();
-                }
-            }
-            return 0;
-        }
-
-        private void printMetadata(DistributedLogManager dlm) throws Exception {
-            printHeader(dlm);
-            if (listSegments) {
-                System.out.println("Ledgers : ");
-                List<LogSegmentMetadata> segments = dlm.getLogSegments();
-                for (LogSegmentMetadata segment : segments) {
-                    if (include(segment)) {
-                        printLedgerRow(segment);
-                    }
-                }
-            }
-        }
-
-        private void printHeader(DistributedLogManager dlm) throws Exception {
-            DLSN firstDlsn = Await.result(dlm.getFirstDLSNAsync());
-            boolean endOfStreamMarked = dlm.isEndOfStreamMarked();
-            DLSN lastDlsn = dlm.getLastDLSN();
-            long firstTxnId = dlm.getFirstTxId();
-            long lastTxnId = dlm.getLastTxId();
-            long recordCount = dlm.getLogRecordCount();
-            String result = String.format("Stream : (firstTxId=%d, lastTxid=%d, firstDlsn=%s, lastDlsn=%s, endOfStreamMarked=%b, recordCount=%d)",
-                firstTxnId, lastTxnId, getDlsnName(firstDlsn), getDlsnName(lastDlsn), endOfStreamMarked, recordCount);
-            System.out.println(result);
-            if (listEppStats) {
-                printEppStatsHeader(dlm);
-            }
-        }
-
-        boolean include(LogSegmentMetadata segment) {
-            return (firstLid <= segment.getLogSegmentSequenceNumber() && (lastLid == -1 || lastLid >= segment.getLogSegmentSequenceNumber()));
-        }
-
-        private void printEppStatsHeader(DistributedLogManager dlm) throws Exception {
-            String label = "Ledger Placement :";
-            System.out.println(label);
-            Map<BookieSocketAddress, Integer> totals = new HashMap<BookieSocketAddress, Integer>();
-            List<LogSegmentMetadata> segments = dlm.getLogSegments();
-            for (LogSegmentMetadata segment : segments) {
-                if (include(segment)) {
-                    merge(totals, getBookieStats(segment));
-                }
-            }
-            List<Map.Entry<BookieSocketAddress, Integer>> entries = new ArrayList<Map.Entry<BookieSocketAddress, Integer>>(totals.entrySet());
-            Collections.sort(entries, new Comparator<Map.Entry<BookieSocketAddress, Integer>>() {
-                @Override
-                public int compare(Map.Entry<BookieSocketAddress, Integer> o1, Map.Entry<BookieSocketAddress, Integer> o2) {
-                    return o2.getValue() - o1.getValue();
-                }
-            });
-            int width = 0;
-            int totalEntries = 0;
-            for (Map.Entry<BookieSocketAddress, Integer> entry : entries) {
-                width = Math.max(width, label.length() + 1 + entry.getKey().toString().length());
-                totalEntries += entry.getValue();
-            }
-            for (Map.Entry<BookieSocketAddress, Integer> entry : entries) {
-                System.out.println(String.format("%"+width+"s\t%6.2f%%\t\t%d", entry.getKey(), entry.getValue()*1.0/totalEntries, entry.getValue()));
-            }
-        }
-
-        private void printLedgerRow(LogSegmentMetadata segment) throws Exception {
-            System.out.println(segment.getLogSegmentSequenceNumber() + "\t: " + segment);
-        }
-
-        private Map<BookieSocketAddress, Integer> getBookieStats(LogSegmentMetadata segment) throws Exception {
-            Map<BookieSocketAddress, Integer> stats = new HashMap<BookieSocketAddress, Integer>();
-            LedgerHandle lh = bkc.client().get().openLedgerNoRecovery(segment.getLogSegmentId(), BookKeeper.DigestType.CRC32,
-                    getConf().getBKDigestPW().getBytes(UTF_8));
-            long eidFirst = 0;
-            for (SortedMap.Entry<Long, ArrayList<BookieSocketAddress>> entry : LedgerReader.bookiesForLedger(lh).entrySet()) {
-                long eidLast = entry.getKey().longValue();
-                long count = eidLast - eidFirst + 1;
-                for (BookieSocketAddress bookie : entry.getValue()) {
-                    merge(stats, bookie, (int) count);
-                }
-                eidFirst = eidLast;
-            }
-            return stats;
-        }
-
-        void merge(Map<BookieSocketAddress, Integer> m, BookieSocketAddress bookie, Integer count) {
-            if (m.containsKey(bookie)) {
-                m.put(bookie, count + m.get(bookie).intValue());
-            } else {
-                m.put(bookie, count);
-            }
-        }
-
-        void merge(Map<BookieSocketAddress, Integer> m1, Map<BookieSocketAddress, Integer> m2) {
-            for (Map.Entry<BookieSocketAddress, Integer> entry : m2.entrySet()) {
-                merge(m1, entry.getKey(), entry.getValue());
-            }
-        }
-
-        String getDlsnName(DLSN dlsn) {
-            if (dlsn.equals(DLSN.InvalidDLSN)) {
-                return "InvalidDLSN";
-            }
-            return dlsn.toString();
-        }
-
-        @Override
-        protected String getUsage() {
-            return "show [options]";
-        }
-    }
-
-    static class CountCommand extends PerStreamCommand {
-
-        DLSN startDLSN = null;
-        DLSN endDLSN = null;
-
-        protected CountCommand() {
-            super("count", "count number records between dlsns");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            String[] args = cmdline.getArgs();
-            if (args.length < 1) {
-                throw new ParseException("Must specify at least start dlsn.");
-            }
-            if (args.length >= 1) {
-                startDLSN = parseDLSN(args[0]);
-            }
-            if (args.length >= 2) {
-                endDLSN = parseDLSN(args[1]);
-            }
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
-            try {
-                long count = 0;
-                if (null == endDLSN) {
-                    count = countToLastRecord(dlm);
-                } else {
-                    count = countFromStartToEnd(dlm);
-                }
-                System.out.println("total is " + count + " records.");
-                return 0;
-            } finally {
-                dlm.close();
-            }
-        }
-
-        int countFromStartToEnd(DistributedLogManager dlm) throws Exception {
-            int count = 0;
-            try {
-                LogReader reader = dlm.getInputStream(startDLSN);
-                try {
-                    LogRecordWithDLSN record = reader.readNext(false);
-                    LogRecordWithDLSN preRecord = record;
-                    System.out.println("first record : " + record);
-                    while (null != record) {
-                        if (record.getDlsn().compareTo(endDLSN) > 0) {
-                            break;
-                        }
-                        ++count;
-                        if (count % 1000 == 0) {
-                            logger.info("read {} records from {}...", count, getStreamName());
-                        }
-                        preRecord = record;
-                        record = reader.readNext(false);
-                    }
-                    System.out.println("last record : " + preRecord);
-                } finally {
-                    reader.close();
-                }
-            } finally {
-                dlm.close();
-            }
-            return count;
-        }
-
-        long countToLastRecord(DistributedLogManager dlm) throws Exception {
-            return Await.result(dlm.getLogRecordCountAsync(startDLSN)).longValue();
-        }
-
-        @Override
-        protected String getUsage() {
-            return "count <start> <end>";
-        }
-    }
-
-    public static class DeleteCommand extends PerStreamCommand {
-
-        protected DeleteCommand() {
-            super("delete", "delete a given stream");
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            getConf().setZkAclId(getZkAclId());
-            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
-            try {
-                dlm.delete();
-            } finally {
-                dlm.close();
-            }
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "delete";
-        }
-    }
-
-    public static class DeleteLedgersCommand extends PerDLCommand {
-
-        private final List<Long> ledgers = new ArrayList<Long>();
-
-        int numThreads = 1;
-
-        protected DeleteLedgersCommand() {
-            super("delete_ledgers", "delete given ledgers");
-            options.addOption("l", "ledgers", true, "List of ledgers, separated by comma");
-            options.addOption("lf", "ledgers-file", true, "File of list of ledgers, each line has a ledger id");
-            options.addOption("t", "concurrency", true, "Number of threads to run deletions");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (cmdline.hasOption("l") && cmdline.hasOption("lf")) {
-                throw new ParseException("Please specify ledgers: either use list or use file only.");
-            }
-            if (!cmdline.hasOption("l") && !cmdline.hasOption("lf")) {
-                throw new ParseException("No ledgers specified. Please specify ledgers either use list or use file only.");
-            }
-            if (cmdline.hasOption("l")) {
-                String ledgersStr = cmdline.getOptionValue("l");
-                String[] ledgerStrs = ledgersStr.split(",");
-                for (String ledgerStr : ledgerStrs) {
-                    ledgers.add(Long.parseLong(ledgerStr));
-                }
-            }
-            if (cmdline.hasOption("lf")) {
-                BufferedReader br = null;
-                try {
-
-                    br = new BufferedReader(new InputStreamReader(
-                            new FileInputStream(new File(cmdline.getOptionValue("lf"))), UTF_8.name()));
-                    String line;
-                    while ((line = br.readLine()) != null) {
-                        ledgers.add(Long.parseLong(line));
-                    }
-                } catch (FileNotFoundException e) {
-                    throw new ParseException("No ledgers file " + cmdline.getOptionValue("lf") + " found.");
-                } catch (IOException e) {
-                    throw new ParseException("Invalid ledgers file " + cmdline.getOptionValue("lf") + " found.");
-                } finally {
-                    if (null != br) {
-                        try {
-                            br.close();
-                        } catch (IOException e) {
-                            // no-op
-                        }
-                    }
-                }
-            }
-            if (cmdline.hasOption("t")) {
-                numThreads = Integer.parseInt(cmdline.getOptionValue("t"));
-            }
-        }
-
-        @Override
-        protected String getUsage() {
-            return "delete_ledgers [options]";
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
-            try {
-                final AtomicInteger numLedgers = new AtomicInteger(0);
-                final CountDownLatch doneLatch = new CountDownLatch(numThreads);
-                final AtomicInteger numFailures = new AtomicInteger(0);
-                final LinkedBlockingQueue<Long> ledgerQueue =
-                        new LinkedBlockingQueue<Long>();
-                ledgerQueue.addAll(ledgers);
-                for (int i = 0; i < numThreads; i++) {
-                    final int tid = i;
-                    executorService.submit(new Runnable() {
-                        @Override
-                        public void run() {
-                            while (true) {
-                                Long ledger = ledgerQueue.poll();
-                                if (null == ledger) {
-                                    break;
-                                }
-                                try {
-                                    getBookKeeperClient().get().deleteLedger(ledger);
-                                    int numLedgersDeleted = numLedgers.incrementAndGet();
-                                    if (numLedgersDeleted % 1000 == 0) {
-                                        System.out.println("Deleted " + numLedgersDeleted + " ledgers.");
-                                    }
-                                } catch (BKException.BKNoSuchLedgerExistsException e) {
-                                    int numLedgersDeleted = numLedgers.incrementAndGet();
-                                    if (numLedgersDeleted % 1000 == 0) {
-                                        System.out.println("Deleted " + numLedgersDeleted + " ledgers.");
-                                    }
-                                } catch (Exception e) {
-                                    numFailures.incrementAndGet();
-                                    break;
-                                }
-                            }
-                            doneLatch.countDown();
-                            System.out.println("Thread " + tid + " quits");
-                        }
-                    });
-                }
-                doneLatch.await();
-                if (numFailures.get() > 0) {
-                    throw new IOException("Encounter " + numFailures.get() + " failures during deleting ledgers");
-                }
-            } finally {
-                executorService.shutdown();
-            }
-            return 0;
-        }
-    }
-
-    public static class CreateCommand extends PerDLCommand {
-
-        final List<String> streams = new ArrayList<String>();
-
-        String streamPrefix = null;
-        String streamExpression = null;
-
-        CreateCommand() {
-            super("create", "create streams under a given namespace");
-            options.addOption("r", "prefix", true, "Prefix of stream name. E.g. 'QuantumLeapTest-'.");
-            options.addOption("e", "expression", true, "Expression to generate stream suffix. " +
-                              "Currently we support range 'x-y', list 'x,y,z' and name 'xyz'");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (cmdline.hasOption("r")) {
-                streamPrefix = cmdline.getOptionValue("r");
-            }
-            if (cmdline.hasOption("e")) {
-                streamExpression = cmdline.getOptionValue("e");
-            }
-            if (null == streamPrefix || null == streamExpression) {
-                throw new ParseException("Please specify stream prefix & expression.");
-            }
-        }
-
-        protected void generateStreams(String streamPrefix, String streamExpression) throws ParseException {
-            // parse the stream expression
-            if (streamExpression.contains("-")) {
-                // a range expression
-                String[] parts = streamExpression.split("-");
-                if (parts.length != 2) {
-                    throw new ParseException("Invalid stream index range : " + streamExpression);
-                }
-                try {
-                    int start = Integer.parseInt(parts[0]);
-                    int end = Integer.parseInt(parts[1]);
-                    if (start > end) {
-                        throw new ParseException("Invalid stream index range : " + streamExpression);
-                    }
-                    for (int i = start; i <= end; i++) {
-                        streams.add(streamPrefix + i);
-                    }
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid stream index range : " + streamExpression);
-                }
-            } else if (streamExpression.contains(",")) {
-                // a list expression
-                String[] parts = streamExpression.split(",");
-                try {
-                    for (String part : parts) {
-                        int idx = Integer.parseInt(part);
-                        streams.add(streamPrefix + idx);
-                    }
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid stream suffix list : " + streamExpression);
-                }
-            } else {
-                streams.add(streamPrefix + streamExpression);
-            }
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            generateStreams(streamPrefix, streamExpression);
-            if (streams.isEmpty()) {
-                System.out.println("Nothing to create.");
-                return 0;
-            }
-            if (!getForce() && !IOUtils.confirmPrompt("You are going to create streams : " + streams)) {
-                return 0;
-            }
-            getConf().setZkAclId(getZkAclId());
-            for (String stream : streams) {
-                getNamespace().createLog(stream);
-            }
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "create [options]";
-        }
-
-        protected void setPrefix(String prefix) {
-            this.streamPrefix = prefix;
-        }
-
-        protected void setExpression(String expression) {
-            this.streamExpression = expression;
-        }
-    }
-
-    protected static class DumpCommand extends PerStreamCommand {
-
-        boolean printHex = false;
-        boolean skipPayload = false;
-        Long fromTxnId = null;
-        DLSN fromDLSN = null;
-        int count = 100;
-
-        DumpCommand() {
-            super("dump", "dump records of a given stream");
-            options.addOption("x", "hex", false, "Print record in hex format");
-            options.addOption("sp", "skip-payload", false, "Skip printing the payload of the record");
-            options.addOption("o", "offset", true, "Txn ID to start dumping.");
-            options.addOption("n", "seqno", true, "Sequence Number to start dumping");
-            options.addOption("e", "eid", true, "Entry ID to start dumping");
-            options.addOption("t", "slot", true, "Slot to start dumping");
-            options.addOption("l", "limit", true, "Number of entries to dump. Default is 100.");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            printHex = cmdline.hasOption("x");
-            skipPayload = cmdline.hasOption("sp");
-            if (cmdline.hasOption("o")) {
-                try {
-                    fromTxnId = Long.parseLong(cmdline.getOptionValue("o"));
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid txn id " + cmdline.getOptionValue("o"));
-                }
-            }
-            if (cmdline.hasOption("l")) {
-                try {
-                    count = Integer.parseInt(cmdline.getOptionValue("l"));
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid count " + cmdline.getOptionValue("l"));
-                }
-                if (count <= 0) {
-                    throw new ParseException("Negative count found : " + count);
-                }
-            }
-            if (cmdline.hasOption("n")) {
-                long seqno;
-                try {
-                    seqno = Long.parseLong(cmdline.getOptionValue("n"));
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid sequence number " + cmdline.getOptionValue("n"));
-                }
-                long eid;
-                if (cmdline.hasOption("e")) {
-                    eid = Long.parseLong(cmdline.getOptionValue("e"));
-                } else {
-                    eid = 0;
-                }
-                long slot;
-                if (cmdline.hasOption("t")) {
-                    slot = Long.parseLong(cmdline.getOptionValue("t"));
-                } else {
-                    slot = 0;
-                }
-                fromDLSN = new DLSN(seqno, eid, slot);
-            }
-            if (null == fromTxnId && null == fromDLSN) {
-                throw new ParseException("No start Txn/DLSN is specified.");
-            }
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
-            long totalCount = dlm.getLogRecordCount();
-            try {
-                AsyncLogReader reader;
-                Object startOffset;
-                try {
-                    DLSN lastDLSN = Await.result(dlm.getLastDLSNAsync());
-                    System.out.println("Last DLSN : " + lastDLSN);
-                    if (null == fromDLSN) {
-                        reader = dlm.getAsyncLogReader(fromTxnId);
-                        startOffset = fromTxnId;
-                    } else {
-                        reader = dlm.getAsyncLogReader(fromDLSN);
-                        startOffset = fromDLSN;
-                    }
-                } catch (LogNotFoundException lee) {
-                    System.out.println("No stream found to dump records.");
-                    return 0;
-                }
-                try {
-                    System.out.println(String.format("Dump records for %s (from = %s, dump count = %d, total records = %d)",
-                            getStreamName(), startOffset, count, totalCount));
-
-                    dumpRecords(reader);
-                } finally {
-                    Utils.close(reader);
-                }
-            } finally {
-                dlm.close();
-            }
-            return 0;
-        }
-
-        private void dumpRecords(AsyncLogReader reader) throws Exception {
-            int numRead = 0;
-            LogRecord record = Await.result(reader.readNext());
-            while (record != null) {
-                // dump the record
-                dumpRecord(record);
-                ++numRead;
-                if (numRead >= count) {
-                    break;
-                }
-                record = Await.result(reader.readNext());
-            }
-            if (numRead == 0) {
-                System.out.println("No records.");
-            } else {
-                System.out.println("------------------------------------------------");
-            }
-        }
-
-        private void dumpRecord(LogRecord record) {
-            System.out.println("------------------------------------------------");
-            if (record instanceof LogRecordWithDLSN) {
-                System.out.println("Record (txn = " + record.getTransactionId() + ", bytes = "
-                        + record.getPayload().length + ", dlsn = "
-                        + ((LogRecordWithDLSN) record).getDlsn() + ", sequence id = "
-                        + ((LogRecordWithDLSN) record).getSequenceId() + ")");
-            } else {
-                System.out.println("Record (txn = " + record.getTransactionId() + ", bytes = "
-                        + record.getPayload().length + ")");
-            }
-            System.out.println("");
-
-            if (skipPayload) {
-                return;
-            }
-
-            if (printHex) {
-                System.out.println(Hex.encodeHexString(record.getPayload()));
-            } else {
-                System.out.println(new String(record.getPayload(), UTF_8));
-            }
-        }
-
-        @Override
-        protected String getUsage() {
-            return "dump [options]";
-        }
-
-        protected void setFromTxnId(Long fromTxnId) {
-            this.fromTxnId = fromTxnId;
-        }
-    }
-
-    /**
-     * TODO: refactor inspect & inspectstream
-     * TODO: support force
-     *
-     * inspectstream -lac -gap (different options for different operations for a single stream)
-     * inspect -lac -gap (inspect the namespace, which will use inspect stream)
-     */
-    static class InspectStreamCommand extends PerStreamCommand {
-
-        InspectStreamCommand() {
-            super("inspectstream", "Inspect a given stream to identify any metadata corruptions");
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
-            try {
-                return inspectAndRepair(dlm.getLogSegments());
-            } finally {
-                dlm.close();
-            }
-        }
-
-        protected int inspectAndRepair(List<LogSegmentMetadata> segments) throws Exception {
-            LogSegmentMetadataStore metadataStore = getLogSegmentMetadataStore();
-            ZooKeeperClient zkc = getZooKeeperClient();
-            BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, getUri());
-            BKDLConfig.propagateConfiguration(bkdlConfig, getConf());
-            BookKeeperClient bkc = BookKeeperClientBuilder.newBuilder()
-                    .dlConfig(getConf())
-                    .zkServers(bkdlConfig.getBkZkServersForReader())
-                    .ledgersPath(bkdlConfig.getBkLedgersPath())
-                    .name("dlog")
-                    .build();
-            try {
-                List<LogSegmentMetadata> segmentsToRepair = inspectLogSegments(bkc, segments);
-                if (segmentsToRepair.isEmpty()) {
-                    System.out.println("The stream is good. No log segments to repair.");
-                    return 0;
-                }
-                System.out.println(segmentsToRepair.size() + " segments to repair : ");
-                System.out.println(segmentsToRepair);
-                System.out.println();
-                if (!IOUtils.confirmPrompt("Do you want to repair them (Y/N): ")) {
-                    return 0;
-                }
-                repairLogSegments(metadataStore, bkc, segmentsToRepair);
-                return 0;
-            } finally {
-                bkc.close();
-            }
-        }
-
-        protected List<LogSegmentMetadata> inspectLogSegments(
-                BookKeeperClient bkc, List<LogSegmentMetadata> segments) throws Exception {
-            List<LogSegmentMetadata> segmentsToRepair = new ArrayList<LogSegmentMetadata>();
-            for (LogSegmentMetadata segment : segments) {
-                if (!segment.isInProgress() && !inspectLogSegment(bkc, segment)) {
-                    segmentsToRepair.add(segment);
-                }
-            }
-            return segmentsToRepair;
-        }
-
-        /**
-         * Inspect a given log segment.
-         *
-         * @param bkc
-         *          bookkeeper client
-         * @param metadata
-         *          metadata of the log segment to
-         * @return true if it is a good stream, false if the stream has inconsistent metadata.
-         * @throws Exception
-         */
-        protected boolean inspectLogSegment(BookKeeperClient bkc,
-                                            LogSegmentMetadata metadata) throws Exception {
-            if (metadata.isInProgress()) {
-                System.out.println("Skip inprogress log segment " + metadata);
-                return true;
-            }
-            long ledgerId = metadata.getLogSegmentId();
-            LedgerHandle lh = bkc.get().openLedger(ledgerId, BookKeeper.DigestType.CRC32,
-                    getConf().getBKDigestPW().getBytes(UTF_8));
-            LedgerHandle readLh = bkc.get().openLedger(ledgerId, BookKeeper.DigestType.CRC32,
-                    getConf().getBKDigestPW().getBytes(UTF_8));
-            LedgerReader lr = new LedgerReader(bkc.get());
-            final AtomicReference<List<LedgerEntry>> entriesHolder = new AtomicReference<List<LedgerEntry>>(null);
-            final AtomicInteger rcHolder = new AtomicInteger(-1234);
-            final CountDownLatch doneLatch = new CountDownLatch(1);
-            try {
-                lr.forwardReadEntriesFromLastConfirmed(readLh, new BookkeeperInternalCallbacks.GenericCallback<List<LedgerEntry>>() {
-                    @Override
-                    public void operationComplete(int rc, List<LedgerEntry> entries) {
-                        rcHolder.set(rc);
-                        entriesHolder.set(entries);
-                        doneLatch.countDown();
-                    }
-                });
-                doneLatch.await();
-                if (BKException.Code.OK != rcHolder.get()) {
-                    throw BKException.create(rcHolder.get());
-                }
-                List<LedgerEntry> entries = entriesHolder.get();
-                long lastEntryId;
-                if (entries.isEmpty()) {
-                    lastEntryId = LedgerHandle.INVALID_ENTRY_ID;
-                } else {
-                    LedgerEntry lastEntry = entries.get(entries.size() - 1);
-                    lastEntryId = lastEntry.getEntryId();
-                }
-                if (lastEntryId != lh.getLastAddConfirmed()) {
-                    System.out.println("Inconsistent Last Add Confirmed Found for LogSegment " + metadata.getLogSegmentSequenceNumber() + ": ");
-                    System.out.println("\t metadata: " + metadata);
-                    System.out.println("\t lac in ledger metadata is " + lh.getLastAddConfirmed() + ", but lac in bookies is " + lastEntryId);
-                    return false;
-                } else {
-                    return true;
-                }
-            } finally {
-                lh.close();
-                readLh.close();
-            }
-        }
-
-        protected void repairLogSegments(LogSegmentMetadataStore metadataStore,
-                                         BookKeeperClient bkc,
-                                         List<LogSegmentMetadata> segments) throws Exception {
-            BookKeeperAdmin bkAdmin = new BookKeeperAdmin(bkc.get());
-            try {
-                MetadataUpdater metadataUpdater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(
-                        getConf(), metadataStore);
-                for (LogSegmentMetadata segment : segments) {
-                    repairLogSegment(bkAdmin, metadataUpdater, segment);
-                }
-            } finally {
-                bkAdmin.close();
-            }
-        }
-
-        protected void repairLogSegment(BookKeeperAdmin bkAdmin,
-                                        MetadataUpdater metadataUpdater,
-                                        LogSegmentMetadata segment) throws Exception {
-            if (segment.isInProgress()) {
-                System.out.println("Skip inprogress log segment " + segment);
-                return;
-            }
-            LedgerHandle lh = bkAdmin.openLedger(segment.getLogSegmentId(), true);
-            long lac = lh.getLastAddConfirmed();
-            Enumeration<LedgerEntry> entries = lh.readEntries(lac, lac);
-            if (!entries.hasMoreElements()) {
-                throw new IOException("Entry " + lac + " isn't found for " + segment);
-            }
-            LedgerEntry lastEntry = entries.nextElement();
-            Entry.Reader reader = Entry.newBuilder()
-                    .setLogSegmentInfo(segment.getLogSegmentSequenceNumber(), segment.getStartSequenceId())
-                    .setEntryId(lastEntry.getEntryId())
-                    .setEnvelopeEntry(LogSegmentMetadata.supportsEnvelopedEntries(segment.getVersion()))
-                    .setInputStream(lastEntry.getEntryInputStream())
-                    .buildReader();
-            LogRecordWithDLSN record = reader.nextRecord();
-            LogRecordWithDLSN lastRecord = null;
-            while (null != record) {
-                lastRecord = record;
-                record = reader.nextRecord();
-            }
-            if (null == lastRecord) {
-                throw new IOException("No record found in entry " + lac + " for " + segment);
-            }
-            System.out.println("Updating last record for " + segment + " to " + lastRecord);
-            if (!IOUtils.confirmPrompt("Do you want to make this change (Y/N): ")) {
-                return;
-            }
-            metadataUpdater.updateLastRecord(segment, lastRecord);
-        }
-
-        @Override
-        protected String getUsage() {
-            return "inspectstream [options]";
-        }
-    }
-
-    static interface BKCommandRunner {
-        int run(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception;
-    }
-
-    abstract static class PerBKCommand extends PerDLCommand {
-
-        protected PerBKCommand(String name, String description) {
-            super(name, description);
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            return runBKCommand(new BKCommandRunner() {
-                @Override
-                public int run(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception {
-                    return runBKCmd(zkc, bkc);
-                }
-            });
-        }
-
-        protected int runBKCommand(BKCommandRunner runner) throws Exception {
-            return runner.run(getZooKeeperClient(), getBookKeeperClient());
-        }
-
-        abstract protected int runBKCmd(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception;
-    }
-
-    static class RecoverCommand extends PerBKCommand {
-
-        final List<Long> ledgers = new ArrayList<Long>();
-        boolean query = false;
-        boolean dryrun = false;
-        boolean skipOpenLedgers = false;
-        boolean fenceOnly = false;
-        int fenceRate = 1;
-        int concurrency = 1;
-        final Set<BookieSocketAddress> bookiesSrc = new HashSet<BookieSocketAddress>();
-        int partition = 0;
-        int numPartitions = 0;
-
-        RecoverCommand() {
-            super("recover", "Recover the ledger data that stored on failed bookies");
-            options.addOption("l", "ledger", true, "Specific ledger to recover");
-            options.addOption("lf", "ledgerfile", true, "File contains ledgers list");
-            options.addOption("q", "query", false, "Query the ledgers that contain given bookies");
-            options.addOption("d", "dryrun", false, "Print the recovery plan w/o actually recovering");
-            options.addOption("cy", "concurrency", true, "Number of ledgers could be recovered in parallel");
-            options.addOption("sk", "skipOpenLedgers", false, "Skip recovering open ledgers");
-            options.addOption("p", "partition", true, "partition");
-            options.addOption("n", "num-partitions", true, "num partitions");
-            options.addOption("fo", "fence-only", true, "fence the ledgers only w/o re-replicating entries");
-            options.addOption("fr", "fence-rate", true, "rate on fencing ledgers");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            query = cmdline.hasOption("q");
-            force = cmdline.hasOption("f");
-            dryrun = cmdline.hasOption("d");
-            skipOpenLedgers = cmdline.hasOption("sk");
-            fenceOnly = cmdline.hasOption("fo");
-            if (cmdline.hasOption("l")) {
-                String[] lidStrs = cmdline.getOptionValue("l").split(",");
-                try {
-                    for (String lidStr : lidStrs) {
-                        ledgers.add(Long.parseLong(lidStr));
-                    }
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid ledger id provided : " + cmdline.getOptionValue("l"));
-                }
-            }
-            if (cmdline.hasOption("lf")) {
-                String file = cmdline.getOptionValue("lf");
-                try {
-                    BufferedReader br = new BufferedReader(
-                            new InputStreamReader(new FileInputStream(file), UTF_8.name()));
-                    try {
-                        String line = br.readLine();
-
-                        while (line != null) {
-                            ledgers.add(Long.parseLong(line));
-                            line = br.readLine();
-                        }
-                    } finally {
-                        br.close();
-                    }
-                } catch (IOException e) {
-                    throw new ParseException("Invalid ledgers file provided : " + file);
-                }
-            }
-            if (cmdline.hasOption("cy")) {
-                try {
-                    concurrency = Integer.parseInt(cmdline.getOptionValue("cy"));
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid concurrency provided : " + cmdline.getOptionValue("cy"));
-                }
-            }
-            if (cmdline.hasOption("p")) {
-                partition = Integer.parseInt(cmdline.getOptionValue("p"));
-            }
-            if (cmdline.hasOption("n")) {
-                numPartitions = Integer.parseInt(cmdline.getOptionValue("n"));
-            }
-            if (cmdline.hasOption("fr")) {
-                fenceRate = Integer.parseInt(cmdline.getOptionValue("fr"));
-            }
-            // Get bookies list to recover
-            String[] args = cmdline.getArgs();
-            final String[] bookieStrs = args[0].split(",");
-            for (String bookieStr : bookieStrs) {
-                final String bookieStrParts[] = bookieStr.split(":");
-                if (bookieStrParts.length != 2) {
-                    throw new ParseException("BookieSrcs has invalid bookie address format (host:port expected) : "
-                            + bookieStr);
-                }
-                try {
-                    bookiesSrc.add(new BookieSocketAddress(bookieStrParts[0],
-                            Integer.parseInt(bookieStrParts[1])));
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid ledger id provided : " + cmdline.getOptionValue("l"));
-                }
-            }
-        }
-
-        @Override
-        protected int runBKCmd(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception {
-            BookKeeperAdmin bkAdmin = new BookKeeperAdmin(bkc.get());
-            try {
-                if (query) {
-                    return bkQuery(bkAdmin, bookiesSrc);
-                }
-                if (fenceOnly) {
-                    return bkFence(bkc, ledgers, fenceRate);
-                }
-                if (!force) {
-                    System.out.println("Bookies : " + bookiesSrc);
-                    if (!IOUtils.confirmPrompt("Do you want to recover them: (Y/N)")) {
-                        return -1;
-                    }
-                }
-                if (!ledgers.isEmpty()) {
-                    System.out.println("Ledgers : " + ledgers);
-                    long numProcessed = 0;
-                    Iterator<Long> ledgersIter = ledgers.iterator();
-                    LinkedBlockingQueue<Long> ledgersToProcess = new LinkedBlockingQueue<Long>();
-                    while (ledgersIter.hasNext()) {
-                        long lid = ledgersIter.next();
-                        if (numPartitions <=0 || (numPartitions > 0 && lid % numPartitions == partition)) {
-                            ledgersToProcess.add(lid);
-                            ++numProcessed;
-                        }
-                        if (ledgersToProcess.size() == 10000) {
-                            System.out.println("Processing " + numProcessed + " ledgers");
-                            bkRecovery(ledgersToProcess, bookiesSrc, dryrun, skipOpenLedgers);
-                            ledgersToProcess.clear();
-                            System.out.println("Processed " + numProcessed + " ledgers");
-                        }
-                    }
-                    if (!ledgersToProcess.isEmpty()) {
-                        System.out.println("Processing " + numProcessed + " ledgers");
-                        bkRecovery(ledgersToProcess, bookiesSrc, dryrun, skipOpenLedgers);
-                        System.out.println("Processed " + numProcessed + " ledgers");
-                    }
-                    System.out.println("Done.");
-                    CountDownLatch latch = new CountDownLatch(1);
-                    latch.await();
-                    return 0;
-                }
-                return bkRecovery(bkAdmin, bookiesSrc, dryrun, skipOpenLedgers);
-            } finally {
-                bkAdmin.close();
-            }
-        }
-
-        private int bkFence(final BookKeeperClient bkc, List<Long> ledgers, int fenceRate) throws Exception {
-            if (ledgers.isEmpty()) {
-                System.out.println("Nothing to fence. Done.");
-                return 0;
-            }
-            ExecutorService executorService = Executors.newCachedThreadPool();
-            final RateLimiter rateLimiter = RateLimiter.create(fenceRate);
-            final byte[] passwd = getConf().getBKDigestPW().getBytes(UTF_8);
-            final CountDownLatch latch = new CountDownLatch(ledgers.size());
-            final AtomicInteger numPendings = new AtomicInteger(ledgers.size());
-            final LinkedBlockingQueue<Long> ledgersQueue = new LinkedBlockingQueue<Long>();
-            ledgersQueue.addAll(ledgers);
-
-            for (int i = 0; i < concurrency; i++) {
-                executorService.submit(new Runnable() {
-                    @Override
-                    public void run() {
-                        while (!ledgersQueue.isEmpty()) {
-                            rateLimiter.acquire();
-                            Long lid = ledgersQueue.poll();
-                            if (null == lid) {
-                                break;
-                            }
-                            System.out.println("Fencing ledger " + lid);
-                            int numRetries = 3;
-                            while (numRetries > 0) {
-                                try {
-                                    LedgerHandle lh = bkc.get().openLedger(lid, BookKeeper.DigestType.CRC32, passwd);
-                                    lh.close();
-                                    System.out.println("Fenced ledger " + lid + ", " + numPendings.decrementAndGet() + " left.");
-                                    latch.countDown();
-                                } catch (BKException.BKNoSuchLedgerExistsException bke) {
-                                    System.out.println("Skipped fence non-exist ledger " + lid + ", " + numPendings.decrementAndGet() + " left.");
-                                    latch.countDown();
-                                } catch (BKException.BKLedgerRecoveryException lre) {
-                                    --numRetries;
-                                    continue;
-                                } catch (Exception e) {
-                                    e.printStackTrace();
-                                    break;
-                                }
-                                numRetries = 0;
-                            }
-                        }
-                        System.out.println("Thread exits");
-                    }
-                });
-            }
-            latch.await();
-            SchedulerUtils.shutdownScheduler(executorService, 2, TimeUnit.MINUTES);
-            return 0;
-        }
-
-        private int bkQuery(BookKeeperAdmin bkAdmin, Set<BookieSocketAddress> bookieAddrs)
-                throws InterruptedException, BKException {
-            SortedMap<Long, LedgerMetadata> ledgersContainBookies =
-                    bkAdmin.getLedgersContainBookies(bookieAddrs);
-            System.err.println("NOTE: Bookies in inspection list are marked with '*'.");
-            for (Map.Entry<Long, LedgerMetadata> ledger : ledgersContainBookies.entrySet()) {
-                System.out.println("ledger " + ledger.getKey() + " : " + ledger.getValue().getState());
-                Map<Long, Integer> numBookiesToReplacePerEnsemble =
-                        inspectLedger(ledger.getValue(), bookieAddrs);
-                System.out.print("summary: [");
-                for (Map.Entry<Long, Integer> entry : numBookiesToReplacePerEnsemble.entrySet()) {
-                    System.out.print(entry.getKey() + "=" + entry.getValue() + ", ");
-                }
-                System.out.println("]");
-                System.out.println();
-            }
-            System.out.println("Done");
-            return 0;
-        }
-
-        private Map<Long, Integer> inspectLedger(LedgerMetadata metadata, Set<BookieSocketAddress> bookiesToInspect) {
-            Map<Long, Integer> numBookiesToReplacePerEnsemble = new TreeMap<Long, Integer>();
-            for (Map.Entry<Long, ArrayList<BookieSocketAddress>> ensemble : metadata.getEnsembles().entrySet()) {
-                ArrayList<BookieSocketAddress> bookieList = ensemble.getValue();
-                System.out.print(ensemble.getKey() + ":\t");
-                int numBookiesToReplace = 0;
-                for (BookieSocketAddress bookie: bookieList) {
-                    System.out.print(bookie.toString());
-                    if (bookiesToInspect.contains(bookie)) {
-                        System.out.print("*");
-                        ++numBookiesToReplace;
-                    } else {
-                        System.out.print(" ");
-                    }
-                    System.out.print(" ");
-                }
-                System.out.println();
-                numBookiesToReplacePerEnsemble.put(ensemble.getKey(), numBookiesToReplace);
-            }
-            return numBookiesToReplacePerEnsemble;
-        }
-
-        private int bkRecovery(final LinkedBlockingQueue<Long> ledgers, final Set<BookieSocketAddress> bookieAddrs,
-                               final boolean dryrun, final boolean skipOpenLedgers)
-                throws Exception {
-            return runBKCommand(new BKCommandRunner() {
-                @Override
-                public int run(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception {
-                    BookKeeperAdmin bkAdmin = new BookKeeperAdmin(bkc.get());
-                    try {
-                        bkRecovery(bkAdmin, ledgers, bookieAddrs, dryrun, skipOpenLedgers);
-                        return 0;
-                    } finally {
-                        bkAdmin.close();
-                    }
-                }
-            });
-        }
-
-        private int bkRecovery(final BookKeeperAdmin bkAdmin, final LinkedBlockingQueue<Long> ledgers,
-                               final Set<BookieSocketAddress> bookieAddrs,
-                               final boolean dryrun, final boolean skipOpenLedgers)
-                throws InterruptedException, BKException {
-            final AtomicInteger numPendings = new AtomicInteger(ledgers.size());
-            final ExecutorService executorService = Executors.newCachedThreadPool();
-            final CountDownLatch doneLatch = new CountDownLatch(concurrency);
-            Runnable r = new Runnable() {
-                @Override
-                public void run() {
-                    while (!ledgers.isEmpty()) {
-                        long lid = -1L;
-                        try {
-                            lid = ledgers.take();
-                            System.out.println("Recovering ledger " + lid);
-                            bkAdmin.recoverBookieData(lid, bookieAddrs, dryrun, skipOpenLedgers);
-                            System.out.println("Recovered ledger completed : " + lid + ", " + numPendings.decrementAndGet() + " left");
-                        } catch (InterruptedException e) {
-                            Thread.currentThread().interrupt();
-                            doneLatch.countDown();
-                            break;
-                        } catch (BKException ke) {
-                            System.out.println("Recovered ledger failed : " + lid + ", rc = " + BKException.getMessage(ke.getCode()));
-                        }
-                    }
-                    doneLatch.countDown();
-                }
-            };
-            for (int i = 0; i < concurrency; i++) {
-                executorService.submit(r);
-            }
-            doneLatch.await();
-            SchedulerUtils.shutdownScheduler(executorService, 2, TimeUnit.MINUTES);
-            return 0;
-        }
-
-        private int bkRecovery(BookKeeperAdmin bkAdmin, Set<BookieSocketAddress> bookieAddrs,
-                               boolean dryrun, boolean skipOpenLedgers)
-                throws InterruptedException, BKException {
-            bkAdmin.recoverBookieData(bookieAddrs, dryrun, skipOpenLedgers);
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "recover [options] <bookiesSrc>";
-        }
-    }
-
-    /**
-     * Per Ledger Command, which parse common options for per ledger. e.g. ledger id.
-     */
-    abstract static class PerLedgerCommand extends PerDLCommand {
-
-        protected long ledgerId;
-
-        protected PerLedgerCommand(String name, String description) {
-            super(name, description);
-            options.addOption("l", "ledger", true, "Ledger ID");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("l")) {
-                throw new ParseException("No ledger provided.");
-            }
-            ledgerId = Long.parseLong(cmdline.getOptionValue("l"));
-        }
-
-        protected long getLedgerID() {
-            return ledgerId;
-        }
-
-        protected void setLedgerId(long ledgerId) {
-            this.ledgerId = ledgerId;
-        }
-    }
-
-    protected static class RecoverLedgerCommand extends PerLedgerCommand {
-
-        RecoverLedgerCommand() {
-            super("recoverledger", "force recover ledger");
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            LedgerHandle lh = getBookKeeperClient().get().openLedgerNoRecovery(
-                    getLedgerID(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes(UTF_8));
-            final CountDownLatch doneLatch = new CountDownLatch(1);
-            final AtomicInteger resultHolder = new AtomicInteger(-1234);
-            BookkeeperInternalCallbacks.GenericCallback<Void> recoverCb =
-                    new BookkeeperInternalCallbacks.GenericCallback<Void>() {
-                @Override
-                public void operationComplete(int rc, Void result) {
-                    resultHolder.set(rc);
-                    doneLatch.countDown();
-                }
-            };
-            try {
-                BookKeeperAccessor.forceRecoverLedger(lh, recoverCb);
-                doneLatch.await();
-                if (BKException.Code.OK != resultHolder.get()) {
-                    throw BKException.create(resultHolder.get());
-                }
-            } finally {
-                lh.close();
-            }
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "recoverledger [options]";
-        }
-    }
-
-    protected static class FindLedgerCommand extends PerLedgerCommand {
-
-        FindLedgerCommand() {
-            super("findledger", "find the stream for a given ledger");
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            Iterator<String> logs = getNamespace().getLogs();
-            while (logs.hasNext()) {
-                String logName = logs.next();
-                if (processLog(logName)) {
-                    System.out.println("Found ledger " + getLedgerID() + " at log stream '" + logName + "'");
-                }
-            }
-            return 0;
-        }
-
-        boolean processLog(String logName) throws Exception {
-            DistributedLogManager dlm = getNamespace().openLog(logName);
-            try {
-                List<LogSegmentMetadata> segments = dlm.getLogSegments();
-                for (LogSegmentMetadata segment : segments) {
-                    if (getLedgerID() == segment.getLogSegmentId()) {
-                        System.out.println("Found ledger " + getLedgerID() + " at log segment "
-                                + segment + " for stream '" + logName + "'");
-                        return true;
-                    }
-                }
-                return false;
-            } finally {
-                dlm.close();
-            }
-        }
-    }
-
-    protected static class ReadLastConfirmedCommand extends PerLedgerCommand {
-
-        ReadLastConfirmedCommand() {
-            super("readlac", "read last add confirmed for a given ledger");
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            LedgerHandle lh = getBookKeeperClient().get().openLedgerNoRecovery(
-                    getLedgerID(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes(UTF_8));
-            try {
-                long lac = lh.readLastConfirmed();
-                System.out.println("LastAddConfirmed: " + lac);
-            } finally {
-                lh.close();
-            }
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "readlac [options]";
-        }
-    }
-
-    protected static class ReadEntriesCommand extends PerLedgerCommand {
-
-        Long fromEntryId;
-        Long untilEntryId;
-        boolean printHex = false;
-        boolean skipPayload = false;
-        boolean readAllBookies = false;
-        boolean readLac = false;
-        boolean corruptOnly = false;
-
-        int metadataVersion = LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION;
-
-        ReadEntriesCommand() {
-            super("readentries", "read entries for a given ledger");
-            options.addOption("x", "hex", false, "Print record in hex format");
-            options.addOption("sp", "skip-payload", false, "Skip printing the payload of the record");
-            options.addOption("fid", "from", true, "Entry id to start reading");
-            options.addOption("uid", "until", true, "Entry id to read until");
-            options.addOption("bks", "all-bookies", false, "Read entry from all bookies");
-            options.addOption("lac", "last-add-confirmed", false, "Return last add confirmed rather than entry payload");
-            options.addOption("ver", "metadata-version", true, "The log segment metadata version to use");
-            options.addOption("bad", "corrupt-only", false, "Display info for corrupt entries only");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            printHex = cmdline.hasOption("x");
-            skipPayload = cmdline.hasOption("sp");
-            if (cmdline.hasOption("fid")) {
-                fromEntryId = Long.parseLong(cmdline.getOptionValue("fid"));
-            }
-            if (cmdline.hasOption("uid")) {
-                untilEntryId = Long.parseLong(cmdline.getOptionValue("uid"));
-            }
-            if (cmdline.hasOption("ver")) {
-                metadataVersion = Integer.parseInt(cmdline.getOptionValue("ver"));
-            }
-            corruptOnly = cmdline.hasOption("bad");
-            readAllBookies = cmdline.hasOption("bks");
-            readLac = cmdline.hasOption("lac");
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            LedgerHandle lh = getBookKeeperClient().get().openLedgerNoRecovery(getLedgerID(), BookKeeper.DigestType.CRC32,
-                    dlConf.getBKDigestPW().getBytes(UTF_8));
-            try {
-                if (null == fromEntryId) {
-                    fromEntryId = 0L;
-                }
-                if (null == untilEntryId) {
-                    untilEntryId = lh.readLastConfirmed();
-                }
-                if (untilEntryId >= fromEntryId) {
-                    if (readAllBookies) {
-                        LedgerReader lr = new LedgerReader(getBookKeeperClient().get());
-                        if (readLac) {
-                            readLacsFromAllBookies(lr, lh, fromEntryId, untilEntryId);
-                        } else {
-                            readEntriesFromAllBookies(lr, lh, fromEntryId, untilEntryId);
-                        }
-                    } else {
-                        simpleReadEntries(lh, fromEntryId, untilEntryId);
-                    }
-                } else {
-                    System.out.println("No entries.");
-                }
-            } finally {
-                lh.close();
-            }
-            return 0;
-        }
-
-        private void readEntriesFromAllBookies(LedgerReader ledgerReader, LedgerHandle lh, long fromEntryId, long untilEntryId)
-                throws Exception {
-            for (long eid = fromEntryId; eid <= untilEntryId; ++eid) {
-                final CountDownLatch doneLatch = new CountDownLatch(1);
-                final AtomicReference<Set<LedgerReader.ReadResult<InputStream>>> resultHolder =
-                        new AtomicReference<Set<LedgerReader.ReadResult<InputStream>>>();
-                ledgerReader.readEntriesFromAllBookies(lh, eid, new BookkeeperInternalCallbacks.GenericCallback<Set<LedgerReader.ReadResult<InputStream>>>() {
-                    @Override
-                    public void operationComplete(int rc, Set<LedgerReader.ReadResult<InputStream>> readResults) {
-                        if (BKException.Code.OK == rc) {
-                            resultHolder.set(readResults);
-                        } else {
-                            resultHolder.set(null);
-                        }
-                        doneLatch.countDown();
-                    }
-                });
-                doneLatch.await();
-                Set<LedgerReader.ReadResult<InputStream>> readResults = resultHolder.get();
-                if (null == readResults) {
-                    throw new IOException("Failed to read entry " + eid);
-                }
-                boolean printHeader = true;
-                for (LedgerReader.ReadResult<InputStream> rr : readResults) {
-                    if (corruptOnly) {
-                        if (BKException.Code.DigestMatchException == rr.getResultCode()) {
-                            if (printHeader) {
-                                System.out.println("\t" + eid + "\t:");
-                                printHeader = false;
-                            }
-                            System.out.println("\tbookie=" + rr.getBookieAddress());
-                            System.out.println("\t-------------------------------");
-                            System.out.println("status = " + BKException.getMessage(rr.getResultCode()));
-                            System.out.println("\t-------------------------------");
-                        }
-                    } else {
-                        if (printHeader) {
-                            System.out.println("\t" + eid + "\t:");
-                            printHeader = false;
-                        }
-                        System.out.println("\tbookie=" + rr.getBookieAddress());
-                        System.out.println("\t-------------------------------");
-                        if (BKException.Code.OK == rr.getResultCode()) {
-                            Entry.Reader reader = Entry.newBuilder()
-                                    .setLogSegmentInfo(lh.getId(), 0L)
-                                    .setEntryId(eid)
-                                    .setInputStream(rr.getValue())
-                                    .setEnvelopeEntry(LogSegmentMetadata.supportsEnvelopedEntries(metadataVersion))
-                                    .buildReader();
-                            printEntry(reader);
-                        } else {
-                            System.out.println("status = " + BKException.getMessage(rr.getResultCode()));
-                        }
-                        System.out.println("\t-------------------------------");
-                    }
-                }
-            }
-        }
-
-        private void readLacsFromAllBookies(LedgerReader ledgerReader, LedgerHandle lh, long fromEntryId, long untilEntryId)
-                throws Exception {
-            for (long eid = fromEntryId; eid <= untilEntryId; ++eid) {
-                final CountDownLatch doneLatch = new CountDownLatch(1);
-                final AtomicReference<Set<LedgerReader.ReadResult<Long>>> resultHolder =
-                        new AtomicReference<Set<LedgerReader.ReadResult<Long>>>();
-                ledgerReader.readLacs(lh, eid, new BookkeeperInternalCallbacks.GenericCallback<Set<LedgerReader.ReadResult<Long>>>() {
-                    @Override
-                    public void operationComplete(int rc, Set<LedgerReader.ReadResult<Long>> readResults) {
-                        if (BKException.Code.OK == rc) {
-                            resultHolder.set(readResults);
-                        } else {
-                            resultHolder.set(null);
-                        }
-                        doneLatch.countDown();
-                    }
-                });
-                doneLatch.await();
-                Set<LedgerReader.ReadResult<Long>> readResults = resultHolder.get();
-                if (null == readResults) {
-                    throw new IOException("Failed to read entry " + eid);
-                }
-                System.out.println("\t" + eid + "\t:");
-                for (LedgerReader.ReadResult<Long> rr : readResults) {
-                    System.out.println("\tbookie=" + rr.getBookieAddress());
-                    System.out.println("\t-------------------------------");
-                    if (BKException.Code.OK == rr.getResultCode()) {
-                        System.out.println("Eid = " + rr.getEntryId() + ", Lac = " + rr.getValue());
-                    } else {
-                        System.out.println("status = " + BKException.getMessage(rr.getResultCode()));
-                    }
-                    System.out.println("\t-------------------------------");
-                }
-            }
-        }
-
-        private void simpleReadEntries(LedgerHandle lh, long fromEntryId, long untilEntryId) throws Exception {
-            Enumeration<LedgerEntry> entries = lh.readEntries(fromEntryId, untilEntryId);
-            long i = fromEntryId;
-            System.out.println("Entries:");
-            while (entries.hasMoreElements()) {
-                LedgerEntry entry = entries.nextElement();
-                System.out.println("\t" + i  + "(eid=" + entry.getEntryId() + ")\t: ");
-                Entry.Reader reader = Entry.newBuilder()
-                        .setLogSegmentInfo(0L, 0L)
-                        .setEntryId(entry.getEntryId())
-                        .setInputStream(entry.getEntryInputStream())
-                        .setEnvelopeEntry(LogSegmentMetadata.supportsEnvelopedEntries(metadataVersion))
-                        .buildReader();
-                printEntry(reader);
-                ++i;
-            }
-        }
-
-        private void printEntry(Entry.Reader reader) throws Exception {
-            LogRecordWithDLSN record = reader.nextRecord();
-            while (null != record) {
-                System.out.println("\t" + record);
-                if (!skipPayload) {
-                    if (printHex) {
-                        System.out.println(Hex.encodeHexString(record.getPayload()));
-                    } else {
-                        System.out.println(new String(record.getPayload(), UTF_8));
-                    }
-                }
-                System.out.println("");
-                record = reader.nextRecord();
-            }
-        }
-
-        @Override
-        protected String getUsage() {
-            return "readentries [options]";
-        }
-    }
-
-    protected static abstract class AuditCommand extends OptsCommand {
-
-        protected final Options options = new Options();
-        protected final DistributedLogConfiguration dlConf;
-        protected final List<URI> uris = new ArrayList<URI>();
-        protected String zkAclId = null;
-        protected boolean force = false;
-
-        protected AuditCommand(String name, String description) {
-            super(name, description);
-            dlConf = new DistributedLogConfiguration();
-            options.addOption("u", "uris", true, "List of distributedlog uris, separated by comma");
-            options.addOption("c", "conf", true, "DistributedLog Configuration File");
-            options.addOption("a", "zk-acl-id", true, "ZooKeeper ACL ID");
-            options.addOption("f", "force", false, "Force command (no warnings or prompts)");
-        }
-
-        @Override
-        protected int runCmd(CommandLine commandLine) throws Exception {
-            try {
-                parseCommandLine(commandLine);
-            } catch (ParseException pe) {
-                System.err.println("ERROR: failed to parse commandline : '" + pe.getMessage() + "'");
-                printUsage();
-                return -1;
-            }
-            return runCmd();
-        }
-
-        protected abstract int runCmd() throws Exception;
-
-        @Override
-        protected Options getOptions() {
-            return options;
-        }
-
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            if (!cmdline.hasOption("u")) {
-                throw new ParseException("No distributedlog uri provided.");
-            }
-            String urisStr = cmdline.getOptionValue("u");
-            for (String uriStr : urisStr.split(",")) {
-                uris.add(URI.create(uriStr));
-            }
-            if (cmdline.hasOption("c")) {
-                String configFile = cmdline.getOptionValue("c");
-                try {
-                    dlConf.loadConf(new File(configFile).toURI().toURL());
-                } catch (ConfigurationException e) {
-                    throw new ParseException("Failed to load distributedlog configuration from " + configFile + ".");
-                } catch (MalformedURLException e) {
-                    throw new ParseException("Failed to load distributedlog configuration from malformed "
-                            + configFile + ".");
-                }
-            }
-            if (cmdline.hasOption("a")) {
-                zkAclId = cmdline.getOptionValue("a");
-            }
-            if (cmdline.hasOption("f")) {
-                force = true;
-            }
-        }
-
-        protected DistributedLogConfiguration getConf() {
-            return dlConf;
-        }
-
-        protected List<URI> getUris() {
-            return uris;
-        }
-
-        protected String getZkAclId() {
-            return zkAclId;
-        }
-
-        protected boolean getForce() {
-            return force;
-        }
-
-    }
-
-    static class AuditLedgersCommand extends AuditCommand {
-
-        String ledgersFilePrefix;
-        final List<List<String>> allocationPaths =
-                new ArrayList<List<String>>();
-
-        AuditLedgersCommand() {
-            super("audit_ledgers", "Audit ledgers between bookkeeper and DL uris");
-            options.addOption("lf", "ledgers-file", true, "Prefix of filename to store ledgers");
-            options.addOption("ap", "allocation-paths", true, "Allocation paths per uri. E.g ap10;ap11,ap20");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (cmdline.hasOption("lf")) {
-                ledgersFilePrefix = cmdline.getOptionValue("lf");
-            } else {
-                throw new ParseException("No file specified to store leak ledgers");
-            }
-            if (cmdline.hasOption("ap")) {
-                String[] aps = cmdline.getOptionValue("ap").split(",");
-                for(String ap : aps) {
-                    List<String> list = new ArrayList<String>();
-                    String[] array = ap.split(";");
-                    Collections.addAll(list, array);
-                    allocationPaths.add(list);
-                }
-            } else {
-                throw new ParseException("No allocation paths provided.");
-            }
-        }
-
-        void dumpLedgers(Set<Long> ledgers, File targetFile) throws Exception {
-            PrintWriter pw = new PrintWriter(new OutputStreamWriter(new FileOutputStream(targetFile), UTF_8.name()));
-            try {
-                for (Long ledger : ledgers) {
-                    pw.println(ledger);
-                }
-            } finally {
-                pw.close();
-            }
-            System.out.println("Dump " + ledgers.size() + " ledgers to file : " + targetFile);
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            if (!getForce() && !IOUtils.confirmPrompt("Do you want to audit uris : "
-                    + getUris() + ", allocation paths = " + allocationPaths)) {
-                return 0;
-            }
-
-            DLAuditor dlAuditor = new DLAuditor(getConf());
-            try {
-                Pair<Set<Long>, Set<Long>> bkdlLedgers = dlAuditor.collectLedgers(getUris(), allocationPaths);
-                dumpLedgers(bkdlLedgers.getLeft(), new File(ledgersFilePrefix + "-bkledgers.txt"));
-                dumpLedgers(bkdlLedgers.getRight(), new File(ledgersFilePrefix + "-dlledgers.txt"));
-                dumpLedgers(Sets.difference(bkdlLedgers.getLeft(), bkdlLedgers.getRight()),
-                            new File(ledgersFilePrefix + "-leakledgers.txt"));
-            } finally {
-                dlAuditor.close();
-            }
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "audit_ledgers [options]";
-        }
-    }
-
-    public static class AuditDLSpaceCommand extends PerDLCommand {
-
-        private String regex = null;
-
-        AuditDLSpaceCommand() {
-            super("audit_dl_space", "Audit stream space usage for a given dl uri");
-            options.addOption("groupByRegex", true, "Group by the result of applying the regex to stream name");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (cmdline.hasOption("groupByRegex")) {
-                regex = cmdline.getOptionValue("groupByRegex");
-            }
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            DLAuditor dlAuditor = new DLAuditor(getConf());
-            try {
-                Map<String, Long> streamSpaceMap = dlAuditor.calculateStreamSpaceUsage(getUri());
-                if (null != regex) {
-                    printGroupByRegexSpaceUsage(streamSpaceMap, regex);
-                } else {
-                    printSpaceUsage(streamSpaceMap);
-                }
-            } finally {
-                dlAuditor.close();
-            }
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "audit_dl_space [options]";
-        }
-
-        private void printSpaceUsage(Map<String, Long> spaceMap) throws Exception {
-            for (Map.Entry<String, Long> entry : spaceMap.entrySet()) {
-                System.out.println(entry.getKey() + "\t" + entry.getValue());
-            }
-        }
-
-        private void printGroupByRegexSpaceUsage(Map<String, Long> streamSpaceMap, String regex) throws Exception {
-            Pattern pattern = Pattern.compile(regex);
-            Map<String, Long> groupedUsageMap = new HashMap<String, Long>();
-            for (Map.Entry<String, Long> entry : streamSpaceMap.entrySet()) {
-                Matcher matcher = pattern.matcher(entry.getKey());
-                String key = entry.getKey();
-                boolean matches = matcher.matches();
-                if (matches) {
-                    key = matcher.group(1);
-                }
-                Long value = entry.getValue();
-                if (groupedUsageMap.containsKey(key)) {
-                    value += groupedUsageMap.get(key);
-                }
-                groupedUsageMap.put(key, value);
-            }
-            printSpaceUsage(groupedUsageMap);
-        }
-    }
-
-    public static class AuditBKSpaceCommand extends PerDLCommand {
-
-        AuditBKSpaceCommand() {
-            super("audit_bk_space", "Audit bk space usage for a given dl uri");
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            DLAuditor dlAuditor = new DLAuditor(getConf());
-            try {
-                long spaceUsage = dlAuditor.calculateLedgerSpaceUsage(uri);
-                System.out.println("bookkeeper ledgers space usage \t " + spaceUsage);
-            } finally {
-                dlAuditor.close();
-            }
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "audit_bk_space [options]";
-        }
-    }
-
-    protected static class TruncateStreamCommand extends PerStreamCommand {
-
-        DLSN dlsn = DLSN.InvalidDLSN;
-
-        TruncateStreamCommand() {
-            super("truncate_stream", "truncate a stream at a specific position");
-            options.addOption("dlsn", true, "Truncate all records older than this dlsn");
-        }
-
-        public void setDlsn(DLSN dlsn) {
-            this.dlsn = dlsn;
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (cmdline.hasOption("dlsn")) {
-                dlsn = parseDLSN(cmdline.getOptionValue("dlsn"));
-            }
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            getConf().setZkAclId(getZkAclId());
-            return truncateStream(getNamespace(), getStreamName(), dlsn);
-        }
-
-        private int truncateStream(final DistributedLogNamespace 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 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))) {
-                            System.out.println("Failed to truncate.");
-                        }
-                        return 0;
-                    } finally {
-                        Utils.close(writer);
-                    }
-                }
-            } catch (Exception ex) {
-                System.err.println("Failed to truncate " + ex);
-                return 1;
-            } finally {
-                dlm.close();
-            }
-        }
-    }
-
-    public static class DeserializeDLSNCommand extends SimpleCommand {
-
-        String base64Dlsn = "";
-
-        DeserializeDLSNCommand() {
-            super("deserialize_dlsn", "Deserialize DLSN");
-            options.addOption("b64", "base64", true, "Base64 encoded dlsn");
-        }
-
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            if (cmdline.hasOption("b64")) {
-                base64Dlsn = cmdline.getOptionValue("b64");
-            } else {
-                throw new IllegalArgumentException("Argument b64 is required");
-            }
-        }
-
-        @Override
-        protected int runSimpleCmd() throws Exception {
-            System.out.println(DLSN.deserialize(base64Dlsn).toString());
-            return 0;
-        }
-    }
-
-    public static class SerializeDLSNCommand extends SimpleCommand {
-
-        private DLSN dlsn = DLSN.InitialDLSN;
-        private boolean hex = false;
-
-        SerializeDLSNCommand() {
-            super("serialize_dlsn", "Serialize DLSN. Default format is base64 string.");
-            options.addOption("dlsn", true, "DLSN in comma separated format to serialize");
-            options.addOption("x", "hex", false, "Emit hex-encoded string DLSN instead of base 64");
-        }
-
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            if (cmdline.hasOption("dlsn")) {
-                dlsn = parseDLSN(cmdline.getOptionValue("dlsn"));
-            }
-            hex = cmdline.hasOption("x");
-        }
-
-        @Override
-        protected int runSimpleCmd() throws Exception {
-            if (hex) {
-                byte[] bytes = dlsn.serializeBytes();
-                String hexString = Hex.encodeHexString(bytes);
-                System.out.println(hexString);
-            } else {
-                System.out.println(dlsn.serialize());
-            }
-            return 0;
-        }
-    }
-
-    public static class DeleteSubscriberCommand extends PerDLCommand {
-
-        int numThreads = 1;
-        String streamPrefix = null;
-        String subscriberId = null;
-
-        DeleteSubscriberCommand() {
-            super("delete_subscriber", "Delete the subscriber in subscription store. ");
-            options.addOption("s", "subscriberId", true, "SubscriberId to remove from the stream");
-            options.addOption("t", "threads", true, "Number of threads");
-            options.addOption("ft", "filter", true, "Stream filter by prefix");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("s")) {
-                throw new ParseException("No subscriberId provided.");
-            } else {
-                subscriberId = cmdline.getOptionValue("s");
-            }
-            if (cmdline.hasOption("t")) {
-                numThreads = Integer.parseInt(cmdline.getOptionValue("t"));
-            }
-            if (cmdline.hasOption("ft")) {
-                streamPrefix = cmdline.getOptionValue("ft");
-            }
-        }
-
-        @Override
-        protected String getUsage() {
-            return "delete_subscriber [options]";
-        }
-
-        @Override
-        protected int runCmd() throws Exception {
-            getConf().setZkAclId(getZkAclId());
-            return deleteSubscriber(getNamespace());
-        }
-
-        private int deleteSubscriber(final DistributedLogNamespace namespace) throws Exception {
-            Iterator<String> streamCollection = namespace.getLogs();
-            final List<String> streams = new ArrayList<String>();
-            while (streamCollection.hasNext()) {
-                String s = streamCollection.next();
-                if (null != streamPrefix) {
-                    if (s.startsWith(streamPrefix)) {
-                        streams.add(s);
-                    }
-                } else {
-                    streams.add(s);
-                }
-            }
-            if (0 == streams.size()) {
-                return 0;
-            }
-            System.out.println("Streams : " + streams);
-            if (!getForce() && !IOUtils.confirmPrompt("Do you want to delete subscriber "
-                + subscriberId + " for " + streams.size() + " streams ?")) {
-                return 0;
-            }
-            numThreads = Math.min(streams.size(), numThreads);
-            final int numStreamsPerThreads = streams.size() / numThreads + 1;
-            Thread[] threads = new Thread[numThreads];
-            for (int i = 0; i < numThreads; i++) {
-                final int tid = i;
-                threads[i] = new Thread("RemoveSubscriberThread-" + i) {
-                    @Override
-                    public void run() {
-                        try {
-                            deleteSubscriber(namespace, streams, tid, numStreamsPerThreads);
-                            System.out.println("Thread " + tid + " finished.");
-                        } catch (Exception e) {
-                            System.err.println("Thread " + tid + " quits with exception : " + e.getMessage());
-                        }
-                    }
-                };
-                threads[i].start();
-            }
-            for (int i = 0; i < numThreads; i++) {
-                threads[i].join();
-            }
-            return 0;
-        }
-
-        private void deleteSubscriber(DistributedLogNamespace namespace, List<String> streams,
-                                      int tid, int numStreamsPerThreads) throws Exception {
-            int startIdx = tid * numStreamsPerThreads;
-            int endIdx = Math.min(streams.size(), (tid + 1) * numStreamsPerThreads);
-            for (int i = startIdx; i < endIdx; i++) {
-                final String s = streams.get(i);
-                DistributedLogManager dlm = namespace.openLog(s);
-                final CountDownLatch countDownLatch = new CountDownLatch(1);
-                dlm.getSubscriptionsStore().deleteSubscriber(subscriberId)
-                    .addEventListener(new FutureEventListener<Boolean>() {
-                        @Override
-                        public void onFailure(Throwable cause) {
-                            System.out.println("Failed to delete subscriber for stream " + s);
-                            cause.printStackTrace();
-                            countDownLatch.countDown();
-                        }
-
-                        @Override
-                        public void onSuccess(Boolean value) {
-                            countDownLatch.countDown();
-                        }
-                    });
-                countDownLatch.await();
-                dlm.close();
-            }
-        }
-    }
-
-    public DistributedLogTool() {
-        super();
-        addCommand(new AuditBKSpaceCommand());
-        addCommand(new AuditLedgersCommand());
-        addCommand(new AuditDLSpaceCommand());
-        addCommand(new CreateCommand());
-        addCommand(new CountCommand());
-        addCommand(new DeleteCommand());
-        addCommand(new DeleteAllocatorPoolCommand());
-        addCommand(new DeleteLedgersCommand());
-        addCommand(new DumpCommand());
-        addCommand(new FindLedgerCommand());
-        addCommand(new InspectCommand());
-        addCommand(new InspectStreamCommand());
-        addCommand(new ListCommand());
-        addCommand(new ReadLastConfirmedCommand());
-        addCommand(new ReadEntriesCommand());
-        addCommand(new RecoverCommand());
-        addCommand(new RecoverLedgerCommand());
-        addCommand(new ShowCommand());
-        addCommand(new TruncateCommand());
-        addCommand(new TruncateStreamCommand());
-        addCommand(new DeserializeDLSNCommand());
-        addCommand(new SerializeDLSNCommand());
-        addCommand(new WatchNamespaceCommand());
-        addCommand(new DeleteSubscriberCommand());
-    }
-
-    @Override
-    protected String getName() {
-        return "dlog_tool";
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/Tool.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/Tool.java
deleted file mode 100644
index bb14066..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/Tool.java
+++ /dev/null
@@ -1,243 +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 com.twitter.distributedlog.tools;
-
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.cli.BasicParser;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-
-import java.util.Map;
-import java.util.TreeMap;
-
-/**
- * A Tool Framework
- */
-public abstract class Tool {
-
-    /**
-     * Interface of a command to run in a tool.
-     */
-    protected interface Command {
-        String getName();
-        String getDescription();
-        int runCmd(String[] args) throws Exception;
-        void printUsage();
-    }
-
-    /**
-     * {@link org.apache.commons.cli.Options} based command.
-     */
-    protected abstract static class OptsCommand implements Command {
-
-        /**
-         * @return options used by this command.
-         */
-        protected abstract Options getOptions();
-
-        /**
-         * @return usage of this command.
-         */
-        protected String getUsage() {
-            return cmdName + " [options]";
-        }
-
-        /**
-         * Run given command line <i>commandLine</i>.
-         *
-         * @param commandLine
-         *          command line to run.
-         * @return return code of this command.
-         * @throws Exception
-         */
-        protected abstract int runCmd(CommandLine commandLine) throws Exception;
-
-        protected String cmdName;
-        protected String description;
-
-        protected OptsCommand(String name, String description) {
-            this.cmdName = name;
-            this.description = description;
-        }
-
-        @Override
-        public String getName() {
-            return cmdName;
-        }
-
-        @Override
-        public String getDescription() {
-            return description;
-        }
-
-        @Override
-        public int runCmd(String[] args) throws Exception {
-            try {
-                BasicParser parser = new BasicParser();
-                CommandLine cmdline = parser.parse(getOptions(), args);
-                return runCmd(cmdline);
-            } catch (ParseException e) {
-                printUsage();
-                return -1;
-            }
-        }
-
-        @Override
-        public void printUsage() {
-            HelpFormatter helpFormatter = new HelpFormatter();
-            println(cmdName + ": " + getDescription());
-            helpFormatter.printHelp(getUsage(), getOptions());
-        }
-    }
-
-    public class HelpCommand implements Command {
-
-        @Override
-        public String getName() {
-            return "help";
-        }
-
-        @Override
-        public String getDescription() {
-            return "describe the usage of this tool or its sub-commands.";
-        }
-
-        @Override
-        public int runCmd(String[] args) throws Exception {
-            if (args.length == 0) {
-                printToolUsage();
-                return -1;
-            }
-            String cmdName = args[0];
-            Command command = commands.get(cmdName);
-            if (null == command) {
-                System.err.println("Unknown command " + cmdName);
-                printToolUsage();
-                return -1;
-            }
-            command.printUsage();
-            println("");
-            return 0;
-        }
-
-        @Override
-        public void printUsage() {
-            println(getName() + ": " + getDescription());
-            println("");
-            println("usage: " + getName() + " <command>");
-        }
-    }
-
-    // Commands managed by a tool
-    protected final Map<String, Command> commands =
-            new TreeMap<String, Command>();
-
-    protected Tool() {
-        addCommand(new HelpCommand());
-    }
-
-    /**
-     * @return tool name.
-     */
-    protected abstract String getName();
-
-    /**
-     * Add a command in this tool.
-     *
-     * @param command
-     *          command to run in this tool.
-     */
-    protected void addCommand(Command command) {
-        commands.put(command.getName(), command);
-    }
-
-    /**
-     * Print a message in this tool.
-     *
-     * @param msg
-     *          message to print
-     */
-    protected static void println(String msg) {
-        System.out.println(msg);
-    }
-
-    /**
-     * print tool usage.
-     */
-    protected void printToolUsage() {
-        println("Usage: " + getName() + " <command>");
-        println("");
-        int maxKeyLength = 0;
-        for (String key : commands.keySet()) {
-            if (key.length() > maxKeyLength) {
-                maxKeyLength = key.length();
-            }
-        }
-        maxKeyLength += 2;
-        for (Map.Entry<String, Command> entry : commands.entrySet()) {
-            StringBuilder spacesBuilder = new StringBuilder();
-            int numSpaces = maxKeyLength - entry.getKey().length();
-            for (int i = 0; i < numSpaces; i++) {
-                spacesBuilder.append(" ");
-            }
-            println("\t"  + entry.getKey() + spacesBuilder.toString() + ": " + entry.getValue().getDescription());
-        }
-        println("");
-    }
-
-    public int run(String[] args) throws Exception {
-        if (args.length <= 0) {
-            printToolUsage();
-            return -1;
-        }
-        String cmdName = args[0];
-        Command cmd = commands.get(cmdName);
-        if (null == cmd) {
-            System.err.println("ERROR: Unknown command " + cmdName);
-            printToolUsage();
-            return -1;
-        }
-        // prepare new args
-        String[] newArgs = new String[args.length - 1];
-        System.arraycopy(args, 1, newArgs, 0, newArgs.length);
-        return cmd.runCmd(newArgs);
-    }
-
-    public static void main(String args[]) {
-        int rc = -1;
-        if (args.length <= 0) {
-            System.err.println("No tool to run.");
-            System.err.println("");
-            System.err.println("Usage : Tool <tool_class_name> <options>");
-            System.exit(-1);
-        }
-        String toolClass = args[0];
-        try {
-            Tool tool = ReflectionUtils.newInstance(toolClass, Tool.class);
-            String[] newArgs = new String[args.length - 1];
-            System.arraycopy(args, 1, newArgs, 0, newArgs.length);
-            rc = tool.run(newArgs);
-        } catch (Throwable t) {
-            System.err.println("Fail to run tool " + toolClass + " : ");
-            t.printStackTrace();
-        }
-        System.exit(rc);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/package-info.java
deleted file mode 100644
index e2125bc..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Tools for distributedlog
- */
-package com.twitter.distributedlog.tools;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Allocator.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Allocator.java
deleted file mode 100644
index dcc3f58..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Allocator.java
+++ /dev/null
@@ -1,102 +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 com.twitter.distributedlog.util;
-
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.io.AsyncDeleteable;
-import com.twitter.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>.
- *
- * <h3>Usage Example</h3>
- *
- * Here is an example on demonstrating how `Allocator` works.
- *
- * <pre> {@code
- * Allocator<I, T, R> allocator = ...;
- *
- * // issue an allocate request
- * try {
- *   allocator.allocate();
- * } catch (IOException ioe) {
- *   // handle the exception
- *   ...
- *   return;
- * }
- *
- * // Start a transaction
- * final Transaction<T> txn = ...;
- *
- * // Try obtain object I
- * Future<I> tryObtainFuture = allocator.tryObtain(txn, new OpListener<I>() {
- *     public void onCommit(I resource) {
- *         // the obtain succeed, process with the resource
- *     }
- *     public void onAbort() {
- *         // the obtain failed.
- *     }
- * }).addFutureEventListener(new FutureEventListener() {
- *     public void onSuccess(I resource) {
- *         // the try obtain succeed. but the obtain has not been confirmed or aborted.
- *         // execute the transaction to confirm if it could complete obtain
- *         txn.execute();
- *     }
- *     public void onFailure(Throwable t) {
- *         // handle the failure of try obtain
- *     }
- * });
- *
- * }</pre>
- */
-public interface Allocator<I, T> extends AsyncCloseable, AsyncDeleteable {
-
-    /**
-     * Issue allocation request to allocate <i>I</i>.
-     * The implementation should be non-blocking call.
-     *
-     * @throws IOException
-     *          if fail to request allocating a <i>I</i>.
-     */
-    void allocate() throws IOException;
-
-    /**
-     * Try obtaining an <i>I</i> in a given transaction <i>T</i>. The object obtained is tentative.
-     * Whether the object is obtained or aborted is determined by the result of the execution. You could
-     * register a listener under this `tryObtain` operation to know whether the object is obtained or
-     * aborted.
-     *
-     * <p>
-     * It is a typical two-phases operation on obtaining a resource from allocator.
-     * The future returned by this method acts as a `prepare` operation, the resource is tentative obtained
-     * from the allocator. The execution of the txn acts as a `commit` operation, the resource is confirmed
-     * to be obtained by this transaction. <code>listener</code> is for the whole completion of the obtain.
-     * <p>
-     * <code>listener</code> is only triggered after `prepare` succeed. if `prepare` failed, no actions will
-     * happen to the listener.
-     *
-     * @param txn
-     *          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);
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/CommandLineUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/CommandLineUtils.java
deleted file mode 100644
index 95ef3e2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/CommandLineUtils.java
+++ /dev/null
@@ -1,56 +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 com.twitter.distributedlog.util;
-
-import com.google.common.base.Optional;
-import org.apache.commons.cli.CommandLine;
-
-/**
- * Utils to commandline
- */
-public class CommandLineUtils {
-
-    public static Optional<String> getOptionalStringArg(CommandLine cmdline, String arg) {
-        if (cmdline.hasOption(arg)) {
-            return Optional.of(cmdline.getOptionValue(arg));
-        } else {
-            return Optional.absent();
-        }
-    }
-
-    public static Optional<Boolean> getOptionalBooleanArg(CommandLine cmdline, String arg) {
-        if (cmdline.hasOption(arg)) {
-            return Optional.of(true);
-        } else {
-            return Optional.absent();
-        }
-    }
-
-    public static Optional<Integer> getOptionalIntegerArg(CommandLine cmdline, String arg) throws IllegalArgumentException {
-        try {
-            if (cmdline.hasOption(arg)) {
-                return Optional.of(Integer.parseInt(cmdline.getOptionValue(arg)));
-            } else {
-                return Optional.absent();
-            }
-        } catch (NumberFormatException ex) {
-            throw new IllegalArgumentException(arg + " is not a number");
-        }
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/ConfUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/ConfUtils.java
deleted file mode 100644
index 46dd3b6..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/ConfUtils.java
+++ /dev/null
@@ -1,65 +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 com.twitter.distributedlog.util;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.config.ConcurrentConstConfiguration;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import org.apache.commons.configuration.Configuration;
-
-import java.util.Iterator;
-
-public class ConfUtils {
-
-    /**
-     * Load configurations with prefixed <i>section</i> from source configuration <i>srcConf</i> into
-     * target configuration <i>targetConf</i>.
-     *
-     * @param targetConf
-     *          Target Configuration
-     * @param srcConf
-     *          Source Configuration
-     * @param section
-     *          Section Key
-     */
-    public static void loadConfiguration(Configuration targetConf, Configuration srcConf, String section) {
-        Iterator confKeys = srcConf.getKeys();
-        while (confKeys.hasNext()) {
-            Object keyObject = confKeys.next();
-            if (!(keyObject instanceof String)) {
-                continue;
-            }
-            String key = (String) keyObject;
-            if (key.startsWith(section)) {
-                targetConf.setProperty(key.substring(section.length()), srcConf.getProperty(key));
-            }
-        }
-    }
-
-    /**
-     * Create const dynamic configuration based on distributedlog configuration.
-     *
-     * @param conf
-     *          static distributedlog configuration.
-     * @return dynamic configuration
-     */
-    public static DynamicDistributedLogConfiguration getConstDynConf(DistributedLogConfiguration conf) {
-        ConcurrentConstConfiguration constConf = new ConcurrentConstConfiguration(conf);
-        return new DynamicDistributedLogConfiguration(constConf);
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/DLUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/DLUtils.java
deleted file mode 100644
index 2f9e091..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/DLUtils.java
+++ /dev/null
@@ -1,321 +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 com.twitter.distributedlog.util;
-
-import com.google.common.base.Objects;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import org.apache.commons.lang.StringUtils;
-
-import java.net.InetAddress;
-import java.net.URI;
-import java.net.URISyntaxException;
-import java.util.List;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-/**
- * Utilities about DL implementations like uri, log segments, metadata serialization and deserialization.
- */
-public class DLUtils {
-
-    /**
-     * Find the log segment whose transaction ids are not less than provided <code>transactionId</code>.
-     *
-     * @param segments
-     *          segments to search
-     * @param transactionId
-     *          transaction id to find
-     * @return the first log segment whose transaction ids are not less than <code>transactionId</code>.
-     */
-    public static int findLogSegmentNotLessThanTxnId(List<LogSegmentMetadata> segments,
-                                                     long transactionId) {
-        int found = -1;
-        for (int i = segments.size() - 1; i >= 0; i--) {
-            LogSegmentMetadata segment = segments.get(i);
-            if (segment.getFirstTxId() <= transactionId) {
-                found = i;
-                break;
-            }
-        }
-        if (found <= -1) {
-            return -1;
-        }
-        if (found == 0 && segments.get(0).getFirstTxId() == transactionId) {
-            return 0;
-        }
-        LogSegmentMetadata foundSegment = segments.get(found);
-        if (foundSegment.getFirstTxId() == transactionId) {
-            for (int i = found - 1; i >= 0; i--) {
-                LogSegmentMetadata segment = segments.get(i);
-                if (segment.isInProgress()) {
-                    break;
-                }
-                if (segment.getLastTxId() < transactionId) {
-                    break;
-                }
-                found = i;
-            }
-            return found;
-        } else {
-            if (foundSegment.isInProgress()
-                    || found == segments.size() - 1) {
-                return found;
-            }
-            if (foundSegment.getLastTxId() >= transactionId) {
-                return found;
-            }
-            return found + 1;
-        }
-    }
-
-    /**
-     * Assign next log segment sequence number based on a decreasing list of log segments.
-     *
-     * @param segmentListDesc
-     *          a decreasing list of log segments
-     * @return null if no log segments was assigned a sequence number in <code>segmentListDesc</code>.
-     *         otherwise, return next log segment sequence number
-     */
-    public static Long nextLogSegmentSequenceNumber(List<LogSegmentMetadata> segmentListDesc) {
-        int lastAssignedLogSegmentIdx = -1;
-        Long lastAssignedLogSegmentSeqNo = null;
-        Long nextLogSegmentSeqNo = null;
-
-        for (int i = 0; i < segmentListDesc.size(); i++) {
-            LogSegmentMetadata metadata = segmentListDesc.get(i);
-            if (LogSegmentMetadata.supportsLogSegmentSequenceNo(metadata.getVersion())) {
-                lastAssignedLogSegmentSeqNo = metadata.getLogSegmentSequenceNumber();
-                lastAssignedLogSegmentIdx = i;
-                break;
-            }
-        }
-
-        if (null != lastAssignedLogSegmentSeqNo) {
-            // latest log segment is assigned with a sequence number, start with next sequence number
-            nextLogSegmentSeqNo = lastAssignedLogSegmentSeqNo + lastAssignedLogSegmentIdx + 1;
-        }
-        return nextLogSegmentSeqNo;
-    }
-
-    /**
-     * Compute the start sequence id for <code>segment</code>, based on previous segment list
-     * <code>segmentListDesc</code>.
-     *
-     * @param logSegmentDescList
-     *          list of segments in descending order
-     * @param segment
-     *          segment to compute start sequence id for
-     * @return start sequence id
-     */
-    public static long computeStartSequenceId(List<LogSegmentMetadata> logSegmentDescList,
-                                              LogSegmentMetadata segment)
-            throws UnexpectedException {
-        long startSequenceId = 0L;
-        for (LogSegmentMetadata metadata : logSegmentDescList) {
-            if (metadata.getLogSegmentSequenceNumber() >= segment.getLogSegmentSequenceNumber()) {
-                continue;
-            } else if (metadata.getLogSegmentSequenceNumber() < (segment.getLogSegmentSequenceNumber() - 1)) {
-                break;
-            }
-            if (metadata.isInProgress()) {
-                throw new UnexpectedException("Should not complete log segment " + segment.getLogSegmentSequenceNumber()
-                        + " since it's previous log segment is still inprogress : " + logSegmentDescList);
-            }
-            if (metadata.supportsSequenceId()) {
-                startSequenceId = metadata.getStartSequenceId() + metadata.getRecordCount();
-            }
-        }
-        return startSequenceId;
-    }
-
-    /**
-     * Deserialize log segment sequence number for bytes <code>data</code>.
-     *
-     * @param data
-     *          byte representation of log segment sequence number
-     * @return log segment sequence number
-     * @throws NumberFormatException if the bytes aren't valid
-     */
-    public static long deserializeLogSegmentSequenceNumber(byte[] data) {
-        String seqNoStr = new String(data, UTF_8);
-        return Long.parseLong(seqNoStr);
-    }
-
-    /**
-     * Serilize log segment sequence number <code>logSegmentSeqNo</code> into bytes.
-     *
-     * @param logSegmentSeqNo
-     *          log segment sequence number
-     * @return byte representation of log segment sequence number
-     */
-    public static byte[] serializeLogSegmentSequenceNumber(long logSegmentSeqNo) {
-        return Long.toString(logSegmentSeqNo).getBytes(UTF_8);
-    }
-
-    /**
-     * Deserialize log record transaction id for bytes <code>data</code>.
-     *
-     * @param data
-     *          byte representation of log record transaction id
-     * @return log record transaction id
-     * @throws NumberFormatException if the bytes aren't valid
-     */
-    public static long deserializeTransactionId(byte[] data) {
-        String seqNoStr = new String(data, UTF_8);
-        return Long.parseLong(seqNoStr);
-    }
-
-    /**
-     * Serilize log record transaction id <code>transactionId</code> into bytes.
-     *
-     * @param transactionId
-     *          log record transaction id
-     * @return byte representation of log record transaction id.
-     */
-    public static byte[] serializeTransactionId(long transactionId) {
-        return Long.toString(transactionId).getBytes(UTF_8);
-    }
-
-    /**
-     * Serialize log segment id into bytes.
-     *
-     * @param logSegmentId
-     *          log segment id
-     * @return bytes representation of log segment id
-     */
-    public static byte[] logSegmentId2Bytes(long logSegmentId) {
-        return Long.toString(logSegmentId).getBytes(UTF_8);
-    }
-
-    /**
-     * Deserialize bytes into log segment id.
-     *
-     * @param data
-     *          bytes representation of log segment id
-     * @return log segment id
-     */
-    public static long bytes2LogSegmentId(byte[] data) {
-        return Long.parseLong(new String(data, UTF_8));
-    }
-
-    /**
-     * Normalize the uri.
-     *
-     * @param uri the distributedlog uri.
-     * @return the normalized uri
-     */
-    public static URI normalizeURI(URI uri) {
-        checkNotNull(uri, "DistributedLog uri is null");
-        String scheme = uri.getScheme();
-        checkNotNull(scheme, "Invalid distributedlog uri : " + uri);
-        scheme = scheme.toLowerCase();
-        String[] schemeParts = StringUtils.split(scheme, '-');
-        checkArgument(Objects.equal(DistributedLogConstants.SCHEME_PREFIX, schemeParts[0].toLowerCase()),
-                "Unknown distributedlog scheme found : " + uri);
-        URI normalizedUri;
-        try {
-            normalizedUri = new URI(
-                    schemeParts[0],     // remove backend info
-                    uri.getAuthority(),
-                    uri.getPath(),
-                    uri.getQuery(),
-                    uri.getFragment());
-        } catch (URISyntaxException e) {
-            throw new IllegalArgumentException("Invalid distributedlog uri found : " + uri, e);
-        }
-        return normalizedUri;
-    }
-
-    private static String getHostIpLockClientId() {
-        try {
-            return InetAddress.getLocalHost().toString();
-        } catch(Exception ex) {
-            return DistributedLogConstants.UNKNOWN_CLIENT_ID;
-        }
-    }
-
-    /**
-     * Normalize the client id.
-     *
-     * @return the normalized client id.
-     */
-    public static String normalizeClientId(String clientId) {
-        String normalizedClientId;
-        if (clientId.equals(DistributedLogConstants.UNKNOWN_CLIENT_ID)) {
-            normalizedClientId = getHostIpLockClientId();
-        } else {
-            normalizedClientId = clientId;
-        }
-        return normalizedClientId;
-    }
-
-    /**
-     * Is it a reserved stream name in bkdl namespace?
-     *
-     * @param name
-     *          stream name
-     * @return true if it is reserved name, otherwise false.
-     */
-    public static boolean isReservedStreamName(String name) {
-        return name.startsWith(".");
-    }
-
-    /**
-     * Validate the stream name.
-     *
-     * @param nameOfStream
-     *          name of stream
-     * @throws InvalidStreamNameException
-     */
-    public static void validateName(String nameOfStream)
-            throws InvalidStreamNameException {
-        String reason = null;
-        char chars[] = nameOfStream.toCharArray();
-        char c;
-        // validate the stream to see if meet zookeeper path's requirement
-        for (int i = 0; i < chars.length; i++) {
-            c = chars[i];
-
-            if (c == 0) {
-                reason = "null character not allowed @" + i;
-                break;
-            } else if (c == '/') {
-                reason = "'/' not allowed @" + i;
-                break;
-            } else if (c > '\u0000' && c < '\u001f'
-                    || c > '\u007f' && c < '\u009F'
-                    || c > '\ud800' && c < '\uf8ff'
-                    || c > '\ufff0' && c < '\uffff') {
-                reason = "invalid charater @" + i;
-                break;
-            }
-        }
-        if (null != reason) {
-            throw new InvalidStreamNameException(nameOfStream, reason);
-        }
-        if (isReservedStreamName(nameOfStream)) {
-            throw new InvalidStreamNameException(nameOfStream,
-                    "Stream Name is reserved");
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FailpointUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FailpointUtils.java
deleted file mode 100644
index 64101b3..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FailpointUtils.java
+++ /dev/null
@@ -1,134 +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 com.twitter.distributedlog.util;
-
-import java.io.IOException;
-import java.util.concurrent.ConcurrentHashMap;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class FailpointUtils {
-    static final Logger logger = LoggerFactory.getLogger(FailpointUtils.class);
-
-    public enum FailPointName {
-        FP_StartLogSegmentBeforeLedgerCreate,
-        FP_StartLogSegmentAfterLedgerCreate,
-        FP_StartLogSegmentAfterInProgressCreate,
-        FP_StartLogSegmentOnAssignLogSegmentSequenceNumber,
-        FP_FinalizeLedgerBeforeDelete,
-        FP_TransmitBeforeAddEntry,
-        FP_TransmitComplete,
-        FP_WriteInternalLostLock,
-        FP_TransmitFailGetBuffer,
-        FP_LockUnlockCleanup,
-        FP_LockTryCloseRaceCondition,
-        FP_LockTryAcquire,
-        FP_ZooKeeperConnectionLoss,
-        FP_RecoverIncompleteLogSegments,
-        FP_LogWriterIssuePending,
-    }
-
-    public static interface FailPointAction {
-        boolean checkFailPoint() throws IOException;
-        boolean checkFailPointNoThrow();
-    }
-
-    public static abstract class AbstractFailPointAction implements FailPointAction {
-        @Override
-        public boolean checkFailPointNoThrow() {
-            try {
-                return checkFailPoint();
-            } catch (IOException ex) {
-                logger.error("failpoint action raised unexpected exception");
-                return true;
-            }
-        }
-    }
-
-    public static final FailPointAction DEFAULT_ACTION = new AbstractFailPointAction() {
-        @Override
-        public boolean checkFailPoint() throws IOException {
-            return true;
-        }
-    };
-
-    public static final FailPointAction THROW_ACTION = new AbstractFailPointAction() {
-        @Override
-        public boolean checkFailPoint() throws IOException {
-            throw new IOException("Throw ioexception for failure point");
-        }
-    };
-
-    public enum FailPointActions {
-        FailPointAction_Default,
-        FailPointAction_Throw
-    }
-
-    static ConcurrentHashMap<FailPointName, FailPointAction> failPointState =
-            new ConcurrentHashMap<FailPointName, FailPointAction>();
-
-    public static void setFailpoint(FailPointName failpoint, FailPointActions action) {
-        FailPointAction fpAction = null;
-        switch (action) {
-        case FailPointAction_Default:
-            fpAction = DEFAULT_ACTION;
-            break;
-        case FailPointAction_Throw:
-            fpAction = THROW_ACTION;
-            break;
-        default:
-            break;
-        }
-        setFailpoint(failpoint, fpAction);
-    }
-
-    public static void setFailpoint(FailPointName failpoint, FailPointAction action) {
-        if (null != action) {
-            failPointState.put(failpoint, action);
-        }
-    }
-
-    public static void removeFailpoint(FailPointName failpoint) {
-        failPointState.remove(failpoint);
-    }
-
-    public static boolean checkFailPoint(FailPointName failPoint) throws IOException {
-        FailPointAction action = failPointState.get(failPoint);
-
-        if (action == null) {
-            return false;
-        }
-
-        try {
-            return action.checkFailPoint();
-        } catch (IOException ioe) {
-            throw new IOException("Induced Exception at:" + failPoint, ioe);
-        }
-    }
-
-    public static boolean checkFailPointNoThrow(FailPointName failPoint) {
-        FailPointAction action = failPointState.get(failPoint);
-
-        if (action == null) {
-            return false;
-        }
-
-        return action.checkFailPointNoThrow();
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java
deleted file mode 100644
index f206a25..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java
+++ /dev/null
@@ -1,534 +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 com.twitter.distributedlog.util;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.stats.OpStatsListener;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureCancelledException;
-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.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 javax.annotation.Nullable;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-
-/**
- * Utilities to process future
- */
-public class FutureUtils {
-
-    private static final Logger logger = LoggerFactory.getLogger(FutureUtils.class);
-
-    public static class OrderedFutureEventListener<R>
-            implements FutureEventListener<R> {
-
-        public static <R> OrderedFutureEventListener<R> of(
-                FutureEventListener<R> listener,
-                OrderedScheduler scheduler,
-                Object key) {
-            return new OrderedFutureEventListener<R>(scheduler, key, listener);
-        }
-
-        private final OrderedScheduler scheduler;
-        private final Object key;
-        private final FutureEventListener<R> listener;
-
-        private OrderedFutureEventListener(OrderedScheduler scheduler,
-                                           Object key,
-                                           FutureEventListener<R> listener) {
-            this.scheduler = scheduler;
-            this.key = key;
-            this.listener = listener;
-        }
-
-        @Override
-        public void onSuccess(final R value) {
-            scheduler.submit(key, new Runnable() {
-                @Override
-                public void run() {
-                    listener.onSuccess(value);
-                }
-            });
-        }
-
-        @Override
-        public void onFailure(final Throwable cause) {
-            scheduler.submit(key, new Runnable() {
-                @Override
-                public void run() {
-                    listener.onFailure(cause);
-                }
-            });
-        }
-    }
-
-    public static class FutureEventListenerRunnable<R>
-            implements FutureEventListener<R> {
-
-        public static <R> FutureEventListenerRunnable<R> of(
-                FutureEventListener<R> listener,
-                ExecutorService executorService) {
-            return new FutureEventListenerRunnable<R>(executorService, listener);
-        }
-
-        private final ExecutorService executorService;
-        private final FutureEventListener<R> listener;
-
-        private FutureEventListenerRunnable(ExecutorService executorService,
-                                            FutureEventListener<R> listener) {
-            this.executorService = executorService;
-            this.listener = listener;
-        }
-
-        @Override
-        public void onSuccess(final R value) {
-            executorService.submit(new Runnable() {
-                @Override
-                public void run() {
-                    listener.onSuccess(value);
-                }
-            });
-        }
-
-        @Override
-        public void onFailure(final Throwable cause) {
-            executorService.submit(new Runnable() {
-                @Override
-                public void run() {
-                    listener.onFailure(cause);
-                }
-            });
-        }
-    }
-
-    private static class ListFutureProcessor<T, R>
-            extends Function<Throwable, BoxedUnit>
-            implements FutureEventListener<R>, Runnable {
-
-        private volatile boolean interrupted = false;
-        private final Iterator<T> itemsIter;
-        private final Function<T, Future<R>> processFunc;
-        private final Promise<List<R>> promise;
-        private final List<R> results;
-        private final ExecutorService callbackExecutor;
-
-        ListFutureProcessor(List<T> items,
-                            Function<T, Future<R>> processFunc,
-                            ExecutorService callbackExecutor) {
-            this.itemsIter = items.iterator();
-            this.processFunc = processFunc;
-            this.promise = new Promise<List<R>>();
-            this.promise.setInterruptHandler(this);
-            this.results = new ArrayList<R>();
-            this.callbackExecutor = callbackExecutor;
-        }
-
-        @Override
-        public BoxedUnit apply(Throwable cause) {
-            interrupted = true;
-            return BoxedUnit.UNIT;
-        }
-
-        @Override
-        public void onSuccess(R value) {
-            results.add(value);
-            if (null == callbackExecutor) {
-                run();
-            } else {
-                callbackExecutor.submit(this);
-            }
-        }
-
-        @Override
-        public void onFailure(final Throwable cause) {
-            if (null == callbackExecutor) {
-                promise.setException(cause);
-            } else {
-                callbackExecutor.submit(new Runnable() {
-                    @Override
-                    public void run() {
-                        promise.setException(cause);
-                    }
-                });
-            }
-        }
-
-        @Override
-        public void run() {
-            if (interrupted) {
-                logger.debug("ListFutureProcessor is interrupted.");
-                return;
-            }
-            if (!itemsIter.hasNext()) {
-                promise.setValue(results);
-                return;
-            }
-            processFunc.apply(itemsIter.next()).addEventListener(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> Future<List<R>> processList(List<T> collection,
-                                                     Function<T, Future<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;
-    }
-
-    /**
-     * 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> Future<T> stats(Future<T> result,
-                                      OpStatsLogger opStatsLogger,
-                                      Stopwatch stopwatch) {
-        return result.addEventListener(new OpStatsListener<T>(opStatsLogger, stopwatch));
-    }
-
-    /**
-     * Await for the result of the future and thrown bk related exceptions.
-     *
-     * @param result future to wait for
-     * @return the result of future
-     * @throws BKException when exceptions are thrown by the future. If there is unkown exceptions
-     *         thrown from the future, the exceptions will be wrapped into
-     *         {@link org.apache.bookkeeper.client.BKException.BKUnexpectedConditionException}.
-     */
-    public static <T> T bkResult(Future<T> result) throws BKException {
-        try {
-            return Await.result(result);
-        } catch (BKException bke) {
-            throw bke;
-        } catch (InterruptedException ie) {
-            throw BKException.create(BKException.Code.InterruptedException);
-        } catch (Exception e) {
-            logger.warn("Encountered unexpected exception on waiting bookkeeper results : ", e);
-            throw BKException.create(BKException.Code.UnexpectedConditionException);
-        }
-    }
-
-    /**
-     * Return the bk exception return code for a <i>throwable</i>.
-     *
-     * @param throwable the cause of the exception
-     * @return the bk exception return code. if the exception isn't bk exceptions,
-     *         it would return {@link BKException.Code#UnexpectedConditionException}.
-     */
-    public static int bkResultCode(Throwable throwable) {
-        if (throwable instanceof BKException) {
-            return ((BKException)throwable).getCode();
-        }
-        return BKException.Code.UnexpectedConditionException;
-    }
-
-    /**
-     * Wait for the result until it completes.
-     *
-     * @param result result to wait
-     * @return the result
-     * @throws IOException when encountered exceptions on the result
-     */
-    public static <T> T result(Future<T> result) throws IOException {
-        return result(result, Duration.Top());
-    }
-
-    /**
-     * 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}.
-     *
-     * @param result result to wait
-     * @param duration duration to wait
-     * @return the result
-     * @throws IOException when encountered exceptions on the result or waiting for the result.
-     */
-    public static <T> T result(Future<T> result, Duration duration)
-            throws IOException {
-        try {
-            return Await.result(result, duration);
-        } catch (KeeperException ke) {
-            throw new ZKException("Encountered zookeeper exception on waiting result", ke);
-        } catch (BKException bke) {
-            throw new BKTransmitException("Encountered bookkeeper exception on waiting result", bke.getCode());
-        } catch (IOException ioe) {
-            throw ioe;
-        } catch (InterruptedException ie) {
-            throw new DLInterruptedException("Interrupted on waiting result", ie);
-        } catch (Exception e) {
-            throw new IOException("Encountered exception on waiting result", e);
-        }
-    }
-
-    /**
-     * Wait for the result of a lock operation.
-     *
-     * @param result result to wait
-     * @param lockPath path of the lock
-     * @return the result
-     * @throws LockingException when encountered exceptions on the result of lock operation
-     */
-    public static <T> T lockResult(Future<T> result, String lockPath) throws LockingException {
-        try {
-            return Await.result(result);
-        } catch (LockingException le) {
-            throw le;
-        } catch (Exception e) {
-            throw new LockingException(lockPath, "Encountered exception on locking ", e);
-        }
-    }
-
-    /**
-     * 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);
-        }
-    }
-
-    /**
-     * Cancel the future. It would interrupt the future.
-     *
-     * @param future future to cancel
-     */
-    public static <T> void cancel(Future<T> future) {
-        future.raise(new FutureCancelledException());
-    }
-
-    /**
-     * 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> Promise<T> within(final Promise<T> promise,
-                                        final long timeout,
-                                        final TimeUnit unit,
-                                        final Throwable cause,
-                                        final OrderedScheduler scheduler,
-                                        final Object key) {
-        if (timeout < DistributedLogConstants.FUTURE_TIMEOUT_IMMEDIATE || promise.isDefined()) {
-            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.isDefined() && FutureUtils.setException(promise, cause)) {
-                    logger.info("Raise exception", cause);
-                }
-            }
-        }, timeout, unit);
-        // when the promise is satisfied, cancel the timeout task
-        promise.respond(new AbstractFunction1<Try<T>, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Try<T> value) {
-                if (!task.cancel(true)) {
-                    logger.debug("Failed to cancel the timeout task");
-                }
-                return BoxedUnit.UNIT;
-            }
-        });
-        return promise;
-    }
-
-    /**
-     * 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
-     * @param value value to satisfy
-     * @param scheduler scheduler to satisfy the promise with provided value
-     * @param key the submit key of the ordered scheduler
-     */
-    public static <T> void setValue(final Promise<T> promise,
-                                    final T value,
-                                    OrderedScheduler scheduler,
-                                    Object key) {
-        scheduler.submit(key, new Runnable() {
-            @Override
-            public void run() {
-                setValue(promise, value);
-            }
-        });
-    }
-
-    /**
-     * Satisfy the <i>promise</i> with provide value.
-     * <p>If the promise was already satisfied, nothing will be changed.
-     *
-     * @param promise promise to satisfy
-     * @param value value to satisfy
-     * @return true if successfully satisfy the future. false if the promise has been satisfied.
-     */
-    public static <T> boolean setValue(Promise<T> promise, T value) {
-        boolean success = promise.updateIfEmpty(new Return<T>(value));
-        if (!success) {
-            logger.info("Result set multiple times. Value = '{}', New = 'Return({})'",
-                    promise.poll(), value);
-        }
-        return success;
-    }
-
-    /**
-     * 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 scheduler the scheduler to satisfy the promise
-     * @param key submit key of the ordered scheduler
-     */
-    public static <T> void setException(final Promise<T> promise,
-                                        final Throwable cause,
-                                        OrderedScheduler scheduler,
-                                        Object key) {
-        scheduler.submit(key, new Runnable() {
-            @Override
-            public void run() {
-                setException(promise, cause);
-            }
-        });
-    }
-
-    /**
-     * Satisfy the <i>promise</i> with provided <i>cause</i>.
-     *
-     * @param promise promise to satisfy
-     * @param cause cause to satisfy
-     * @return true if successfully satisfy the future. false if the promise has been satisfied.
-     */
-    public static <T> boolean setException(Promise<T> promise, Throwable cause) {
-        boolean success = promise.updateIfEmpty(new Throw<T>(cause));
-        if (!success) {
-            logger.info("Result set multiple times. Value = '{}', New = 'Throw({})'",
-                    promise.poll(), cause);
-        }
-        return success;
-    }
-
-    /**
-     * Ignore exception from the <i>future</i>.
-     *
-     * @param future the original future
-     * @return a transformed future ignores exceptions
-     */
-    public static <T> Promise<Void> ignore(Future<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> Promise<Void> ignore(Future<T> future, final String errorMsg) {
-        final Promise<Void> promise = new Promise<Void>();
-        future.addEventListener(new FutureEventListener<T>() {
-            @Override
-            public void onSuccess(T value) {
-                setValue(promise, null);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                if (null != errorMsg) {
-                    logger.error(errorMsg, cause);
-                }
-                setValue(promise, null);
-            }
-        });
-        return promise;
-    }
-
-    /**
-     * 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);
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredFuturePool.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredFuturePool.java
deleted file mode 100644
index e06023e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/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 com.twitter.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/com/twitter/distributedlog/util/MonitoredScheduledThreadPoolExecutor.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredScheduledThreadPoolExecutor.java
deleted file mode 100644
index 75223f2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/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 com.twitter.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/com/twitter/distributedlog/util/OrderedScheduler.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/OrderedScheduler.java
deleted file mode 100644
index 287bd6d..0000000
--- a/distributedlog-core/src/main/java/com/twitter/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 com.twitter.distributedlog.util;
-
-import com.google.common.base.Objects;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.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/com/twitter/distributedlog/util/PermitLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitLimiter.java
deleted file mode 100644
index 41c28a3..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitLimiter.java
+++ /dev/null
@@ -1,57 +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 com.twitter.distributedlog.util;
-
-/**
- * A simple limiter interface which tracks acquire/release of permits, for
- * example for tracking outstanding writes.
- */
-public interface PermitLimiter {
-
-    public static PermitLimiter NULL_PERMIT_LIMITER = new PermitLimiter() {
-        @Override
-        public boolean acquire() {
-            return true;
-        }
-        @Override
-        public void release(int permits) {
-        }
-
-        @Override
-        public void close() {
-
-        }
-    };
-
-    /**
-     * Acquire a permit.
-     *
-     * @return true if successfully acquire a permit, otherwise false.
-     */
-    boolean acquire();
-
-    /**
-     * Release a permit.
-     */
-    void release(int permits);
-
-    /**
-     * Close the resources created by the limiter
-     */
-    void close();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitManager.java
deleted file mode 100644
index 6a6d574..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitManager.java
+++ /dev/null
@@ -1,93 +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 com.twitter.distributedlog.util;
-
-public interface PermitManager {
-
-    public static interface Permit {
-        static final Permit ALLOWED = new Permit() {
-            @Override
-            public boolean isAllowed() {
-                return true;
-            }
-        };
-        boolean isAllowed();
-    }
-
-    public static PermitManager UNLIMITED_PERMIT_MANAGER = new PermitManager() {
-        @Override
-        public Permit acquirePermit() {
-            return Permit.ALLOWED;
-        }
-
-        @Override
-        public void releasePermit(Permit permit) {
-            // nop
-        }
-
-        @Override
-        public boolean allowObtainPermits() {
-            return true;
-        }
-
-        @Override
-        public boolean disallowObtainPermits(Permit permit) {
-            return false;
-        }
-
-        @Override
-        public void close() {
-            // nop
-        }
-
-    };
-
-    /**
-     * Obetain a permit from permit manager.
-     *
-     * @return permit.
-     */
-    Permit acquirePermit();
-
-    /**
-     * Release a given permit.
-     *
-     * @param permit
-     *          permit to release
-     */
-    void releasePermit(Permit permit);
-
-    /**
-     * Allow obtaining permits.
-     */
-    boolean allowObtainPermits();
-
-    /**
-     * Disallow obtaining permits. Disallow needs to be performed under the context
-     * of <i>permit</i>.
-     *
-     * @param permit
-     *          permit context to disallow
-     */
-    boolean disallowObtainPermits(Permit permit);
-
-    /**
-     * Release the resources
-     */
-    void close();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/RetryPolicyUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/RetryPolicyUtils.java
deleted file mode 100644
index 3565f98..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/RetryPolicyUtils.java
+++ /dev/null
@@ -1,45 +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 com.twitter.distributedlog.util;
-
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.bookkeeper.zookeeper.RetryPolicy;
-
-/**
- * Utils for {@link org.apache.bookkeeper.zookeeper.RetryPolicy}
- */
-public class RetryPolicyUtils {
-
-    /**
-     * Infinite retry policy
-     */
-    public static final RetryPolicy DEFAULT_INFINITE_RETRY_POLICY = infiniteRetry(200, 2000);
-
-    /**
-     * Create an infinite retry policy with backoff time between <i>baseBackOffTimeMs</i> and
-     * <i>maxBackoffTimeMs</i>.
-     *
-     * @param baseBackoffTimeMs base backoff time in milliseconds
-     * @param maxBackoffTimeMs maximum backoff time in milliseconds
-     * @return an infinite retry policy
-     */
-    public static RetryPolicy infiniteRetry(long baseBackoffTimeMs, long maxBackoffTimeMs) {
-        return new BoundExponentialBackoffRetryPolicy(baseBackoffTimeMs, maxBackoffTimeMs, Integer.MAX_VALUE);
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SafeQueueingFuturePool.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SafeQueueingFuturePool.java
deleted file mode 100644
index d139a80..0000000
--- a/distributedlog-core/src/main/java/com/twitter/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 com.twitter.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/com/twitter/distributedlog/util/SchedulerUtils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SchedulerUtils.java
deleted file mode 100644
index 9f756f0..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SchedulerUtils.java
+++ /dev/null
@@ -1,56 +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 com.twitter.distributedlog.util;
-
-import org.apache.bookkeeper.util.OrderedSafeExecutor;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-
-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;
-        }
-        service.shutdown();
-        try {
-            service.awaitTermination(timeout, timeUnit);
-        } catch (InterruptedException e) {
-            logger.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/com/twitter/distributedlog/util/Sequencer.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sequencer.java
deleted file mode 100644
index 7ec50ba..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sequencer.java
+++ /dev/null
@@ -1,31 +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 com.twitter.distributedlog.util;
-
-/**
- * Sequencer generating transaction id.
- */
-public interface Sequencer {
-
-    /**
-     * Return next transaction id generated by the sequencer.
-     *
-     * @return next transaction id generated by the sequencer.
-     */
-    long nextId();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SimplePermitLimiter.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SimplePermitLimiter.java
deleted file mode 100644
index 4086a1e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SimplePermitLimiter.java
+++ /dev/null
@@ -1,117 +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 com.twitter.distributedlog.util;
-
-import com.google.common.annotations.VisibleForTesting;
-
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-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;
-
-/**
- * Simple counter based {@link PermitLimiter}.
- *
- * <h3>Metrics</h3>
- * <ul>
- * <li> `permits`: gauge. how many permits are acquired right now?
- * <li> `permits`/*: opstats. the characteristics about number of permits already acquired on each acquires.
- * <li> `acquireFailure`: counter. how many acquires failed? failure means it already reached maximum permits
- * when trying to acquire.
- * </ul>
- */
-public class SimplePermitLimiter implements PermitLimiter {
-
-    final Counter acquireFailureCounter;
-    final OpStatsLogger permitsMetric;
-    final AtomicInteger permits;
-    final int permitsMax;
-    final boolean darkmode;
-    final Feature disableWriteLimitFeature;
-    private StatsLogger statsLogger = null;
-    private Gauge<Number> permitsGauge = null;
-    private String permitsGaugeLabel = "";
-
-    public SimplePermitLimiter(boolean darkmode, int permitsMax, StatsLogger statsLogger,
-                               boolean singleton, Feature disableWriteLimitFeature) {
-        this.permits = new AtomicInteger(0);
-        this.permitsMax = permitsMax;
-        this.darkmode = darkmode;
-        this.disableWriteLimitFeature = disableWriteLimitFeature;
-
-        // stats
-        if (singleton) {
-            this.statsLogger = statsLogger;
-            this.permitsGauge = new Gauge<Number>() {
-                @Override
-                public Number getDefaultValue() {
-                    return 0;
-                }
-                @Override
-                public Number getSample() {
-                    return permits.get();
-                }
-            };
-            this.permitsGaugeLabel = "permits";
-            statsLogger.registerGauge(permitsGaugeLabel, permitsGauge);
-        }
-        acquireFailureCounter = statsLogger.getCounter("acquireFailure");
-        permitsMetric = statsLogger.getOpStatsLogger("permits");
-    }
-
-    public boolean isDarkmode() {
-        return darkmode || disableWriteLimitFeature.isAvailable();
-    }
-
-    @Override
-    public boolean acquire() {
-        permitsMetric.registerSuccessfulEvent(permits.get());
-        if (permits.incrementAndGet() <= permitsMax || isDarkmode()) {
-            return true;
-        } else {
-            acquireFailureCounter.inc();
-            permits.decrementAndGet();
-            return false;
-        }
-    }
-
-    @Override
-    public void release(int permitsToRelease) {
-        permits.addAndGet(-permitsToRelease);
-    }
-
-    @Override
-    public void close() {
-        unregisterGauge();
-    }
-
-    @VisibleForTesting
-    public int getPermits() {
-        return permits.get();
-    }
-
-    public void unregisterGauge() {
-        if (this.statsLogger != null && this.permitsGauge != null) {
-            this.statsLogger.unregisterGauge(permitsGaugeLabel, permitsGauge);
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sizable.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sizable.java
deleted file mode 100644
index 216d5ea..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sizable.java
+++ /dev/null
@@ -1,31 +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 com.twitter.distributedlog.util;
-
-/**
- * The {@code Sizable} interface is to provide the capability of calculating size
- * of any objects.
- */
-public interface Sizable {
-    /**
-     * Calculate the size for this instance.
-     *
-     * @return size of the instance.
-     */
-    long size();
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/TimeSequencer.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/TimeSequencer.java
deleted file mode 100644
index 96e564e..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/TimeSequencer.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 com.twitter.distributedlog.util;
-
-import com.twitter.distributedlog.DistributedLogConstants;
-
-/**
- * Time based sequencer. It generated non-decreasing transaction id using milliseconds.
- * It isn't thread-safe. The caller takes the responsibility on synchronization.
- */
-public class TimeSequencer implements Sequencer {
-
-    private long lastId = DistributedLogConstants.INVALID_TXID;
-
-    public void setLastId(long lastId) {
-        this.lastId = lastId;
-    }
-
-    @Override
-    public long nextId() {
-        lastId = Math.max(lastId, System.currentTimeMillis());
-        return lastId;
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Transaction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Transaction.java
deleted file mode 100644
index 422bbda..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Transaction.java
+++ /dev/null
@@ -1,97 +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 com.twitter.distributedlog.util;
-
-import com.google.common.annotations.Beta;
-import com.twitter.util.Future;
-
-/**
- * Util class represents a transaction
- */
-@Beta
-public interface Transaction<OpResult> {
-
-    /**
-     * An operation executed in a transaction.
-     */
-    interface Op<OpResult> {
-
-        /**
-         * Execute after the transaction succeeds
-         */
-        void commit(OpResult r);
-
-        /**
-         * Execute after the transaction fails
-         */
-        void abort(Throwable t, OpResult r);
-
-    }
-
-    /**
-     * Listener on the result of an {@link com.twitter.distributedlog.util.Transaction.Op}.
-     *
-     * @param <OpResult>
-     */
-    interface OpListener<OpResult> {
-
-        /**
-         * Trigger on operation committed.
-         *
-         * @param r
-         *          result to return
-         */
-        void onCommit(OpResult r);
-
-        /**
-         * Trigger on operation aborted.
-         *
-         * @param t
-         *          reason to abort
-         */
-        void onAbort(Throwable t);
-    }
-
-    /**
-     * Add the operation to current transaction.
-     *
-     * @param operation
-     *          operation to execute under current transaction
-     */
-    void addOp(Op<OpResult> operation);
-
-    /**
-     * Execute the current transaction. If the transaction succeed, all operations will be
-     * committed (via {@link com.twitter.distributedlog.util.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();
-
-    /**
-     * Abort current transaction. If this is called and the transaction haven't been executed by
-     * {@link #execute()}, it would abort all operations. If the transaction has been executed,
-     * the behavior is left up to implementation - if transaction is cancellable, the {@link #abort(Throwable)}
-     * could attempt to cancel it.
-     *
-     * @param reason reason to abort the transaction
-     */
-    void abort(Throwable reason);
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Utils.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Utils.java
deleted file mode 100644
index fce9bcd..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Utils.java
+++ /dev/null
@@ -1,607 +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 com.twitter.distributedlog.util;
-
-import java.io.Closeable;
-import java.io.IOException;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.atomic.AtomicInteger;
-import javax.annotation.Nullable;
-
-import com.google.common.base.Objects;
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.google.common.io.Closeables;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.function.VoidFunctions;
-import com.twitter.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;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-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.
- */
-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
-     * changes. This is appropriate to use when computing how much longer to
-     * wait for an interval to expire.
-     *
-     * @return current time in nanoseconds.
-     */
-    public static long nowInNanos() {
-        return System.nanoTime();
-    }
-
-    /**
-     * Current time from some fixed base time - so useful for cross machine
-     * comparison
-     *
-     * @return current time in milliseconds.
-     */
-    public static long nowInMillis() {
-        return System.currentTimeMillis();
-    }
-
-    /**
-     * Milliseconds elapsed since the time specified, the input is nanoTime
-     * the only conversion happens when computing the elapsed time
-     *
-     * @param startMsecTime the start of the interval that we are measuring
-     * @return elapsed time in milliseconds.
-     */
-    public static long elapsedMSec(long startMsecTime) {
-        return (System.currentTimeMillis() - startMsecTime);
-    }
-
-    public static boolean randomPercent(double percent) {
-        return (Math.random() * 100.0) <= percent;
-    }
-
-    /**
-     * Synchronously create zookeeper path recursively and optimistically.
-     *
-     * @see #zkAsyncCreateFullPathOptimistic(ZooKeeperClient, String, byte[], List, CreateMode)
-     * @param zkc Zookeeper client
-     * @param path Zookeeper full path
-     * @param data Zookeeper data
-     * @param acl Acl of the zk path
-     * @param createMode Create mode of zk path
-     * @throws ZooKeeperClient.ZooKeeperConnectionException
-     * @throws KeeperException
-     * @throws InterruptedException
-     */
-    public static void zkCreateFullPathOptimistic(
-        ZooKeeperClient zkc,
-        String path,
-        byte[] data,
-        final List<ACL> acl,
-        final CreateMode createMode)
-        throws ZooKeeperClient.ZooKeeperConnectionException, KeeperException, InterruptedException {
-        try {
-            Await.result(zkAsyncCreateFullPathOptimistic(zkc, path, data, acl, createMode));
-        } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
-            throw zkce;
-        } catch (KeeperException ke) {
-            throw ke;
-        } catch (InterruptedException ie) {
-            throw ie;
-        } catch (RuntimeException rte) {
-            throw rte;
-        } catch (Exception exc) {
-            throw new RuntimeException("Unexpected Exception", exc);
-        }
-    }
-
-    /**
-     * Asynchronously create zookeeper path recursively and optimistically.
-     *
-     * @param zkc Zookeeper client
-     * @param pathToCreate  Zookeeper full path
-     * @param parentPathShouldNotCreate The recursive creation should stop if this path doesn't exist
-     * @param data Zookeeper data
-     * @param acl Acl of the zk path
-     * @param createMode Create mode of zk path
-     * @param callback Callback
-     * @param ctx Context object
-     */
-    public static void zkAsyncCreateFullPathOptimisticRecursive(
-        final ZooKeeperClient zkc,
-        final String pathToCreate,
-        final Optional<String> parentPathShouldNotCreate,
-        final byte[] data,
-        final List<ACL> acl,
-        final CreateMode createMode,
-        final AsyncCallback.StringCallback callback,
-        final Object ctx) {
-        try {
-            zkc.get().create(pathToCreate, data, acl, createMode, new AsyncCallback.StringCallback() {
-                @Override
-                public void processResult(int rc, String path, Object ctx, String name) {
-
-                    if (rc != KeeperException.Code.NONODE.intValue()) {
-                        callback.processResult(rc, path, ctx, name);
-                        return;
-                    }
-
-                    // Since we got a nonode, it means that my parents may not exist
-                    // ephemeral nodes can't have children so Create mode is always
-                    // persistent parents
-                    int lastSlash = pathToCreate.lastIndexOf('/');
-                    if (lastSlash <= 0) {
-                        callback.processResult(rc, path, ctx, name);
-                        return;
-                    }
-                    String parent = pathToCreate.substring(0, lastSlash);
-                    if (parentPathShouldNotCreate.isPresent() && Objects.equal(parentPathShouldNotCreate.get(), parent)) {
-                        // we should stop here
-                        callback.processResult(rc, path, ctx, name);
-                        return;
-                    }
-                    zkAsyncCreateFullPathOptimisticRecursive(zkc, parent, parentPathShouldNotCreate, new byte[0], acl,
-                            CreateMode.PERSISTENT, new AsyncCallback.StringCallback() {
-                                @Override
-                                public void processResult(int rc, String path, Object ctx, String name) {
-                                    if (rc == KeeperException.Code.OK.intValue() || rc == KeeperException.Code.NODEEXISTS.intValue()) {
-                                        // succeeded in creating the parent, now create the original path
-                                        zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate, parentPathShouldNotCreate,
-                                                data, acl, createMode, callback, ctx);
-                                    } else {
-                                        callback.processResult(rc, path, ctx, name);
-                                    }
-                                }
-                            }, ctx);
-                }
-            }, ctx);
-        } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
-            callback.processResult(DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE, zkce.getMessage(), ctx, pathToCreate);
-        } catch (InterruptedException ie) {
-            callback.processResult(DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE, ie.getMessage(), ctx, pathToCreate);
-        }
-    }
-
-    /**
-     * Asynchronously create zookeeper path recursively and optimistically.
-     *
-     * @param zkc Zookeeper client
-     * @param pathToCreate  Zookeeper full path
-     * @param data Zookeeper data
-     * @param acl Acl of the zk path
-     * @param createMode Create mode of zk path
-     */
-    public static Future<BoxedUnit> zkAsyncCreateFullPathOptimistic(
-        final ZooKeeperClient zkc,
-        final String pathToCreate,
-        final byte[] data,
-        final List<ACL> acl,
-        final CreateMode createMode) {
-        Optional<String> parentPathShouldNotCreate = Optional.absent();
-        return zkAsyncCreateFullPathOptimistic(
-                zkc,
-                pathToCreate,
-                parentPathShouldNotCreate,
-                data,
-                acl,
-                createMode);
-    }
-
-    /**
-     * Asynchronously create zookeeper path recursively and optimistically
-     *
-     * @param zkc Zookeeper client
-     * @param pathToCreate  Zookeeper full path
-     * @param parentPathShouldNotCreate zookeeper parent path should not be created
-     * @param data Zookeeper data
-     * @param acl Acl of the zk path
-     * @param createMode Create mode of zk path
-     */
-    public static Future<BoxedUnit> 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>();
-
-        zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate, parentPathShouldNotCreate,
-                data, acl, createMode, new AsyncCallback.StringCallback() {
-            @Override
-            public void processResult(int rc, String path, Object ctx, String name) {
-                handleKeeperExceptionCode(rc, path, result);
-            }
-        }, result);
-
-        return result;
-    }
-
-    /**
-     * Asynchronously create zookeeper path recursively and optimistically.
-     *
-     * @param zkc Zookeeper client
-     * @param pathToCreate  Zookeeper full path
-     * @param data Zookeeper data
-     * @param acl Acl of the zk path
-     * @param createMode Create mode of zk path
-     */
-    public static Future<BoxedUnit> zkAsyncCreateFullPathOptimisticAndSetData(
-        final ZooKeeperClient zkc,
-        final String pathToCreate,
-        final byte[] data,
-        final List<ACL> acl,
-        final CreateMode createMode) {
-        final Promise<BoxedUnit> result = new Promise<BoxedUnit>();
-
-        try {
-            zkc.get().setData(pathToCreate, data, -1, new AsyncCallback.StatCallback() {
-                @Override
-                public void processResult(int rc, String path, Object ctx, Stat stat) {
-                    if (rc != KeeperException.Code.NONODE.intValue()) {
-                        handleKeeperExceptionCode(rc, path, result);
-                        return;
-                    }
-
-                    Optional<String> parentPathShouldNotCreate = Optional.absent();
-                    zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate, parentPathShouldNotCreate,
-                            data, acl, createMode, new AsyncCallback.StringCallback() {
-                        @Override
-                        public void processResult(int rc, String path, Object ctx, String name) {
-                            handleKeeperExceptionCode(rc, path, result);
-                        }
-                    }, result);
-                }
-            }, result);
-        } catch (Exception exc) {
-            result.setException(exc);
-        }
-
-        return result;
-    }
-
-    private static void handleKeeperExceptionCode(int rc, String pathOrMessage, Promise<BoxedUnit> result) {
-        if (KeeperException.Code.OK.intValue() == rc) {
-            result.setValue(BoxedUnit.UNIT);
-        } else if (DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE == rc) {
-            result.setException(new ZooKeeperClient.ZooKeeperConnectionException(pathOrMessage));
-        } else if (DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE == rc) {
-            result.setException(new DLInterruptedException(pathOrMessage));
-        } else {
-            result.setException(KeeperException.create(KeeperException.Code.get(rc), pathOrMessage));
-        }
-    }
-
-    public static Future<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));
-        } catch (InterruptedException e) {
-            return Future.exception(FutureUtils.zkException(e, path));
-        }
-        return zkGetData(zk, path, watch);
-    }
-
-    /**
-     * Retrieve data from zookeeper <code>path</code>.
-     *
-     * @param path
-     *          zookeeper path to retrieve data
-     * @param watch
-     *          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[]>>();
-        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));
-                    } else {
-                        promise.setValue(new Versioned<byte[]>(data, new ZkVersion(stat.getVersion())));
-                    }
-                } else if (KeeperException.Code.NONODE.intValue() == rc) {
-                    promise.setValue(new Versioned<byte[]>(null, null));
-                } else {
-                    promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
-                }
-            }
-        }, null);
-        return promise;
-    }
-
-    public static Future<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));
-        } catch (InterruptedException e) {
-            return Future.exception(FutureUtils.zkException(e, path));
-        }
-        return zkSetData(zk, path, data, version);
-    }
-
-    /**
-     * Set <code>data</code> to zookeeper <code>path</code>.
-     *
-     * @param zk
-     *          zookeeper client
-     * @param path
-     *          path to set data
-     * @param data
-     *          data to set
-     * @param version
-     *          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>();
-        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())));
-                    return;
-                }
-                promise.updateIfEmpty(new Throw<ZkVersion>(
-                        KeeperException.create(KeeperException.Code.get(rc))));
-                return;
-            }
-        }, null);
-        return promise;
-    }
-
-    public static Future<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));
-        } catch (InterruptedException e) {
-            return Future.exception(FutureUtils.zkException(e, path));
-        }
-        return zkDelete(zk, path, version);
-    }
-
-    /**
-     * Delete the given <i>path</i> from zookeeper.
-     *
-     * @param zk
-     *          zookeeper client
-     * @param path
-     *          path to delete
-     * @param version
-     *          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>();
-        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));
-                    return;
-                }
-                promise.updateIfEmpty(new Throw<Void>(
-                        KeeperException.create(KeeperException.Code.get(rc))));
-                return;
-            }
-        }, null);
-        return promise;
-    }
-
-    /**
-     * Delete the given <i>path</i> from zookeeper.
-     *
-     * @param zkc
-     *          zookeeper client
-     * @param path
-     *          path to delete
-     * @param version
-     *          version used to set data
-     * @return future representing if the delete is successful. Return true if the node is deleted,
-     * false if the node doesn't exist, otherwise future will throw exception
-     *
-     */
-    public static Future<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));
-        } catch (InterruptedException e) {
-            return Future.exception(FutureUtils.zkException(e, path));
-        }
-        final Promise<Boolean> promise = new Promise<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);
-                } else if (KeeperException.Code.NONODE.intValue() == rc) {
-                    promise.setValue(false);
-                } else {
-                    promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
-                }
-            }
-        }, null);
-        return promise;
-    }
-
-    public static Future<Void> asyncClose(@Nullable AsyncCloseable closeable,
-                                          boolean swallowIOException) {
-        if (null == closeable) {
-            return Future.Void();
-        } else if (swallowIOException) {
-            return FutureUtils.ignore(closeable.asyncClose());
-        } else {
-            return closeable.asyncClose();
-        }
-    }
-
-    /**
-     * Sync zookeeper client on given <i>path</i>.
-     *
-     * @param zkc
-     *          zookeeper client
-     * @param path
-     *          path to sync
-     * @return zookeeper client after sync
-     * @throws IOException
-     */
-    public static ZooKeeper sync(ZooKeeperClient zkc, String path) throws IOException {
-        ZooKeeper zk;
-        try {
-            zk = zkc.get();
-        } catch (InterruptedException e) {
-            throw new DLInterruptedException("Interrupted on checking if log " + path + " exists", e);
-        }
-        final CountDownLatch syncLatch = new CountDownLatch(1);
-        final AtomicInteger syncResult = new AtomicInteger(0);
-        zk.sync(path, new AsyncCallback.VoidCallback() {
-            @Override
-            public void processResult(int rc, String path, Object ctx) {
-                syncResult.set(rc);
-                syncLatch.countDown();
-            }
-        }, null);
-        try {
-            syncLatch.await();
-        } catch (InterruptedException e) {
-            throw new DLInterruptedException("Interrupted on syncing zookeeper connection", e);
-        }
-        if (KeeperException.Code.OK.intValue() != syncResult.get()) {
-            throw new ZKException("Error syncing zookeeper connection ",
-                    KeeperException.Code.get(syncResult.get()));
-        }
-        return zk;
-    }
-
-    /**
-     * Close a closeable.
-     *
-     * @param closeable
-     *          closeable to close
-     */
-    public static void close(@Nullable Closeable closeable) {
-        if (null == closeable) {
-            return;
-        }
-        try {
-            Closeables.close(closeable, true);
-        } catch (IOException e) {
-            // no-op. the exception is swallowed.
-        }
-    }
-
-    /**
-     * Close an async closeable.
-     *
-     * @param closeable
-     *          closeable to close
-     */
-    public static void close(@Nullable AsyncCloseable closeable)
-            throws IOException {
-        if (null == closeable) {
-            return;
-        }
-        FutureUtils.result(closeable.asyncClose());
-    }
-
-    /**
-     * Close an async closeable.
-     *
-     * @param closeable
-     *          closeable to close
-     */
-    public static void closeQuietly(@Nullable AsyncCloseable closeable) {
-        if (null == closeable) {
-            return;
-        }
-        try {
-            FutureUtils.result(closeable.asyncClose());
-        } catch (IOException e) {
-            // no-op. the exception is swallowed.
-        }
-    }
-
-    /**
-     * Close the closeables in sequence.
-     *
-     * @param closeables
-     *          closeables to close
-     * @return future represents the close future
-     */
-    public static Future<Void> closeSequence(ExecutorService executorService,
-                                             AsyncCloseable... closeables) {
-        return closeSequence(executorService, false, closeables);
-    }
-
-    /**
-     * Close the closeables in sequence and ignore errors during closing.
-     *
-     * @param executorService executor to execute closeable
-     * @param ignoreCloseError whether to ignore errors during closing
-     * @param closeables list of closeables
-     * @return future represents the close future.
-     */
-    public static Future<Void> closeSequence(ExecutorService executorService,
-                                             boolean ignoreCloseError,
-                                             AsyncCloseable... closeables) {
-        List<AsyncCloseable> closeableList = Lists.newArrayListWithExpectedSize(closeables.length);
-        for (AsyncCloseable closeable : closeables) {
-            if (null == closeable) {
-                closeableList.add(AsyncCloseable.NULL);
-            } else {
-                closeableList.add(closeable);
-            }
-        }
-        return FutureUtils.processList(
-                closeableList,
-                ignoreCloseError ? AsyncCloseable.CLOSE_FUNC_IGNORE_ERRORS : AsyncCloseable.CLOSE_FUNC,
-                executorService).map(VoidFunctions.LIST_TO_VOID_FUNC);
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/package-info.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/util/package-info.java
deleted file mode 100644
index 193b814..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/util/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Utils
- */
-package com.twitter.distributedlog.util;
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/DefaultZKOp.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/DefaultZKOp.java
deleted file mode 100644
index 78292e9..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/DefaultZKOp.java
+++ /dev/null
@@ -1,55 +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 com.twitter.distributedlog.zk;
-
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
-
-import javax.annotation.Nullable;
-
-/**
- * Default zookeeper operation. No action on commiting or aborting.
- */
-public class DefaultZKOp extends ZKOp {
-
-    public static DefaultZKOp of(Op op, OpListener<Void> listener) {
-        return new DefaultZKOp(op, listener);
-    }
-
-    private final OpListener<Void> listener;
-
-    private DefaultZKOp(Op op, @Nullable OpListener<Void> opListener) {
-        super(op);
-        this.listener = opListener;
-    }
-
-    @Override
-    protected void commitOpResult(OpResult opResult) {
-        if (null != listener) {
-            listener.onCommit(null);
-        }
-    }
-
-    @Override
-    protected void abortOpResult(Throwable t, OpResult opResult) {
-        if (null != listener) {
-            listener.onAbort(t);
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/LimitedPermitManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/LimitedPermitManager.java
deleted file mode 100644
index 78ff0a2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/LimitedPermitManager.java
+++ /dev/null
@@ -1,195 +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 com.twitter.distributedlog.zk;
-
-import com.twitter.distributedlog.util.PermitManager;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.Semaphore;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Manager to control all the log segments rolling.
- */
-public class LimitedPermitManager implements PermitManager, Runnable, Watcher {
-
-    static final Logger LOG = LoggerFactory.getLogger(LimitedPermitManager.class);
-
-    static enum PermitState {
-        ALLOWED, DISALLOWED, DISABLED
-    }
-
-    class EpochPermit implements Permit {
-
-        final PermitState state;
-        final int epoch;
-
-        EpochPermit(PermitState state) {
-            this.state = state;
-            this.epoch = LimitedPermitManager.this.epoch.get();
-        }
-
-        int getEpoch() {
-            return epoch;
-        }
-
-        @Override
-        public boolean isAllowed() {
-            return PermitState.ALLOWED == state;
-        }
-    }
-
-    boolean enablePermits = true;
-    final Semaphore semaphore;
-    final int period;
-    final TimeUnit timeUnit;
-    final ScheduledExecutorService executorService;
-    final AtomicInteger epoch = new AtomicInteger(0);
-    private StatsLogger statsLogger = null;
-    private Gauge<Number> outstandingGauge = null;
-
-    public LimitedPermitManager(int concurrency, int period, TimeUnit timeUnit,
-                                ScheduledExecutorService executorService) {
-        this(concurrency, period, timeUnit, executorService, NullStatsLogger.INSTANCE);
-    }
-
-    public LimitedPermitManager(final int concurrency, int period, TimeUnit timeUnit,
-            ScheduledExecutorService executorService, StatsLogger statsLogger) {
-        if (concurrency > 0) {
-            this.semaphore = new Semaphore(concurrency);
-        } else {
-            this.semaphore = null;
-        }
-        this.period = period;
-        this.timeUnit = timeUnit;
-        this.executorService = executorService;
-        this.statsLogger = statsLogger;
-        this.outstandingGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return null == semaphore ? 0 : concurrency - semaphore.availablePermits();
-            }
-        };
-        this.statsLogger.scope("permits").registerGauge("outstanding", this.outstandingGauge);
-    }
-
-    @Override
-    synchronized public Permit acquirePermit() {
-        if (!enablePermits) {
-            return new EpochPermit(PermitState.DISABLED);
-        }
-        if (null != semaphore) {
-            return semaphore.tryAcquire() ? new EpochPermit(PermitState.ALLOWED) :
-                    new EpochPermit(PermitState.DISALLOWED);
-        } else {
-            return new EpochPermit(PermitState.ALLOWED);
-        }
-    }
-
-    @Override
-    synchronized public void releasePermit(Permit permit) {
-        if (null != semaphore && permit.isAllowed()) {
-            if (period <= 0) {
-                semaphore.release();
-            } else {
-                try {
-                    executorService.schedule(this, period, timeUnit);
-                } catch (RejectedExecutionException ree) {
-                    LOG.warn("Failed on scheduling releasing permit in given period ({}ms)." +
-                            " Release it immediately : ", timeUnit.toMillis(period), ree);
-                    semaphore.release();
-                }
-            }
-        }
-    }
-
-    @Override
-    synchronized public boolean disallowObtainPermits(Permit permit) {
-        if (!(permit instanceof EpochPermit)) {
-            return false;
-        }
-        if (epoch.getAndIncrement() == ((EpochPermit)permit).getEpoch()) {
-            this.enablePermits = false;
-            LOG.info("EnablePermits = {}, Epoch = {}.", this.enablePermits, epoch.get());
-            return true;
-        } else {
-            return false;
-        }
-    }
-
-    @Override
-    public void close() {
-        unregisterGauge();
-    }
-
-    @Override
-    synchronized public boolean allowObtainPermits() {
-        forceSetAllowPermits(true);
-        return true;
-    }
-
-    synchronized void forceSetAllowPermits(boolean allowPermits) {
-        epoch.getAndIncrement();
-        this.enablePermits = allowPermits;
-        LOG.info("EnablePermits = {}, Epoch = {}.", this.enablePermits, epoch.get());
-    }
-
-    @Override
-    public void run() {
-        semaphore.release();
-    }
-
-    @Override
-    public void process(WatchedEvent event) {
-        if (event.getType().equals(Event.EventType.None)) {
-            switch (event.getState()) {
-            case SyncConnected:
-                forceSetAllowPermits(true);
-                break;
-            case Disconnected:
-                forceSetAllowPermits(false);
-                break;
-            case Expired:
-                forceSetAllowPermits(false);
-                break;
-            default:
-                break;
-            }
-        }
-    }
-
-    public void unregisterGauge() {
-        if(this.statsLogger != null && this.outstandingGauge != null) {
-            this.statsLogger.scope("permits").unregisterGauge("outstanding", this.outstandingGauge);
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKOp.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKOp.java
deleted file mode 100644
index 5675574..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKOp.java
+++ /dev/null
@@ -1,63 +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 com.twitter.distributedlog.zk;
-
-import com.twitter.distributedlog.util.Transaction;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
-
-import javax.annotation.Nullable;
-
-/**
- * ZooKeeper Transaction Operation
- */
-public abstract class ZKOp implements Transaction.Op<Object> {
-
-    protected final Op op;
-
-    protected ZKOp(Op op) {
-        this.op = op;
-    }
-
-    public Op getOp() {
-        return op;
-    }
-
-    @Override
-    public void commit(Object r) {
-        assert(r instanceof OpResult);
-        commitOpResult((OpResult) r);
-    }
-
-    protected abstract void commitOpResult(OpResult opResult);
-
-    @Override
-    public void abort(Throwable t, Object r) {
-        assert(r instanceof OpResult);
-        abortOpResult(t, (OpResult) r);
-    }
-
-    /**
-     * Abort the operation with exception <i>t</i> and result <i>opResult</i>.
-     *
-     * @param t the reason to abort the operation
-     * @param opResult the result of operation
-     */
-    protected abstract void abortOpResult(Throwable t,
-                                          @Nullable OpResult opResult);
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKTransaction.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKTransaction.java
deleted file mode 100644
index 57f9aa3..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKTransaction.java
+++ /dev/null
@@ -1,103 +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 com.twitter.distributedlog.zk;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Transaction;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-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
- */
-public class ZKTransaction implements Transaction<Object>, AsyncCallback.MultiCallback {
-
-    private final ZooKeeperClient zkc;
-    private final List<ZKOp> ops;
-    private final List<org.apache.zookeeper.Op> zkOps;
-    private final Promise<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>();
-    }
-
-    @Override
-    public void addOp(Op<Object> operation) {
-        if (done.get()) {
-            throw new IllegalStateException("Add an operation to a finished transaction");
-        }
-        assert(operation instanceof ZKOp);
-        ZKOp zkOp = (ZKOp) operation;
-        this.ops.add(zkOp);
-        this.zkOps.add(zkOp.getOp());
-    }
-
-    @Override
-    public Future<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, ""));
-        } catch (InterruptedException e) {
-            result.setException(FutureUtils.zkException(e, ""));
-        }
-        return result;
-    }
-
-    @Override
-    public void abort(Throwable cause) {
-        if (!done.compareAndSet(false, true)) {
-            return;
-        }
-        for (int i = 0; i < ops.size(); i++) {
-            ops.get(i).abortOpResult(cause, null);
-        }
-        FutureUtils.setException(result, cause);
-    }
-
-    @Override
-    public void processResult(int rc, String path, Object ctx, List<OpResult> results) {
-        if (KeeperException.Code.OK.intValue() == rc) { // transaction succeed
-            for (int i = 0; i < ops.size(); i++) {
-                ops.get(i).commitOpResult(results.get(i));
-            }
-            FutureUtils.setValue(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);
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKVersionedSetOp.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKVersionedSetOp.java
deleted file mode 100644
index 5b788e2..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKVersionedSetOp.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 com.twitter.distributedlog.zk;
-
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
-
-import javax.annotation.Nullable;
-
-/**
- * ZooKeeper Operation that plays with {@link org.apache.bookkeeper.versioning.Version}
- */
-public class ZKVersionedSetOp extends ZKOp {
-
-    private final OpListener<Version> listener;
-
-    public ZKVersionedSetOp(Op op,
-                            @Nullable OpListener<Version> opListener) {
-        super(op);
-        this.listener = opListener;
-    }
-
-    @Override
-    protected void commitOpResult(OpResult opResult) {
-        assert(opResult instanceof OpResult.SetDataResult);
-        OpResult.SetDataResult setDataResult = (OpResult.SetDataResult) opResult;
-        if (null != listener) {
-            listener.onCommit(new ZkVersion(setDataResult.getStat().getVersion()));
-        }
-    }
-
-    @Override
-    protected void abortOpResult(Throwable t,
-                                 @Nullable OpResult opResult) {
-        Throwable cause;
-        if (null == opResult) {
-            cause = t;
-        } else {
-            assert (opResult instanceof OpResult.ErrorResult);
-            OpResult.ErrorResult errorResult = (OpResult.ErrorResult) opResult;
-            if (KeeperException.Code.OK.intValue() == errorResult.getErr()) {
-                cause = t;
-            } else {
-                cause = KeeperException.create(KeeperException.Code.get(errorResult.getErr()));
-            }
-        }
-        if (null != listener) {
-            listener.onAbort(cause);
-        }
-    }
-
-}
diff --git a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKWatcherManager.java b/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKWatcherManager.java
deleted file mode 100644
index 8ef33ea..0000000
--- a/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKWatcherManager.java
+++ /dev/null
@@ -1,239 +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 com.twitter.distributedlog.zk;
-
-import com.twitter.distributedlog.ZooKeeperClient;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.HashSet;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.atomic.AtomicInteger;
-
-/**
- * Watcher Manager to manage watchers.
- * <h3>Metrics</h3>
- * <ul>
- * <li> `total_watches`: total number of watches that managed by this watcher manager.
- * <li> `num_child_watches`: number of paths that are watched for children changes by this watcher manager.
- * </ul>
- */
-public class ZKWatcherManager implements Watcher {
-
-    static final Logger logger = LoggerFactory.getLogger(ZKWatcherManager.class);
-
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    public static class Builder {
-
-        private String _name;
-        private StatsLogger _statsLogger;
-        private ZooKeeperClient _zkc;
-
-        public Builder name(String name) {
-            this._name = name;
-            return this;
-        }
-
-        public Builder zkc(ZooKeeperClient zkc) {
-            this._zkc = zkc;
-            return this;
-        }
-
-        public Builder statsLogger(StatsLogger statsLogger) {
-            this._statsLogger = statsLogger;
-            return this;
-        }
-
-        public ZKWatcherManager build() {
-            return new ZKWatcherManager(_name, _zkc, _statsLogger);
-        }
-    }
-
-    private final String name;
-    private final ZooKeeperClient zkc;
-    private final StatsLogger statsLogger;
-    // Gauges and their labels
-    private final Gauge<Number> totalWatchesGauge;
-    private static final String totalWatchesGauageLabel = "total_watches";
-    private final Gauge<Number> numChildWatchesGauge;
-    private static final String numChildWatchesGauageLabel = "num_child_watches";
-
-    protected final ConcurrentMap<String, Set<Watcher>> childWatches;
-    protected final AtomicInteger allWatchesGauge;
-
-    private ZKWatcherManager(String name,
-                             ZooKeeperClient zkc,
-                             StatsLogger statsLogger) {
-        this.name = name;
-        this.zkc = zkc;
-        this.statsLogger = statsLogger;
-
-        // watches
-        this.childWatches = new ConcurrentHashMap<String, Set<Watcher>>();
-        this.allWatchesGauge = new AtomicInteger(0);
-
-        // stats
-        totalWatchesGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return allWatchesGauge.get();
-            }
-        };
-        this.statsLogger.registerGauge(totalWatchesGauageLabel, totalWatchesGauge);
-
-        numChildWatchesGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return childWatches.size();
-            }
-        };
-
-        this.statsLogger.registerGauge(numChildWatchesGauageLabel, numChildWatchesGauge);
-    }
-
-    public Watcher registerChildWatcher(String path, Watcher watcher) {
-        Set<Watcher> watchers = childWatches.get(path);
-        if (null == watchers) {
-            Set<Watcher> newWatchers = new HashSet<Watcher>();
-            Set<Watcher> oldWatchers = childWatches.putIfAbsent(path, newWatchers);
-            watchers = (null == oldWatchers) ? newWatchers : oldWatchers;
-        }
-        synchronized (watchers) {
-            if (childWatches.get(path) == watchers) {
-                if (watchers.add(watcher)) {
-                    allWatchesGauge.incrementAndGet();
-                }
-            } else {
-                logger.warn("Watcher set for path {} has been changed while registering child watcher {}.",
-                        path, watcher);
-            }
-        }
-        return this;
-    }
-
-    public void unregisterChildWatcher(String path, Watcher watcher, boolean removeFromServer) {
-        Set<Watcher> watchers = childWatches.get(path);
-        if (null == watchers) {
-            logger.warn("No watchers found on path {} while unregistering child watcher {}.",
-                    path, watcher);
-            return;
-        }
-        synchronized (watchers) {
-            if (watchers.remove(watcher)) {
-                allWatchesGauge.decrementAndGet();
-            } else {
-                logger.warn("Remove a non-registered child watcher {} from path {}", watcher, path);
-            }
-            if (watchers.isEmpty()) {
-                // best-efforts to remove watches
-                try {
-                    if (null != zkc && removeFromServer) {
-                        zkc.get().removeWatches(path, this, WatcherType.Children, true, new AsyncCallback.VoidCallback() {
-                            @Override
-                            public void processResult(int rc, String path, Object ctx) {
-                                if (KeeperException.Code.OK.intValue() == rc) {
-                                    logger.debug("Successfully removed children watches from {}", path);
-                                } else {
-                                    logger.debug("Encountered exception on removing children watches from {}",
-                                            path, KeeperException.create(KeeperException.Code.get(rc)));
-                                }
-                            }
-                        }, null);
-                    }
-                } catch (InterruptedException e) {
-                    logger.debug("Encountered exception on removing watches from {}", path, e);
-                } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
-                    logger.debug("Encountered exception on removing watches from {}", path, e);
-                }
-                childWatches.remove(path, watchers);
-            }
-        }
-    }
-
-    public void unregisterGauges() {
-        this.statsLogger.unregisterGauge(totalWatchesGauageLabel, totalWatchesGauge);
-        this.statsLogger.unregisterGauge(numChildWatchesGauageLabel, numChildWatchesGauge);
-    }
-
-    @Override
-    public void process(WatchedEvent event) {
-        switch (event.getType()) {
-            case None:
-                handleKeeperStateEvent(event);
-                break;
-            case NodeChildrenChanged:
-                handleChildWatchEvent(event);
-                break;
-            default:
-                break;
-        }
-    }
-
-    private void handleKeeperStateEvent(WatchedEvent event) {
-        Set<Watcher> savedAllWatches = new HashSet<Watcher>(allWatchesGauge.get());
-        for (Set<Watcher> watcherSet : childWatches.values()) {
-            synchronized (watcherSet) {
-                savedAllWatches.addAll(watcherSet);
-            }
-        }
-        for (Watcher watcher : savedAllWatches) {
-            watcher.process(event);
-        }
-    }
-
-    private void handleChildWatchEvent(WatchedEvent event) {
-        String path = event.getPath();
-        if (null == path) {
-            logger.warn("Received zookeeper watch event with null path : {}", event);
-            return;
-        }
-        Set<Watcher> watchers = childWatches.get(path);
-        if (null == watchers) {
-            return;
-        }
-        Set<Watcher> watchersToFire;
-        synchronized (watchers) {
-            watchersToFire = new HashSet<Watcher>(watchers.size());
-            watchersToFire.addAll(watchers);
-        }
-        for (Watcher watcher : watchersToFire) {
-            watcher.process(event);
-        }
-    }
-}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
new file mode 100644
index 0000000..1d96f0e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
@@ -0,0 +1,198 @@
+/**
+ * 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.base.Preconditions;
+
+import java.io.IOException;
+import java.io.InputStream;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AppendOnlyStreamReader extends InputStream {
+    static final Logger LOG = LoggerFactory.getLogger(AppendOnlyStreamReader.class);
+
+    private LogRecordWithInputStream currentLogRecord = null;
+    private final DistributedLogManager dlm;
+    private LogReader reader;
+    private long currentPosition;
+    private static final int SKIP_BUFFER_SIZE = 512;
+
+    // Cache the input stream for a log record.
+    private static class LogRecordWithInputStream {
+        private final InputStream payloadStream;
+        private final LogRecordWithDLSN logRecord;
+
+        LogRecordWithInputStream(LogRecordWithDLSN logRecord) {
+            Preconditions.checkNotNull(logRecord);
+
+            LOG.debug("Got record dlsn = {}, txid = {}, len = {}",
+                new Object[] {logRecord.getDlsn(), logRecord.getTransactionId(), logRecord.getPayload().length});
+
+            this.logRecord = logRecord;
+            this.payloadStream = logRecord.getPayLoadInputStream();
+        }
+
+        InputStream getPayLoadInputStream() {
+            return payloadStream;
+        }
+
+        LogRecordWithDLSN getLogRecord() {
+            return logRecord;
+        }
+
+        // The last txid of the log record is the position of the next byte in the stream.
+        // Subtract length to get starting offset.
+        long getOffset() {
+            return logRecord.getTransactionId() - logRecord.getPayload().length;
+        }
+    }
+
+    /**
+     * Construct ledger input stream
+     *
+     * @param dlm the Distributed Log Manager to access the stream
+     */
+    AppendOnlyStreamReader(DistributedLogManager dlm)
+        throws IOException {
+        this.dlm = dlm;
+        reader = dlm.getInputStream(0);
+        currentPosition = 0;
+    }
+
+    /**
+     * Get input stream representing next entry in the
+     * ledger.
+     *
+     * @return input stream, or null if no more entries
+     */
+    private LogRecordWithInputStream nextLogRecord() throws IOException {
+        return nextLogRecord(reader);
+    }
+
+    private static LogRecordWithInputStream nextLogRecord(LogReader reader) throws IOException {
+        LogRecordWithDLSN record = reader.readNext(false);
+
+        if (null != record) {
+            return new LogRecordWithInputStream(record);
+        } else {
+            record = reader.readNext(false);
+            if (null != record) {
+                return new LogRecordWithInputStream(record);
+            } else {
+                LOG.debug("No record");
+                return null;
+            }
+        }
+    }
+
+    @Override
+    public int read() throws IOException {
+        byte[] b = new byte[1];
+        if (read(b, 0, 1) != 1) {
+            return -1;
+        } else {
+            return b[0];
+        }
+    }
+
+    @Override
+    public int read(byte[] b, int off, int len) throws IOException {
+        int read = 0;
+        if (currentLogRecord == null) {
+            currentLogRecord = nextLogRecord();
+            if (currentLogRecord == null) {
+                return read;
+            }
+        }
+
+        while (read < len) {
+            int thisread = currentLogRecord.getPayLoadInputStream().read(b, off + read, (len - read));
+            if (thisread == -1) {
+                currentLogRecord = nextLogRecord();
+                if (currentLogRecord == null) {
+                    return read;
+                }
+            } else {
+                LOG.debug("Offset saved = {}, persisted = {}",
+                    currentPosition, currentLogRecord.getLogRecord().getTransactionId());
+                currentPosition += thisread;
+                read += thisread;
+            }
+        }
+        return read;
+    }
+
+    /**
+     * Position the reader at the given offset. If we fail to skip to the desired position
+     * and don't hit end of stream, return false.
+     *
+     * @throws org.apache.distributedlog.exceptions.EndOfStreamException if we attempt to
+     *         skip past the end of the stream.
+     */
+    public boolean skipTo(long position) throws IOException {
+
+        // No need to skip anywhere.
+        if (position == position()) {
+            return true;
+        }
+
+        LogReader skipReader = dlm.getInputStream(position);
+        LogRecordWithInputStream logRecord = null;
+        try {
+            logRecord = nextLogRecord(skipReader);
+        } catch (IOException ex) {
+            skipReader.close();
+            throw ex;
+        }
+
+        if (null == logRecord) {
+            return false;
+        }
+
+        // We may end up with a reader positioned *before* the requested position if
+        // we're near the tail and the writer is still active, or if the desired position
+        // is not at a log record payload boundary.
+        // Transaction ID gives us the starting position of the log record. Read ahead
+        // if necessary.
+        currentPosition = logRecord.getOffset();
+        currentLogRecord = logRecord;
+        LogReader oldReader = reader;
+        reader = skipReader;
+
+        // Close the oldreader after swapping AppendOnlyStreamReader state. Close may fail
+        // and we need to make sure it leaves AppendOnlyStreamReader in a consistent state.
+        oldReader.close();
+
+        byte[] skipBuffer = new byte[SKIP_BUFFER_SIZE];
+        while (currentPosition < position) {
+            long bytesToRead = Math.min(position - currentPosition, SKIP_BUFFER_SIZE);
+            long bytesRead = read(skipBuffer, 0, (int)bytesToRead);
+            if (bytesRead < bytesToRead) {
+                return false;
+            }
+        }
+
+        return true;
+    }
+
+    public long position() {
+        return currentPosition;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
new file mode 100644
index 0000000..8278c68
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
@@ -0,0 +1,107 @@
+/**
+ * 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 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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class AppendOnlyStreamWriter implements Closeable {
+    static final Logger LOG = LoggerFactory.getLogger(AppendOnlyStreamWriter.class);
+
+    // Use a 1-length array to satisfy Java's inner class reference rules. Use primitive
+    // type because synchronized block is needed anyway.
+    final long[] syncPos = new long[1];
+    BKAsyncLogWriter logWriter;
+    long requestPos = 0;
+
+    public AppendOnlyStreamWriter(BKAsyncLogWriter logWriter, long pos) {
+        LOG.debug("initialize at position {}", pos);
+        this.logWriter = logWriter;
+        this.syncPos[0] = pos;
+        this.requestPos = pos;
+    }
+
+    public Future<DLSN> write(byte[] data) {
+        requestPos += data.length;
+        Future<DLSN> writeResult = logWriter.write(new LogRecord(requestPos, data));
+        return writeResult.addEventListener(new WriteCompleteListener(requestPos));
+    }
+
+    public void force(boolean metadata) throws IOException {
+        long pos = 0;
+        try {
+            pos = Await.result(logWriter.flushAndCommit());
+        } catch (IOException ioe) {
+            throw ioe;
+        } catch (Exception ex) {
+            LOG.error("unexpected exception in AppendOnlyStreamWriter.force ", ex);
+            throw new UnexpectedException("unexpected exception in AppendOnlyStreamWriter.force", ex);
+        }
+        synchronized (syncPos) {
+            syncPos[0] = pos;
+        }
+    }
+
+    public long position() {
+        synchronized (syncPos) {
+            return syncPos[0];
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        logWriter.closeAndComplete();
+    }
+
+    public void markEndOfStream() throws IOException {
+        try {
+            Await.result(logWriter.markEndOfStream());
+        } catch (IOException ioe) {
+            throw ioe;
+        } catch (Exception ex) {
+            throw new UnexpectedException("Mark end of stream hit unexpected exception", ex);
+        }
+    }
+
+    class WriteCompleteListener implements FutureEventListener<DLSN> {
+        private final long position;
+        public WriteCompleteListener(long position) {
+            this.position = position;
+        }
+        @Override
+        public void onSuccess(DLSN response) {
+            synchronized (syncPos) {
+                if (position > syncPos[0]) {
+                    syncPos[0] = position;
+                }
+            }
+        }
+        @Override
+        public void onFailure(Throwable cause) {
+            // Handled at the layer above
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java
new file mode 100644
index 0000000..e3ace05
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.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;
+
+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 {
+
+    /**
+     * Get stream name that the reader reads from.
+     *
+     * @return stream name.
+     */
+    public String getStreamName();
+
+    /**
+     * 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();
+
+    /**
+     * Read next <i>numEntries</i> entries. The future is only satisfied with non-empty list
+     * of entries. It doesn't block until returning exact <i>numEntries</i>. It is a best effort
+     * call.
+     *
+     * @param numEntries
+     *          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);
+
+    /**
+     * 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>.
+     * The only exception is if there isn't any new entries written within <i>waitTime</i>, it would
+     * wait until new entries are available.
+     *
+     * @param numEntries
+     *          max entries to return
+     * @param waitTime
+     *          maximum wait time if there are entries already for read
+     * @param timeUnit
+     *          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);
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
new file mode 100644
index 0000000..53b393b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
@@ -0,0 +1,70 @@
+/**
+ * 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 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 {
+
+    /**
+     * Get the last committed transaction id.
+     *
+     * @return last committed transaction id.
+     */
+    public long getLastTxId();
+
+    /**
+     * Write a log record to the stream.
+     *
+     * @param record single log record
+     * @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);
+
+    /**
+     * Write log records to the stream in bulk. Each future in the list represents the result of
+     * one write operation. The size of the result list is equal to the size of the input list.
+     * Buffers are written in order, and the list of result futures has the same order.
+     *
+     * @param record set of log records
+     * @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);
+
+    /**
+     * Truncate the log until <i>dlsn</i>.
+     *
+     * @param dlsn
+     *          dlsn to truncate until.
+     * @return A Future indicates whether the operation succeeds or not, or an exception
+     * if the truncation fails.
+     */
+    public Future<Boolean> truncate(DLSN dlsn);
+
+    /**
+     * Get the name of the stream this writer writes data to
+     */
+    public String getStreamName();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java
new file mode 100644
index 0000000..c12bd10
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java
@@ -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.
+ */
+package org.apache.distributedlog;
+
+public interface AsyncNotification {
+    /**
+     * Triggered when the background activity encounters an exception
+     *
+     * @param reason the exception that encountered.
+     */
+    void notifyOnError(Throwable reason);
+
+    /**
+     *  Triggered when the background activity completes an operation
+     */
+    void notifyOnOperationComplete();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
new file mode 100644
index 0000000..4a2ef30
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
@@ -0,0 +1,555 @@
+/**
+ * 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 org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.io.Abortable;
+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.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;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+abstract class BKAbstractLogWriter implements Closeable, AsyncCloseable, Abortable, AsyncAbortable {
+    static final Logger LOG = LoggerFactory.getLogger(BKAbstractLogWriter.class);
+
+    protected final DistributedLogConfiguration conf;
+    private final DynamicDistributedLogConfiguration dynConf;
+    protected final BKDistributedLogManager bkDistributedLogManager;
+
+    // States
+    private Promise<Void> closePromise = null;
+    private volatile boolean forceRolling = false;
+    private boolean forceRecovery = false;
+
+    // Truncation Related
+    private Future<List<LogSegmentMetadata>> lastTruncationAttempt = null;
+    @VisibleForTesting
+    private Long minTimestampToKeepOverride = null;
+
+    // Log Segment Writers
+    protected BKLogSegmentWriter segmentWriter = null;
+    protected Future<BKLogSegmentWriter> segmentWriterFuture = null;
+    protected BKLogSegmentWriter allocatedSegmentWriter = null;
+    protected BKLogWriteHandler writeHandler = null;
+
+    BKAbstractLogWriter(DistributedLogConfiguration conf,
+                        DynamicDistributedLogConfiguration dynConf,
+                        BKDistributedLogManager bkdlm) {
+        this.conf = conf;
+        this.dynConf = dynConf;
+        this.bkDistributedLogManager = bkdlm;
+        LOG.debug("Initial retention period for {} : {}", bkdlm.getStreamName(),
+                TimeUnit.MILLISECONDS.convert(dynConf.getRetentionPeriodHours(), TimeUnit.HOURS));
+    }
+
+    // manage write handler
+
+    synchronized protected BKLogWriteHandler getCachedWriteHandler() {
+        return writeHandler;
+    }
+
+    protected BKLogWriteHandler getWriteHandler() throws IOException {
+        BKLogWriteHandler writeHandler = createAndCacheWriteHandler();
+        writeHandler.checkMetadataException();
+        return writeHandler;
+    }
+
+    protected BKLogWriteHandler createAndCacheWriteHandler()
+            throws IOException {
+        synchronized (this) {
+            if (writeHandler != null) {
+                return writeHandler;
+            }
+        }
+        // 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));
+        boolean success = false;
+        try {
+            synchronized (this) {
+                if (writeHandler == null) {
+                    writeHandler = newHandler;
+                    success = true;
+                }
+                return writeHandler;
+            }
+        } finally {
+            if (!success) {
+                newHandler.asyncAbort();
+            }
+        }
+    }
+
+    // manage log segment writers
+
+    protected synchronized BKLogSegmentWriter getCachedLogWriter() {
+        return segmentWriter;
+    }
+
+    protected synchronized Future<BKLogSegmentWriter> getCachedLogWriterFuture() {
+        return segmentWriterFuture;
+    }
+
+    protected synchronized void cacheLogWriter(BKLogSegmentWriter logWriter) {
+        this.segmentWriter = logWriter;
+        this.segmentWriterFuture = Future.value(logWriter);
+    }
+
+    protected synchronized BKLogSegmentWriter removeCachedLogWriter() {
+        try {
+            return segmentWriter;
+        } finally {
+            segmentWriter = null;
+            segmentWriterFuture = null;
+        }
+    }
+
+    protected synchronized BKLogSegmentWriter getAllocatedLogWriter() {
+        return allocatedSegmentWriter;
+    }
+
+    protected synchronized void cacheAllocatedLogWriter(BKLogSegmentWriter logWriter) {
+        this.allocatedSegmentWriter = logWriter;
+    }
+
+    protected synchronized BKLogSegmentWriter removeAllocatedLogWriter() {
+        try {
+            return allocatedSegmentWriter;
+        } finally {
+            allocatedSegmentWriter = null;
+        }
+    }
+
+    private Future<Void> asyncCloseAndComplete(boolean shouldThrow) {
+        BKLogSegmentWriter segmentWriter = getCachedLogWriter();
+        BKLogWriteHandler writeHandler = getCachedWriteHandler();
+        if (null != segmentWriter && null != writeHandler) {
+            cancelTruncation();
+            Promise<Void> completePromise = new Promise<Void>();
+            asyncCloseAndComplete(segmentWriter, writeHandler, completePromise, shouldThrow);
+            return completePromise;
+        } else {
+            return closeNoThrow();
+        }
+    }
+
+    private void asyncCloseAndComplete(final BKLogSegmentWriter segmentWriter,
+                                       final BKLogWriteHandler writeHandler,
+                                       final Promise<Void> completePromise,
+                                       final boolean shouldThrow) {
+        writeHandler.completeAndCloseLogSegment(segmentWriter)
+                .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+                    @Override
+                    public void onSuccess(LogSegmentMetadata segment) {
+                        removeCachedLogWriter();
+                        complete(null);
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        LOG.error("Completing Log segments encountered exception", cause);
+                        complete(cause);
+                    }
+
+                    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;
+                            }
+                        });
+                    }
+                });
+    }
+
+    @VisibleForTesting
+    void closeAndComplete() throws IOException {
+        FutureUtils.result(asyncCloseAndComplete(true));
+    }
+
+    protected Future<Void> asyncCloseAndComplete() {
+        return asyncCloseAndComplete(true);
+    }
+
+    @Override
+    public void close() throws IOException {
+        FutureUtils.result(asyncClose());
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        return asyncCloseAndComplete(false);
+    }
+
+    /**
+     * Close the writer and release all the underlying resources
+     */
+    protected Future<Void> closeNoThrow() {
+        Promise<Void> closeFuture;
+        synchronized (this) {
+            if (null != closePromise) {
+                return closePromise;
+            }
+            closeFuture = closePromise = new Promise<Void>();
+        }
+        cancelTruncation();
+        Utils.closeSequence(bkDistributedLogManager.getScheduler(),
+                true, /** ignore close errors **/
+                getCachedLogWriter(),
+                getAllocatedLogWriter(),
+                getCachedWriteHandler()
+        ).proxyTo(closeFuture);
+        return closeFuture;
+    }
+
+    @Override
+    public void abort() throws IOException {
+        FutureUtils.result(asyncAbort());
+    }
+
+    @Override
+    public Future<Void> asyncAbort() {
+        Promise<Void> closeFuture;
+        synchronized (this) {
+            if (null != closePromise) {
+                return closePromise;
+            }
+            closeFuture = closePromise = new Promise<Void>();
+        }
+        cancelTruncation();
+        Abortables.abortSequence(bkDistributedLogManager.getScheduler(),
+                getCachedLogWriter(),
+                getAllocatedLogWriter(),
+                getCachedWriteHandler()).proxyTo(closeFuture);
+        return closeFuture;
+    }
+
+    // used by sync writer
+    protected BKLogSegmentWriter getLedgerWriter(final long startTxId,
+                                                 final boolean allowMaxTxID)
+            throws IOException {
+        Future<BKLogSegmentWriter> logSegmentWriterFuture = asyncGetLedgerWriter(true);
+        BKLogSegmentWriter logSegmentWriter = null;
+        if (null != logSegmentWriterFuture) {
+            logSegmentWriter = FutureUtils.result(logSegmentWriterFuture);
+        }
+        if (null == logSegmentWriter || (shouldStartNewSegment(logSegmentWriter) || forceRolling)) {
+            logSegmentWriter = FutureUtils.result(rollLogSegmentIfNecessary(
+                    logSegmentWriter, startTxId, true /* bestEffort */, allowMaxTxID));
+        }
+        return logSegmentWriter;
+    }
+
+    // used by async writer
+    synchronized protected Future<BKLogSegmentWriter> asyncGetLedgerWriter(boolean resetOnError) {
+        final BKLogSegmentWriter ledgerWriter = getCachedLogWriter();
+        Future<BKLogSegmentWriter> ledgerWriterFuture = getCachedLogWriterFuture();
+        if (null == ledgerWriterFuture || null == ledgerWriter) {
+            return null;
+        }
+
+        // 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;
+            if (ledgerWriter.isLogSegmentInError()) {
+                closeFuture = ledgerWriter.asyncAbort();
+            } else {
+                closeFuture = ledgerWriter.asyncClose();
+            }
+            return closeFuture.flatMap(
+                    new AbstractFunction1<Void, Future<BKLogSegmentWriter>>() {
+                @Override
+                public Future<BKLogSegmentWriter> apply(Void result) {
+                    removeCachedLogWriter();
+
+                    if (ledgerWriter.isLogSegmentInError()) {
+                        return Future.value(null);
+                    }
+
+                    BKLogWriteHandler writeHandler;
+                    try {
+                        writeHandler = getWriteHandler();
+                    } catch (IOException e) {
+                        return Future.exception(e);
+                    }
+                    if (null != writeHandler && forceRecovery) {
+                        return writeHandler.completeAndCloseLogSegment(ledgerWriter)
+                                .map(new AbstractFunction1<LogSegmentMetadata, BKLogSegmentWriter>() {
+                            @Override
+                            public BKLogSegmentWriter apply(LogSegmentMetadata completedLogSegment) {
+                                return null;
+                            }
+                        });
+                    } else {
+                        return Future.value(null);
+                    }
+                }
+            });
+        } else {
+            return ledgerWriterFuture;
+        }
+    }
+
+    boolean shouldStartNewSegment(BKLogSegmentWriter ledgerWriter) throws IOException {
+        BKLogWriteHandler writeHandler = getWriteHandler();
+        return null == ledgerWriter || writeHandler.shouldStartNewSegment(ledgerWriter) || forceRolling;
+    }
+
+    private void truncateLogSegmentsIfNecessary(BKLogWriteHandler writeHandler) {
+        boolean truncationEnabled = false;
+
+        long minTimestampToKeep = 0;
+
+        long retentionPeriodInMillis = TimeUnit.MILLISECONDS.convert(dynConf.getRetentionPeriodHours(), TimeUnit.HOURS);
+        if (retentionPeriodInMillis > 0) {
+            minTimestampToKeep = Utils.nowInMillis() - retentionPeriodInMillis;
+            truncationEnabled = true;
+        }
+
+        if (null != minTimestampToKeepOverride) {
+            minTimestampToKeep = minTimestampToKeepOverride;
+            truncationEnabled = true;
+        }
+
+        // skip scheduling if there is task that's already running
+        //
+        synchronized (this) {
+            if (truncationEnabled && ((lastTruncationAttempt == null) || lastTruncationAttempt.isDefined())) {
+                lastTruncationAttempt = writeHandler.purgeLogSegmentsOlderThanTimestamp(minTimestampToKeep);
+            }
+        }
+    }
+
+    private Future<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) {
+                        cacheLogWriter(newSegmentWriter);
+                        return BoxedUnit.UNIT;
+                    }
+                });
+            }
+        });
+    }
+
+    private Future<BKLogSegmentWriter> closeOldLogSegmentAndStartNewOneWithPermit(
+            final BKLogSegmentWriter oldSegmentWriter,
+            final BKLogWriteHandler writeHandler,
+            final long startTxId,
+            final boolean bestEffort,
+            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());
+                            bkDistributedLogManager.getLogSegmentRollingPermitManager().disallowObtainPermits(switchPermit);
+                            return Future.value(oldSegmentWriter);
+                        }
+                    }
+                    return Future.exception(cause);
+                }
+            }).ensure(new AbstractFunction0<BoxedUnit>() {
+                @Override
+                public BoxedUnit apply() {
+                    bkDistributedLogManager.getLogSegmentRollingPermitManager()
+                            .releasePermit(switchPermit);
+                    return BoxedUnit.UNIT;
+                }
+            });
+        } else {
+            bkDistributedLogManager.getLogSegmentRollingPermitManager().releasePermit(switchPermit);
+            return Future.value(oldSegmentWriter);
+        }
+    }
+
+    private Future<BKLogSegmentWriter> closeOldLogSegmentAndStartNewOne(
+            final BKLogSegmentWriter oldSegmentWriter,
+            final BKLogWriteHandler writeHandler,
+            final long startTxId,
+            final boolean bestEffort,
+            final boolean allowMaxTxID) {
+        // we switch only when we could allocate a new log segment.
+        BKLogSegmentWriter newSegmentWriter = getAllocatedLogWriter();
+        if (null == newSegmentWriter) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Allocating a new log segment from {} for {}.", startTxId,
+                        writeHandler.getFullyQualifiedName());
+            }
+            return writeHandler.asyncStartLogSegment(startTxId, bestEffort, allowMaxTxID)
+                    .flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<BKLogSegmentWriter>>() {
+                        @Override
+                        public Future<BKLogSegmentWriter> apply(BKLogSegmentWriter newSegmentWriter) {
+                            if (null == newSegmentWriter) {
+                                if (bestEffort) {
+                                    return Future.value(oldSegmentWriter);
+                                } else {
+                                    return Future.exception(
+                                            new UnexpectedException("StartLogSegment returns null for bestEffort rolling"));
+                                }
+                            }
+                            cacheAllocatedLogWriter(newSegmentWriter);
+                            if (LOG.isDebugEnabled()) {
+                                LOG.debug("Allocated a new log segment from {} for {}.", startTxId,
+                                        writeHandler.getFullyQualifiedName());
+                            }
+                            return completeOldSegmentAndCacheNewLogSegmentWriter(oldSegmentWriter, newSegmentWriter);
+                        }
+                    });
+        } else {
+            return completeOldSegmentAndCacheNewLogSegmentWriter(oldSegmentWriter, newSegmentWriter);
+        }
+    }
+
+    private Future<BKLogSegmentWriter> completeOldSegmentAndCacheNewLogSegmentWriter(
+            BKLogSegmentWriter oldSegmentWriter,
+            final BKLogSegmentWriter newSegmentWriter) {
+        final Promise<BKLogSegmentWriter> completePromise = new Promise<BKLogSegmentWriter>();
+        // complete the old log segment
+        writeHandler.completeAndCloseLogSegment(oldSegmentWriter)
+                .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+
+                    @Override
+                    public void onSuccess(LogSegmentMetadata value) {
+                        cacheLogWriter(newSegmentWriter);
+                        removeAllocatedLogWriter();
+                        FutureUtils.setValue(completePromise, newSegmentWriter);
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        FutureUtils.setException(completePromise, cause);
+                    }
+                });
+        return completePromise;
+    }
+
+    synchronized protected Future<BKLogSegmentWriter> rollLogSegmentIfNecessary(
+            final BKLogSegmentWriter segmentWriter,
+            long startTxId,
+            boolean bestEffort,
+            boolean allowMaxTxID) {
+        final BKLogWriteHandler writeHandler;
+        try {
+            writeHandler = getWriteHandler();
+        } catch (IOException e) {
+            return Future.exception(e);
+        }
+        Future<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);
+        }
+        return rollPromise.map(new AbstractFunction1<BKLogSegmentWriter, BKLogSegmentWriter>() {
+            @Override
+            public BKLogSegmentWriter apply(BKLogSegmentWriter newSegmentWriter) {
+                if (segmentWriter == newSegmentWriter) {
+                    return newSegmentWriter;
+                }
+                truncateLogSegmentsIfNecessary(writeHandler);
+                return newSegmentWriter;
+            }
+        });
+    }
+
+    protected synchronized void checkClosedOrInError(String operation) throws AlreadyClosedException {
+        if (null != closePromise) {
+            LOG.error("Executing " + operation + " on already closed Log Writer");
+            throw new AlreadyClosedException("Executing " + operation + " on already closed Log Writer");
+        }
+    }
+
+    @VisibleForTesting
+    public void setForceRolling(boolean forceRolling) {
+        this.forceRolling = forceRolling;
+    }
+
+    @VisibleForTesting
+    public synchronized void overRideMinTimeStampToKeep(Long minTimestampToKeepOverride) {
+        this.minTimestampToKeepOverride = minTimestampToKeepOverride;
+    }
+
+    protected synchronized void cancelTruncation() {
+        if (null != lastTruncationAttempt) {
+            FutureUtils.cancel(lastTruncationAttempt);
+            lastTruncationAttempt = null;
+        }
+    }
+
+    @VisibleForTesting
+    public synchronized void setForceRecovery(boolean forceRecovery) {
+        this.forceRecovery = forceRecovery;
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
new file mode 100644
index 0000000..eedfbd6
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
@@ -0,0 +1,751 @@
+/**
+ * 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 com.google.common.base.Optional;
+import com.google.common.base.Stopwatch;
+import com.google.common.base.Ticker;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.IdleReaderException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.exceptions.ReadCancelledException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+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.
+ *
+ * <h3>Metrics</h3>
+ * All the metrics are exposed under `async_reader`.
+ * <ul>
+ * <li> `async_reader`/future_set: opstats. time spent on satisfying futures of read requests.
+ * if it is high, it means that the caller takes time on processing the result of read requests.
+ * The side effect is blocking consequent reads.
+ * <li> `async_reader`/schedule: opstats. time spent on scheduling next reads.
+ * <li> `async_reader`/background_read: opstats. time spent on background reads.
+ * <li> `async_reader`/read_next_exec: opstats. time spent on executing {@link #readNext()}.
+ * <li> `async_reader`/time_between_read_next: opstats. time spent on between two consequent {@link #readNext()}.
+ * if it is high, it means that the caller is slowing down on calling {@link #readNext()}.
+ * <li> `async_reader`/delay_until_promise_satisfied: opstats. total latency for the read requests.
+ * <li> `async_reader`/idle_reader_error: counter. the number idle reader errors.
+ * </ul>
+ */
+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 final String streamName;
+    protected final BKDistributedLogManager bkDistributedLogManager;
+    protected final BKLogReadHandler readHandler;
+    private final AtomicReference<Throwable> lastException = new AtomicReference<Throwable>();
+    private final OrderedScheduler scheduler;
+    private final ConcurrentLinkedQueue<PendingReadRequest> pendingRequests = new ConcurrentLinkedQueue<PendingReadRequest>();
+    private final Object scheduleLock = new Object();
+    private final AtomicLong scheduleCount = new AtomicLong(0);
+    final private Stopwatch scheduleDelayStopwatch;
+    final private Stopwatch readNextDelayStopwatch;
+    private DLSN startDLSN;
+    private ReadAheadEntryReader readAheadReader = null;
+    private int lastPosition = 0;
+    private final boolean positionGapDetectionEnabled;
+    private final int idleErrorThresholdMillis;
+    final ScheduledFuture<?> idleReaderTimeoutTask;
+    private ScheduledFuture<?> backgroundScheduleTask = null;
+    // last process time
+    private final Stopwatch lastProcessTime;
+
+    protected Promise<Void> closeFuture = null;
+
+    private boolean lockStream = false;
+
+    private final boolean returnEndOfStreamRecord;
+
+    private final Runnable BACKGROUND_READ_SCHEDULER = new Runnable() {
+        @Override
+        public void run() {
+            synchronized (scheduleLock) {
+                backgroundScheduleTask = null;
+            }
+            scheduleBackgroundRead();
+        }
+    };
+
+    // State
+    private Entry.Reader currentEntry = null;
+    private LogRecordWithDLSN nextRecord = null;
+
+    // Failure Injector
+    private boolean disableProcessingReadRequests = false;
+
+    // Stats
+    private final OpStatsLogger readNextExecTime;
+    private final OpStatsLogger delayUntilPromiseSatisfied;
+    private final OpStatsLogger timeBetweenReadNexts;
+    private final OpStatsLogger futureSetLatency;
+    private final OpStatsLogger scheduleLatency;
+    private final OpStatsLogger backgroundReaderRunTime;
+    private final Counter idleReaderCheckCount;
+    private final Counter idleReaderCheckIdleReadRequestCount;
+    private final Counter idleReaderCheckIdleReadAheadCount;
+    private final Counter idleReaderError;
+
+    private class PendingReadRequest {
+        private final Stopwatch enqueueTime;
+        private final int numEntries;
+        private final List<LogRecordWithDLSN> records;
+        private final Promise<List<LogRecordWithDLSN>> promise;
+        private final long deadlineTime;
+        private final TimeUnit deadlineTimeUnit;
+
+        PendingReadRequest(int numEntries,
+                           long deadlineTime,
+                           TimeUnit deadlineTimeUnit) {
+            this.numEntries = numEntries;
+            this.enqueueTime = Stopwatch.createStarted();
+            // optimize the space usage for single read.
+            if (numEntries == 1) {
+                this.records = new ArrayList<LogRecordWithDLSN>(1);
+            } else {
+                this.records = new ArrayList<LogRecordWithDLSN>();
+            }
+            this.promise = new Promise<List<LogRecordWithDLSN>>();
+            this.deadlineTime = deadlineTime;
+            this.deadlineTimeUnit = deadlineTimeUnit;
+        }
+
+        Promise<List<LogRecordWithDLSN>> getPromise() {
+            return promise;
+        }
+
+        long elapsedSinceEnqueue(TimeUnit timeUnit) {
+            return enqueueTime.elapsed(timeUnit);
+        }
+
+        void setException(Throwable throwable) {
+            Stopwatch stopwatch = Stopwatch.createStarted();
+            if (promise.updateIfEmpty(new Throw<List<LogRecordWithDLSN>>(throwable))) {
+                futureSetLatency.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+                delayUntilPromiseSatisfied.registerFailedEvent(enqueueTime.elapsed(TimeUnit.MICROSECONDS));
+            }
+        }
+
+        boolean hasReadRecords() {
+            return records.size() > 0;
+        }
+
+        boolean hasReadEnoughRecords() {
+            return records.size() >= numEntries;
+        }
+
+        long getRemainingWaitTime() {
+            if (deadlineTime <= 0L) {
+                return 0L;
+            }
+            return deadlineTime - elapsedSinceEnqueue(deadlineTimeUnit);
+        }
+
+        void addRecord(LogRecordWithDLSN record) {
+            records.add(record);
+        }
+
+        void complete() {
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("{} : Satisfied promise with {} records", readHandler.getFullyQualifiedName(), records.size());
+            }
+            delayUntilPromiseSatisfied.registerSuccessfulEvent(enqueueTime.stop().elapsed(TimeUnit.MICROSECONDS));
+            Stopwatch stopwatch = Stopwatch.createStarted();
+            promise.setValue(records);
+            futureSetLatency.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+        }
+    }
+
+    BKAsyncLogReader(BKDistributedLogManager bkdlm,
+                     OrderedScheduler scheduler,
+                     DLSN startDLSN,
+                     Optional<String> subscriberId,
+                     boolean returnEndOfStreamRecord,
+                     StatsLogger statsLogger) {
+        this.streamName = bkdlm.getStreamName();
+        this.bkDistributedLogManager = bkdlm;
+        this.scheduler = scheduler;
+        this.readHandler = bkDistributedLogManager.createReadHandler(subscriberId,
+                this, true);
+        LOG.debug("Starting async reader at {}", startDLSN);
+        this.startDLSN = startDLSN;
+        this.scheduleDelayStopwatch = Stopwatch.createUnstarted();
+        this.readNextDelayStopwatch = Stopwatch.createStarted();
+        this.positionGapDetectionEnabled = bkdlm.getConf().getPositionGapDetectionEnabled();
+        this.idleErrorThresholdMillis = bkdlm.getConf().getReaderIdleErrorThresholdMillis();
+        this.returnEndOfStreamRecord = returnEndOfStreamRecord;
+
+        // Stats
+        StatsLogger asyncReaderStatsLogger = statsLogger.scope("async_reader");
+        futureSetLatency = asyncReaderStatsLogger.getOpStatsLogger("future_set");
+        scheduleLatency = asyncReaderStatsLogger.getOpStatsLogger("schedule");
+        backgroundReaderRunTime = asyncReaderStatsLogger.getOpStatsLogger("background_read");
+        readNextExecTime = asyncReaderStatsLogger.getOpStatsLogger("read_next_exec");
+        timeBetweenReadNexts = asyncReaderStatsLogger.getOpStatsLogger("time_between_read_next");
+        delayUntilPromiseSatisfied = asyncReaderStatsLogger.getOpStatsLogger("delay_until_promise_satisfied");
+        idleReaderError = asyncReaderStatsLogger.getCounter("idle_reader_error");
+        idleReaderCheckCount = asyncReaderStatsLogger.getCounter("idle_reader_check_total");
+        idleReaderCheckIdleReadRequestCount = asyncReaderStatsLogger.getCounter("idle_reader_check_idle_read_requests");
+        idleReaderCheckIdleReadAheadCount = asyncReaderStatsLogger.getCounter("idle_reader_check_idle_readahead");
+
+        // Lock the stream if requested. The lock will be released when the reader is closed.
+        this.lockStream = false;
+        this.idleReaderTimeoutTask = scheduleIdleReaderTaskIfNecessary();
+        this.lastProcessTime = Stopwatch.createStarted();
+    }
+
+    private ScheduledFuture<?> scheduleIdleReaderTaskIfNecessary() {
+        if (idleErrorThresholdMillis < Integer.MAX_VALUE) {
+            // Dont run the task more than once every seconds (for sanity)
+            long period = Math.max(idleErrorThresholdMillis / 10, 1000);
+            // Except when idle reader threshold is less than a second (tests?)
+            period = Math.min(period, idleErrorThresholdMillis / 5);
+
+            return scheduler.scheduleAtFixedRate(streamName, new Runnable() {
+                @Override
+                public void run() {
+                    PendingReadRequest nextRequest = pendingRequests.peek();
+
+                    idleReaderCheckCount.inc();
+                    if (null == nextRequest) {
+                        return;
+                    }
+
+                    idleReaderCheckIdleReadRequestCount.inc();
+                    if (nextRequest.elapsedSinceEnqueue(TimeUnit.MILLISECONDS) < idleErrorThresholdMillis) {
+                        return;
+                    }
+
+                    ReadAheadEntryReader readAheadReader = getReadAheadReader();
+
+                    // read request has been idle
+                    //   - cache has records but read request are idle,
+                    //     that means notification was missed between readahead and reader.
+                    //   - cache is empty and readahead is idle (no records added for a long time)
+                    idleReaderCheckIdleReadAheadCount.inc();
+                    try {
+                        if (null == readAheadReader || (!hasMoreRecords() &&
+                                readAheadReader.isReaderIdle(idleErrorThresholdMillis, TimeUnit.MILLISECONDS))) {
+                            markReaderAsIdle();
+                            return;
+                        } else if (lastProcessTime.elapsed(TimeUnit.MILLISECONDS) > idleErrorThresholdMillis) {
+                            markReaderAsIdle();;
+                        }
+                    } catch (IOException e) {
+                        setLastException(e);
+                        return;
+                    }
+                }
+            }, period, period, TimeUnit.MILLISECONDS);
+        }
+        return null;
+    }
+
+    synchronized ReadAheadEntryReader getReadAheadReader() {
+        return readAheadReader;
+    }
+
+    void cancelIdleReaderTask() {
+        // Do this after we have checked that the reader was not previously closed
+        try {
+            if (null != idleReaderTimeoutTask) {
+                idleReaderTimeoutTask.cancel(true);
+            }
+        } catch (Exception exc) {
+            LOG.info("{}: Failed to cancel the background idle reader timeout task", readHandler.getFullyQualifiedName());
+        }
+    }
+
+    private void markReaderAsIdle() {
+        idleReaderError.inc();
+        IdleReaderException ire = new IdleReaderException("Reader on stream "
+                + readHandler.getFullyQualifiedName()
+                + " is idle for " + idleErrorThresholdMillis +" ms");
+        setLastException(ire);
+        // cancel all pending reads directly rather than notifying on error
+        // because idle reader could happen on idle read requests that usually means something wrong
+        // in scheduling reads
+        cancelAllPendingReads(ire);
+    }
+
+    protected synchronized void setStartDLSN(DLSN fromDLSN) throws UnexpectedException {
+        if (null != readAheadReader) {
+            throw new UnexpectedException("Could't reset from dlsn after reader already starts reading.");
+        }
+        startDLSN = fromDLSN;
+    }
+
+    @VisibleForTesting
+    public synchronized DLSN getStartDLSN() {
+        return startDLSN;
+    }
+
+    public Future<Void> lockStream() {
+        this.lockStream = true;
+        return readHandler.lockStream();
+    }
+
+    private boolean checkClosedOrInError(String operation) {
+        if (null == lastException.get()) {
+            try {
+                if (null != readHandler && null != getReadAheadReader()) {
+                    getReadAheadReader().checkLastException();
+                }
+
+                bkDistributedLogManager.checkClosedOrInError(operation);
+            } catch (IOException exc) {
+                setLastException(exc);
+            }
+        }
+
+        if (lockStream) {
+            try {
+                readHandler.checkReadLock();
+            } catch (IOException ex) {
+                setLastException(ex);
+            }
+        }
+
+        if (null != lastException.get()) {
+            LOG.trace("Cancelling pending reads");
+            cancelAllPendingReads(lastException.get());
+            return true;
+        }
+
+        return false;
+    }
+
+    private void setLastException(IOException exc) {
+        lastException.compareAndSet(null, exc);
+    }
+
+    @Override
+    public String getStreamName() {
+        return streamName;
+    }
+
+    /**
+     * @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 Future<List<LogRecordWithDLSN>> readBulk(int numEntries) {
+        return readInternal(numEntries, 0, TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public synchronized Future<List<LogRecordWithDLSN>> readBulk(int numEntries,
+                                                                 long waitTime,
+                                                                 TimeUnit timeUnit) {
+        return readInternal(numEntries, waitTime, timeUnit);
+    }
+
+    /**
+     * Read up to <i>numEntries</i> entries. The future will be satisfied when any number of entries are
+     * ready (1 to <i>numEntries</i>).
+     *
+     * @param numEntries
+     *          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,
+                                                                      long deadlineTime,
+                                                                      TimeUnit deadlineTimeUnit) {
+        timeBetweenReadNexts.registerSuccessfulEvent(readNextDelayStopwatch.elapsed(TimeUnit.MICROSECONDS));
+        readNextDelayStopwatch.reset().start();
+        final PendingReadRequest readRequest = new PendingReadRequest(numEntries, deadlineTime, deadlineTimeUnit);
+
+        if (null == readAheadReader) {
+            final ReadAheadEntryReader readAheadEntryReader = this.readAheadReader = new ReadAheadEntryReader(
+                    getStreamName(),
+                    getStartDLSN(),
+                    bkDistributedLogManager.getConf(),
+                    readHandler,
+                    bkDistributedLogManager.getReaderEntryStore(),
+                    bkDistributedLogManager.getScheduler(),
+                    Ticker.systemTicker(),
+                    bkDistributedLogManager.alertStatsLogger);
+            readHandler.checkLogStreamExists().addEventListener(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;
+                                    }
+                                });
+                    } catch (Exception exc) {
+                        notifyOnError(exc);
+                    }
+                }
+
+                @Override
+                public void onFailure(Throwable cause) {
+                    notifyOnError(cause);
+                }
+            });
+        }
+
+        if (checkClosedOrInError("readNext")) {
+            readRequest.setException(lastException.get());
+        } else {
+            boolean queueEmpty = pendingRequests.isEmpty();
+            pendingRequests.add(readRequest);
+
+            if (queueEmpty) {
+                scheduleBackgroundRead();
+            }
+        }
+
+        readNextExecTime.registerSuccessfulEvent(readNextDelayStopwatch.elapsed(TimeUnit.MICROSECONDS));
+        readNextDelayStopwatch.reset().start();
+
+        return readRequest.getPromise();
+    }
+
+    public synchronized void scheduleBackgroundRead() {
+        // if the reader is already closed, we don't need to schedule background read again.
+        if (null != closeFuture) {
+            return;
+        }
+
+        long prevCount = scheduleCount.getAndIncrement();
+        if (0 == prevCount) {
+            scheduleDelayStopwatch.reset().start();
+            scheduler.submit(streamName, this);
+        }
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        // Cancel the idle reader timeout task, interrupting if necessary
+        ReadCancelledException exception;
+        Promise<Void> closePromise;
+        synchronized (this) {
+            if (null != closeFuture) {
+                return closeFuture;
+            }
+            closePromise = closeFuture = new Promise<Void>();
+            exception = new ReadCancelledException(readHandler.getFullyQualifiedName(), "Reader was closed");
+            setLastException(exception);
+        }
+
+        // Do this after we have checked that the reader was not previously closed
+        cancelIdleReaderTask();
+
+        synchronized (scheduleLock) {
+            if (null != backgroundScheduleTask) {
+                backgroundScheduleTask.cancel(true);
+            }
+        }
+
+        cancelAllPendingReads(exception);
+
+        ReadAheadEntryReader readAheadReader = getReadAheadReader();
+        if (null != readAheadReader) {
+            readHandler.unregisterListener(readAheadReader);
+            readAheadReader.removeStateChangeNotification(this);
+        }
+        Utils.closeSequence(bkDistributedLogManager.getScheduler(), true,
+                readAheadReader,
+                readHandler
+        ).proxyTo(closePromise);
+        return closePromise;
+    }
+
+    private void cancelAllPendingReads(Throwable throwExc) {
+        for (PendingReadRequest promise : pendingRequests) {
+            promise.setException(throwExc);
+        }
+        pendingRequests.clear();
+    }
+
+    synchronized boolean hasMoreRecords() throws IOException {
+        if (null == readAheadReader) {
+            return false;
+        }
+        if (readAheadReader.getNumCachedEntries() > 0 || null != nextRecord) {
+            return true;
+        } else if (null != currentEntry) {
+            nextRecord = currentEntry.nextRecord();
+            return null != nextRecord;
+        }
+        return false;
+    }
+
+    private synchronized LogRecordWithDLSN readNextRecord() throws IOException {
+        if (null == readAheadReader) {
+            return null;
+        }
+        if (null == currentEntry) {
+            currentEntry = readAheadReader.getNextReadAheadEntry(0L, TimeUnit.MILLISECONDS);
+            // no entry after reading from read ahead then return null
+            if (null == currentEntry) {
+                return null;
+            }
+        }
+
+        LogRecordWithDLSN recordToReturn;
+        if (null == nextRecord) {
+            nextRecord = currentEntry.nextRecord();
+            // no more records in current entry
+            if (null == nextRecord) {
+                currentEntry = null;
+                return readNextRecord();
+            }
+        }
+
+        // found a record to return and prefetch the next one
+        recordToReturn = nextRecord;
+        nextRecord = currentEntry.nextRecord();
+        return recordToReturn;
+    }
+
+    @Override
+    public void run() {
+        synchronized(scheduleLock) {
+            if (scheduleDelayStopwatch.isRunning()) {
+                scheduleLatency.registerSuccessfulEvent(scheduleDelayStopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            }
+
+            Stopwatch runTime = Stopwatch.createStarted();
+            int iterations = 0;
+            long scheduleCountLocal = scheduleCount.get();
+            LOG.debug("{}: Scheduled Background Reader", readHandler.getFullyQualifiedName());
+            while(true) {
+                if (LOG.isTraceEnabled()) {
+                    LOG.trace("{}: Executing Iteration: {}", readHandler.getFullyQualifiedName(), iterations++);
+                }
+
+                PendingReadRequest nextRequest = null;
+                synchronized(this) {
+                    nextRequest = pendingRequests.peek();
+
+                    // Queue is empty, nothing to read, return
+                    if (null == nextRequest) {
+                        LOG.trace("{}: Queue Empty waiting for Input", readHandler.getFullyQualifiedName());
+                        scheduleCount.set(0);
+                        backgroundReaderRunTime.registerSuccessfulEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
+                        return;
+                    }
+
+                    if (disableProcessingReadRequests) {
+                        LOG.info("Reader of {} is forced to stop processing read requests", readHandler.getFullyQualifiedName());
+                        return;
+                    }
+                }
+                lastProcessTime.reset().start();
+
+                // If the oldest pending promise is interrupted then we must mark
+                // 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 (checkClosedOrInError("readNext")) {
+                    if (!(lastException.get().getCause() instanceof LogNotFoundException)) {
+                        LOG.warn("{}: Exception", readHandler.getFullyQualifiedName(), lastException.get());
+                    }
+                    backgroundReaderRunTime.registerFailedEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
+                    return;
+                }
+
+                try {
+                    // Fail 10% of the requests when asked to simulate errors
+                    if (bkDistributedLogManager.getFailureInjector().shouldInjectErrors()) {
+                        throw new IOException("Reader Simulated Exception");
+                    }
+                    LogRecordWithDLSN record;
+                    while (!nextRequest.hasReadEnoughRecords()) {
+                        // read single record
+                        do {
+                            record = readNextRecord();
+                        } while (null != record && (record.isControl() || (record.getDlsn().compareTo(getStartDLSN()) < 0)));
+                        if (null == record) {
+                            break;
+                        } else {
+                            if (record.isEndOfStream() && !returnEndOfStreamRecord) {
+                                setLastException(new EndOfStreamException("End of Stream Reached for "
+                                        + readHandler.getFullyQualifiedName()));
+                                break;
+                            }
+
+                            // gap detection
+                            if (recordPositionsContainsGap(record, lastPosition)) {
+                                bkDistributedLogManager.raiseAlert("Gap detected between records at record = {}", record);
+                                if (positionGapDetectionEnabled) {
+                                    throw new DLIllegalStateException("Gap detected between records at record = " + record);
+                                }
+                            }
+                            lastPosition = record.getLastPositionWithinLogSegment();
+
+                            nextRequest.addRecord(record);
+                        }
+                    };
+                } catch (IOException exc) {
+                    setLastException(exc);
+                    if (!(exc instanceof LogNotFoundException)) {
+                        LOG.warn("{} : read with skip Exception", readHandler.getFullyQualifiedName(), lastException.get());
+                    }
+                    continue;
+                }
+
+                if (nextRequest.hasReadRecords()) {
+                    long remainingWaitTime = nextRequest.getRemainingWaitTime();
+                    if (remainingWaitTime > 0 && !nextRequest.hasReadEnoughRecords()) {
+                        backgroundReaderRunTime.registerSuccessfulEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
+                        scheduleDelayStopwatch.reset().start();
+                        scheduleCount.set(0);
+                        // the request could still wait for more records
+                        backgroundScheduleTask = scheduler.schedule(
+                                streamName,
+                                BACKGROUND_READ_SCHEDULER,
+                                remainingWaitTime,
+                                nextRequest.deadlineTimeUnit);
+                        return;
+                    }
+
+                    PendingReadRequest request = pendingRequests.poll();
+                    if (null != request && nextRequest == request) {
+                        request.complete();
+                        if (null != backgroundScheduleTask) {
+                            backgroundScheduleTask.cancel(true);
+                            backgroundScheduleTask = null;
+                        }
+                    } else {
+                        DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at dlsn = "
+                                + nextRequest.records.get(0).getDlsn());
+                        nextRequest.setException(ise);
+                        if (null != request) {
+                            request.setException(ise);
+                        }
+                        // We should never get here as we should have exited the loop if
+                        // pendingRequests were empty
+                        bkDistributedLogManager.raiseAlert("Unexpected condition at dlsn = {}",
+                                nextRequest.records.get(0).getDlsn());
+                        setLastException(ise);
+                    }
+                } else {
+                    if (0 == scheduleCountLocal) {
+                        LOG.trace("Schedule count dropping to zero", lastException.get());
+                        backgroundReaderRunTime.registerSuccessfulEvent(runTime.stop().elapsed(TimeUnit.MICROSECONDS));
+                        return;
+                    }
+                    scheduleCountLocal = scheduleCount.decrementAndGet();
+                }
+            }
+        }
+    }
+
+    private boolean recordPositionsContainsGap(LogRecordWithDLSN record, long lastPosition) {
+        final boolean firstLogRecord = (1 == record.getPositionWithinLogSegment());
+        final boolean endOfStreamRecord = record.isEndOfStream();
+        final boolean emptyLogSegment = (0 == lastPosition);
+        final boolean positionIncreasedByOne = (record.getPositionWithinLogSegment() == (lastPosition + 1));
+
+        return !firstLogRecord && !endOfStreamRecord && !emptyLogSegment &&
+               !positionIncreasedByOne;
+    }
+
+    /**
+     * Triggered when the background activity encounters an exception
+     */
+    @Override
+    public void notifyOnError(Throwable cause) {
+        if (cause instanceof IOException) {
+            setLastException((IOException) cause);
+        } else {
+            setLastException(new IOException(cause));
+        }
+        scheduleBackgroundRead();
+    }
+
+    /**
+     * Triggered when the background activity completes an operation
+     */
+    @Override
+    public void notifyOnOperationComplete() {
+        scheduleBackgroundRead();
+    }
+
+    @VisibleForTesting
+    void simulateErrors() {
+        bkDistributedLogManager.getFailureInjector().injectErrors(true);
+    }
+
+    @VisibleForTesting
+    synchronized void disableReadAheadLogSegmentsNotification() {
+        readHandler.disableReadAheadLogSegmentsNotification();
+    }
+
+    @VisibleForTesting
+    synchronized void disableProcessingReadRequests() {
+        disableProcessingReadRequests = true;
+    }
+}
+
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
new file mode 100644
index 0000000..1102ff5
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
@@ -0,0 +1,559 @@
+/**
+ * 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.base.Stopwatch;
+import com.google.common.annotations.VisibleForTesting;
+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.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.
+ *
+ * <h3>Metrics</h3>
+ * All the metrics are exposed under `log_writer`.
+ * <ul>
+ * <li> `log_writer/write`: opstats. latency characteristics about the time that write operations spent.
+ * <li> `log_writer/bulk_write`: opstats. latency characteristics about the time that bulk_write
+ * operations spent.
+ * are pending in the queue for long time due to log segment rolling.
+ * <li> `log_writer/get_writer`: opstats. the time spent on getting the writer. it could spike when there
+ * is log segment rolling happened during getting the writer. it is a good stat to look into when the latency
+ * is caused by queuing time.
+ * <li> `log_writer/pending_request_dispatch`: counter. the number of queued operations that are dispatched
+ * after log segment is rolled. it is an metric on measuring how many operations has been queued because of
+ * log segment rolling.
+ * </ul>
+ * See {@link BKLogSegmentWriter} for segment writer stats.
+ */
+public class BKAsyncLogWriter extends BKAbstractLogWriter implements AsyncLogWriter {
+
+    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;
+                }
+            };
+
+    // Records pending for roll log segment.
+    class PendingLogRecord implements FutureEventListener<DLSN> {
+
+        final LogRecord record;
+        final Promise<DLSN> promise;
+        final boolean flush;
+
+        PendingLogRecord(LogRecord record, boolean flush) {
+            this.record = record;
+            this.promise = new Promise<DLSN>();
+            this.flush = flush;
+        }
+
+        @Override
+        public void onSuccess(DLSN value) {
+            promise.setValue(value);
+        }
+
+        @Override
+        public void onFailure(Throwable cause) {
+            promise.setException(cause);
+            encounteredError = true;
+        }
+    }
+
+    /**
+     * Last pending record in current log segment. After it is satisified, it would
+     * roll log segment.
+     *
+     * This implementation is based on the assumption that all future satisified in same
+     * order future pool.
+     */
+    class LastPendingLogRecord extends PendingLogRecord {
+
+        LastPendingLogRecord(LogRecord record, boolean flush) {
+            super(record, flush);
+        }
+
+        @Override
+        public void onSuccess(DLSN value) {
+            super.onSuccess(value);
+            // roll log segment and issue all pending requests.
+            rollLogSegmentAndIssuePendingRequests(record.getTransactionId());
+        }
+
+        @Override
+        public void onFailure(Throwable cause) {
+            super.onFailure(cause);
+            // error out pending requests.
+            errorOutPendingRequestsAndWriter(cause);
+        }
+    }
+
+    private final boolean streamFailFast;
+    private final boolean disableRollOnSegmentError;
+    private LinkedList<PendingLogRecord> pendingRequests = null;
+    private volatile boolean encounteredError = false;
+    private Promise<BKLogSegmentWriter> rollingFuture = null;
+    private long lastTxId = DistributedLogConstants.INVALID_TXID;
+
+    private final StatsLogger statsLogger;
+    private final OpStatsLogger writeOpStatsLogger;
+    private final OpStatsLogger markEndOfStreamOpStatsLogger;
+    private final OpStatsLogger bulkWriteOpStatsLogger;
+    private final OpStatsLogger getWriterOpStatsLogger;
+    private final Counter pendingRequestDispatch;
+
+    private final Feature disableLogSegmentRollingFeature;
+
+    BKAsyncLogWriter(DistributedLogConfiguration conf,
+                     DynamicDistributedLogConfiguration dynConf,
+                     BKDistributedLogManager bkdlm,
+                     BKLogWriteHandler writeHandler, /** log writer owns the handler **/
+                     FeatureProvider featureProvider,
+                     StatsLogger dlmStatsLogger) {
+        super(conf, dynConf, bkdlm);
+        this.writeHandler = writeHandler;
+        this.streamFailFast = conf.getFailFastOnStreamNotReady();
+        this.disableRollOnSegmentError = conf.getDisableRollingOnLogSegmentError();
+
+        // features
+        disableLogSegmentRollingFeature = featureProvider.getFeature(CoreFeatureKeys.DISABLE_LOGSEGMENT_ROLLING.name().toLowerCase());
+        // stats
+        this.statsLogger = dlmStatsLogger.scope("log_writer");
+        this.writeOpStatsLogger = statsLogger.getOpStatsLogger("write");
+        this.markEndOfStreamOpStatsLogger = statsLogger.getOpStatsLogger("mark_end_of_stream");
+        this.bulkWriteOpStatsLogger = statsLogger.getOpStatsLogger("bulk_write");
+        this.getWriterOpStatsLogger = statsLogger.getOpStatsLogger("get_writer");
+        this.pendingRequestDispatch = statsLogger.getCounter("pending_request_dispatch");
+    }
+
+    @VisibleForTesting
+    synchronized void setLastTxId(long txId) {
+        lastTxId = Math.max(lastTxId, txId);
+    }
+
+    @Override
+    public synchronized long getLastTxId() {
+        return lastTxId;
+    }
+
+    /**
+     * Write a log record as control record. The method will be used by Monitor Service to enforce a new inprogress segment.
+     *
+     * @param record
+     *          log record
+     * @return future of the write
+     */
+    public Future<DLSN> writeControlRecord(final LogRecord record) {
+        record.setControl();
+        return write(record);
+    }
+
+    private BKLogSegmentWriter getCachedLogSegmentWriter() throws WriteException {
+        if (encounteredError) {
+            throw new WriteException(bkDistributedLogManager.getStreamName(),
+                    "writer has been closed due to error.");
+        }
+        BKLogSegmentWriter segmentWriter = getCachedLogWriter();
+        if (null != segmentWriter
+                && segmentWriter.isLogSegmentInError()
+                && !disableRollOnSegmentError) {
+            return null;
+        } else {
+            return segmentWriter;
+        }
+    }
+
+    private Future<BKLogSegmentWriter> getLogSegmentWriter(long firstTxid,
+                                                           boolean bestEffort,
+                                                           boolean rollLog,
+                                                           boolean allowMaxTxID) {
+        Stopwatch stopwatch = Stopwatch.createStarted();
+        return FutureUtils.stats(
+                doGetLogSegmentWriter(firstTxid, bestEffort, rollLog, allowMaxTxID),
+                getWriterOpStatsLogger,
+                stopwatch);
+    }
+
+    private Future<BKLogSegmentWriter> doGetLogSegmentWriter(final long firstTxid,
+                                                             final boolean bestEffort,
+                                                             final boolean rollLog,
+                                                             final boolean allowMaxTxID) {
+        if (encounteredError) {
+            return Future.exception(new WriteException(bkDistributedLogManager.getStreamName(),
+                    "writer has been closed due to error."));
+        }
+        Future<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);
+                }
+            });
+        } else {
+            return writerFuture;
+        }
+    }
+
+    /**
+     * 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() {
+        return getLogSegmentWriter(DistributedLogConstants.MAX_TXID,
+                                     false /* bestEffort */,
+                                     false /* roll log */,
+                                     true /* allow max txid */);
+    }
+
+    private Future<BKLogSegmentWriter> getLogSegmentWriter(long firstTxid,
+                                                           boolean bestEffort,
+                                                           boolean rollLog) {
+        return getLogSegmentWriter(firstTxid, bestEffort, rollLog, false /* allow max txid */);
+    }
+
+    Future<DLSN> queueRequest(LogRecord record, boolean flush) {
+        PendingLogRecord pendingLogRecord = new PendingLogRecord(record, flush);
+        pendingRequests.add(pendingLogRecord);
+        return pendingLogRecord.promise;
+    }
+
+    boolean shouldRollLog(BKLogSegmentWriter w) {
+        try {
+            return null == w ||
+                    (!disableLogSegmentRollingFeature.isAvailable() &&
+                    shouldStartNewSegment(w));
+        } catch (IOException ioe) {
+            return false;
+        }
+    }
+
+    void startQueueingRequests() {
+        assert(null == pendingRequests && null == rollingFuture);
+        pendingRequests = new LinkedList<PendingLogRecord>();
+        rollingFuture = new Promise<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,
+                                                 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;
+        BKLogSegmentWriter w;
+        try {
+            w = getCachedLogSegmentWriter();
+        } catch (WriteException we) {
+            return Future.exception(we);
+        }
+        if (null != rollingFuture) {
+            if (streamFailFast) {
+                result = Future.exception(new StreamNotReadyException("Rolling log segment"));
+            } else {
+                result = queueRequest(record, flush);
+            }
+        } else if (shouldRollLog(w)) {
+            // insert a last record, so when it called back, we will trigger a log segment rolling
+            startQueueingRequests();
+            if (null != w) {
+                LastPendingLogRecord lastLogRecordInCurrentSegment = new LastPendingLogRecord(record, flush);
+                w.asyncWrite(record, true).addEventListener(lastLogRecordInCurrentSegment);
+                result = lastLogRecordInCurrentSegment.promise;
+            } else { // no log segment yet. roll the log segment and issue pending requests.
+                result = queueRequest(record, flush);
+                rollLogSegmentAndIssuePendingRequests(record.getTransactionId());
+            }
+        } else {
+            result = w.asyncWrite(record, flush);
+        }
+        // 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;
+            }
+        });
+    }
+
+    private List<Future<DLSN>> asyncWriteBulk(List<LogRecord> records) {
+        final ArrayList<Future<DLSN>> results = new ArrayList<Future<DLSN>>(records.size());
+        Iterator<LogRecord> iterator = records.iterator();
+        while (iterator.hasNext()) {
+            LogRecord record = iterator.next();
+            Future<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()) {
+                break;
+            }
+        }
+        if (records.size() > results.size()) {
+            appendCancelledFutures(results, records.size() - results.size());
+        }
+        return results;
+    }
+
+    private void appendCancelledFutures(List<Future<DLSN>> futures, int numToAdd) {
+        final WriteCancelledException cre =
+            new WriteCancelledException(getStreamName());
+        for (int i = 0; i < numToAdd; i++) {
+            Future<DLSN> cancelledFuture = Future.exception(cre);
+            futures.add(cancelledFuture);
+        }
+    }
+
+    private void rollLogSegmentAndIssuePendingRequests(final long firstTxId) {
+        getLogSegmentWriter(firstTxId, true, true)
+                .addEventListener(new FutureEventListener<BKLogSegmentWriter>() {
+            @Override
+            public void onSuccess(BKLogSegmentWriter writer) {
+                try {
+                    synchronized (BKAsyncLogWriter.this) {
+                        for (PendingLogRecord pendingLogRecord : pendingRequests) {
+                            FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_LogWriterIssuePending);
+                            writer.asyncWrite(pendingLogRecord.record, pendingLogRecord.flush)
+                                    .addEventListener(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
+                        // the corresponding bookies would be able to create its ledger.
+                        if (pendingRequests.isEmpty()) {
+                            LogRecord controlRecord = new LogRecord(firstTxId,
+                                    DistributedLogConstants.CONTROL_RECORD_CONTENT);
+                            controlRecord.setControl();
+                            PendingLogRecord controlReq = new PendingLogRecord(controlRecord, false);
+                            writer.asyncWrite(controlReq.record, controlReq.flush)
+                                    .addEventListener(controlReq);
+                        }
+                        if (null != rollingFuture) {
+                            FutureUtils.setValue(rollingFuture, writer);
+                        }
+                        rollingFuture = null;
+                        pendingRequestDispatch.add(pendingRequests.size());
+                        pendingRequests = null;
+                    }
+                } catch (IOException ioe) {
+                    errorOutPendingRequestsAndWriter(ioe);
+                }
+            }
+            @Override
+            public void onFailure(Throwable cause) {
+                errorOutPendingRequestsAndWriter(cause);
+            }
+        });
+    }
+
+    @VisibleForTesting
+    void errorOutPendingRequests(Throwable cause, boolean errorOutWriter) {
+        final List<PendingLogRecord> pendingRequestsSnapshot;
+        synchronized (this) {
+            pendingRequestsSnapshot = pendingRequests;
+            encounteredError = errorOutWriter;
+            pendingRequests = null;
+            if (null != rollingFuture) {
+                FutureUtils.setException(rollingFuture, cause);
+            }
+            rollingFuture = null;
+        }
+
+        pendingRequestDispatch.add(pendingRequestsSnapshot.size());
+
+        // After erroring out the writer above, no more requests
+        // will be enqueued to pendingRequests
+        for (PendingLogRecord pendingLogRecord : pendingRequestsSnapshot) {
+            pendingLogRecord.promise.setException(cause);
+        }
+    }
+
+    void errorOutPendingRequestsAndWriter(Throwable cause) {
+        errorOutPendingRequests(cause, true /* error out writer */);
+    }
+
+    /**
+     * Write a log record to the stream.
+     *
+     * @param record single log record
+     */
+    @Override
+    public Future<DLSN> write(final LogRecord record) {
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+        return FutureUtils.stats(
+                asyncWrite(record, true),
+                writeOpStatsLogger,
+                stopwatch);
+    }
+
+    /**
+     * Write many log records to the stream. The return type here is unfortunate but its a direct result
+     * of having to combine FuturePool and the asyncWriteBulk method which returns a future as well. The
+     * problem is the List that asyncWriteBulk returns can't be materialized until getLogSegmentWriter
+     * completes, so it has to be wrapped in a future itself.
+     *
+     * @param records list of records
+     */
+    @Override
+    public Future<List<Future<DLSN>>> writeBulk(final List<LogRecord> records) {
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+        return FutureUtils.stats(
+                Future.value(asyncWriteBulk(records)),
+                bulkWriteOpStatsLogger,
+                stopwatch);
+    }
+
+    @Override
+    public Future<Boolean> truncate(final DLSN dlsn) {
+        if (DLSN.InvalidDLSN == dlsn) {
+            return Future.value(false);
+        }
+        BKLogWriteHandler writeHandler;
+        try {
+            writeHandler = getWriteHandler();
+        } catch (IOException e) {
+            return Future.exception(e);
+        }
+        return writeHandler.setLogSegmentsOlderThanDLSNTruncated(dlsn).map(TruncationResultConverter);
+    }
+
+    Future<Long> flushAndCommit() {
+        Future<BKLogSegmentWriter> writerFuture;
+        synchronized (this) {
+            if (null != this.rollingFuture) {
+                writerFuture = this.rollingFuture;
+            } else {
+                writerFuture = getCachedLogWriterFuture();
+            }
+        }
+        if (null == writerFuture) {
+            return Future.value(getLastTxId());
+        }
+        return writerFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Long>>() {
+            @Override
+            public Future<Long> apply(BKLogSegmentWriter writer) {
+                return writer.flushAndCommit();
+            }
+        });
+    }
+
+    Future<Long> markEndOfStream() {
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+        Future<BKLogSegmentWriter> logSegmentWriterFuture;
+        synchronized (this) {
+            logSegmentWriterFuture = this.rollingFuture;
+        }
+        if (null == logSegmentWriterFuture) {
+            logSegmentWriterFuture = getLogSegmentWriterForEndOfStream();
+        }
+
+        return FutureUtils.stats(
+                logSegmentWriterFuture.flatMap(new AbstractFunction1<BKLogSegmentWriter, Future<Long>>() {
+                    @Override
+                    public Future<Long> apply(BKLogSegmentWriter w) {
+                        return w.markEndOfStream();
+                    }
+                }),
+                markEndOfStreamOpStatsLogger,
+                stopwatch);
+    }
+
+    @Override
+    protected Future<Void> asyncCloseAndComplete() {
+        Future<BKLogSegmentWriter> logSegmentWriterFuture;
+        synchronized (this) {
+            logSegmentWriterFuture = this.rollingFuture;
+        }
+
+        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();
+                }
+            });
+        }
+    }
+
+    @Override
+    void closeAndComplete() throws IOException {
+        FutureUtils.result(asyncCloseAndComplete());
+    }
+
+    /**
+     * *TEMP HACK*
+     * Get the name of the stream this writer writes data to
+     */
+    @Override
+    public String getStreamName() {
+        return bkDistributedLogManager.getStreamName();
+    }
+
+    @Override
+    public Future<Void> asyncAbort() {
+        Future<Void> result = super.asyncAbort();
+        synchronized (this) {
+            if (pendingRequests != null) {
+                for (PendingLogRecord pendingLogRecord : pendingRequests) {
+                    pendingLogRecord.promise.setException(new WriteException(bkDistributedLogManager.getStreamName(),
+                            "abort wring: writer has been closed due to error."));
+                }
+            }
+        }
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("AsyncLogWriter:%s", getStreamName());
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
new file mode 100644
index 0000000..00e6b5c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
@@ -0,0 +1,1106 @@
+/**
+ * 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 com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+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;
+import org.apache.distributedlog.metadata.LogMetadataForReader;
+import org.apache.distributedlog.metadata.LogMetadataForWriter;
+import org.apache.distributedlog.io.AsyncCloseable;
+import org.apache.distributedlog.lock.DistributedLock;
+import org.apache.distributedlog.lock.NopDistributedLock;
+import org.apache.distributedlog.lock.ZKDistributedLock;
+import org.apache.distributedlog.logsegment.LogSegmentFilter;
+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.util.Allocator;
+import org.apache.distributedlog.util.DLUtils;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.MonitoredFuturePool;
+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.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;
+
+/**
+ * <h3>Metrics</h3>
+ * <ul>
+ * <li> `log_writer/*`: all asynchronous writer related metrics are exposed under scope `log_writer`.
+ * See {@link BKAsyncLogWriter} for detail stats.
+ * <li> `async_reader/*`: all asyncrhonous reader related metrics are exposed under scope `async_reader`.
+ * See {@link BKAsyncLogReader} for detail stats.
+ * <li> `writer_future_pool/*`: metrics about the future pools that used by writers are exposed under
+ * scope `writer_future_pool`. See {@link MonitoredFuturePool} for detail stats.
+ * <li> `reader_future_pool/*`: metrics about the future pools that used by readers are exposed under
+ * scope `reader_future_pool`. See {@link MonitoredFuturePool} for detail stats.
+ * <li> `lock/*`: metrics about the locks used by writers. See {@link ZKDistributedLock} for detail
+ * stats.
+ * <li> `read_lock/*`: metrics about the locks used by readers. See {@link ZKDistributedLock} for
+ * detail stats.
+ * <li> `logsegments/*`: metrics about basic operations on log segments. See {@link BKLogHandler} for details.
+ * <li> `segments/*`: metrics about write operations on log segments. See {@link BKLogWriteHandler} for details.
+ * <li> `readahead_worker/*`: metrics about readahead workers used by readers. See {@link BKLogReadHandler}
+ * for details.
+ * </ul>
+ */
+class BKDistributedLogManager implements DistributedLogManager {
+    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();
+                }
+            };
+
+    static final Function<LogRecordWithDLSN, DLSN> RECORD_2_DLSN_FUNCTION =
+            new Function<LogRecordWithDLSN, DLSN>() {
+                @Override
+                public DLSN apply(LogRecordWithDLSN record) {
+                    return record.getDlsn();
+                }
+            };
+
+    private final URI uri;
+    private final String name;
+    private final String clientId;
+    private final int regionId;
+    private final String streamIdentifier;
+    private final DistributedLogConfiguration conf;
+    private final DynamicDistributedLogConfiguration dynConf;
+    private final NamespaceDriver driver;
+    private Promise<Void> closePromise;
+    private final OrderedScheduler scheduler;
+    private final FeatureProvider featureProvider;
+    private final AsyncFailureInjector failureInjector;
+    private final StatsLogger statsLogger;
+    private final StatsLogger perLogStatsLogger;
+    final AlertStatsLogger alertStatsLogger;
+
+    // log segment metadata cache
+    private final LogSegmentMetadataCache logSegmentMetadataCache;
+
+    //
+    // Writer Related Variables
+    //
+    private final PermitLimiter writeLimiter;
+
+    //
+    // Reader Related Variables
+    ///
+    // read handler for listener.
+    private BKLogReadHandler readHandlerForListener = null;
+    private final PendingReaders pendingReaders;
+
+    // resource to close
+    private final Optional<AsyncCloseable> resourcesCloseable;
+
+    /**
+     * Create a {@link DistributedLogManager} with supplied resources.
+     *
+     * @param name log name
+     * @param conf distributedlog configuration
+     * @param dynConf dynamic distributedlog configuration
+     * @param uri uri location for the log
+     * @param driver namespace driver
+     * @param logSegmentMetadataCache log segment metadata cache
+     * @param scheduler ordered scheduled used by readers and writers
+     * @param clientId client id that used to initiate the locks
+     * @param regionId region id that would be encrypted as part of log segment metadata
+     *                 to indicate which region that the log segment will be created
+     * @param writeLimiter write limiter
+     * @param featureProvider provider to offer features
+     * @param statsLogger stats logger to receive stats
+     * @param perLogStatsLogger stats logger to receive per log stats
+     * @throws IOException
+     */
+    BKDistributedLogManager(String name,
+                            DistributedLogConfiguration conf,
+                            DynamicDistributedLogConfiguration dynConf,
+                            URI uri,
+                            NamespaceDriver driver,
+                            LogSegmentMetadataCache logSegmentMetadataCache,
+                            OrderedScheduler scheduler,
+                            String clientId,
+                            Integer regionId,
+                            PermitLimiter writeLimiter,
+                            FeatureProvider featureProvider,
+                            AsyncFailureInjector failureInjector,
+                            StatsLogger statsLogger,
+                            StatsLogger perLogStatsLogger,
+                            Optional<AsyncCloseable> resourcesCloseable) {
+        this.name = name;
+        this.conf = conf;
+        this.dynConf = dynConf;
+        this.uri = uri;
+        this.driver = driver;
+        this.logSegmentMetadataCache = logSegmentMetadataCache;
+        this.scheduler = scheduler;
+        this.statsLogger = statsLogger;
+        this.perLogStatsLogger = BroadCastStatsLogger.masterslave(perLogStatsLogger, statsLogger);
+        this.pendingReaders = new PendingReaders(scheduler);
+        this.regionId = regionId;
+        this.clientId = clientId;
+        this.streamIdentifier = conf.getUnpartitionedStreamName();
+        this.writeLimiter = writeLimiter;
+        // Feature Provider
+        this.featureProvider = featureProvider;
+        // Failure Injector
+        this.failureInjector = failureInjector;
+        // Stats
+        this.alertStatsLogger = new AlertStatsLogger(this.perLogStatsLogger, "dl_alert");
+        this.resourcesCloseable = resourcesCloseable;
+    }
+
+    @Override
+    public String getStreamName() {
+        return name;
+    }
+
+    @Override
+    public NamespaceDriver getNamespaceDriver() {
+        return driver;
+    }
+
+    URI getUri() {
+        return uri;
+    }
+
+    DistributedLogConfiguration getConf() {
+        return conf;
+    }
+
+    OrderedScheduler getScheduler() {
+        return scheduler;
+    }
+
+    AsyncFailureInjector getFailureInjector() {
+        return failureInjector;
+    }
+
+    //
+    // Test Methods
+    //
+
+    @VisibleForTesting
+    LogStreamMetadataStore getWriterMetadataStore() {
+        return driver.getLogStreamMetadataStore(WRITER);
+    }
+
+    @VisibleForTesting
+    LogSegmentEntryStore getReaderEntryStore() {
+        return driver.getLogSegmentEntryStore(READER);
+    }
+
+    @VisibleForTesting
+    FeatureProvider getFeatureProvider() {
+        return this.featureProvider;
+    }
+
+    private synchronized BKLogReadHandler getReadHandlerAndRegisterListener(
+            boolean create, LogSegmentListener listener) {
+        if (null == readHandlerForListener && create) {
+            readHandlerForListener = createReadHandler();
+            readHandlerForListener.registerListener(listener);
+            // start fetch the log segments after created the listener
+            readHandlerForListener.asyncStartFetchLogSegments();
+            return readHandlerForListener;
+        }
+        if (null != readHandlerForListener && null != listener) {
+            readHandlerForListener.registerListener(listener);
+        }
+        return readHandlerForListener;
+    }
+
+    @Override
+    public List<LogSegmentMetadata> getLogSegments() throws IOException {
+        return FutureUtils.result(getLogSegmentsAsync());
+    }
+
+    protected Future<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));
+    }
+
+    @Override
+    public void registerListener(LogSegmentListener listener) throws IOException {
+        getReadHandlerAndRegisterListener(true, listener);
+    }
+
+    @Override
+    public synchronized void unregisterListener(LogSegmentListener listener) {
+        if (null != readHandlerForListener) {
+            readHandlerForListener.unregisterListener(listener);
+        }
+    }
+
+    public void checkClosedOrInError(String operation) throws AlreadyClosedException {
+        synchronized (this) {
+            if (null != closePromise) {
+                throw new AlreadyClosedException("Executing " + operation + " on already closed DistributedLogManager");
+            }
+        }
+    }
+
+    // Create Read Handler
+
+    synchronized BKLogReadHandler createReadHandler() {
+        Optional<String> subscriberId = Optional.absent();
+        return createReadHandler(subscriberId, false);
+    }
+
+    synchronized BKLogReadHandler createReadHandler(Optional<String> subscriberId) {
+        return createReadHandler(subscriberId, false);
+    }
+
+    synchronized BKLogReadHandler createReadHandler(Optional<String> subscriberId,
+                                                    boolean isHandleForReading) {
+        return createReadHandler(
+                subscriberId,
+                null,
+                isHandleForReading);
+    }
+
+    synchronized BKLogReadHandler createReadHandler(Optional<String> subscriberId,
+                                                    AsyncNotification notification,
+                                                    boolean isHandleForReading) {
+        LogMetadataForReader logMetadata = LogMetadataForReader.of(uri, name, streamIdentifier);
+        return new BKLogReadHandler(
+                logMetadata,
+                subscriberId,
+                conf,
+                dynConf,
+                driver.getLogStreamMetadataStore(READER),
+                logSegmentMetadataCache,
+                driver.getLogSegmentEntryStore(READER),
+                scheduler,
+                alertStatsLogger,
+                statsLogger,
+                perLogStatsLogger,
+                clientId,
+                notification,
+                isHandleForReading);
+    }
+
+    // Create Ledger Allocator
+
+
+
+    // Create Write Handler
+
+    public BKLogWriteHandler createWriteHandler(boolean lockHandler)
+            throws IOException {
+        return FutureUtils.result(asyncCreateWriteHandler(lockHandler));
+    }
+
+    Future<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;
+            }
+        });
+    }
+
+    private void createWriteHandler(LogMetadataForWriter logMetadata,
+                                    boolean lockHandler,
+                                    final Promise<BKLogWriteHandler> createPromise) {
+        // Build the locks
+        DistributedLock lock;
+        if (conf.isWriteLockEnabled()) {
+            lock = driver.getLogStreamMetadataStore(WRITER).createWriteLock(logMetadata);
+        } else {
+            lock = NopDistributedLock.INSTANCE;
+        }
+
+        Allocator<LogSegmentEntryWriter, Object> segmentAllocator;
+        try {
+            segmentAllocator = driver.getLogSegmentEntryStore(WRITER)
+                    .newLogSegmentAllocator(logMetadata, dynConf);
+        } catch (IOException ioe) {
+            FutureUtils.setException(createPromise, ioe);
+            return;
+        }
+
+        // Make sure writer handler created before resources are initialized
+        final BKLogWriteHandler writeHandler = new BKLogWriteHandler(
+                logMetadata,
+                conf,
+                driver.getLogStreamMetadataStore(WRITER),
+                logSegmentMetadataCache,
+                driver.getLogSegmentEntryStore(WRITER),
+                scheduler,
+                segmentAllocator,
+                statsLogger,
+                perLogStatsLogger,
+                alertStatsLogger,
+                clientId,
+                regionId,
+                writeLimiter,
+                featureProvider,
+                dynConf,
+                lock);
+        if (lockHandler) {
+            writeHandler.lockHandler().addEventListener(new FutureEventListener<DistributedLock>() {
+                @Override
+                public void onSuccess(DistributedLock lock) {
+                    FutureUtils.setValue(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;
+                        }
+                    });
+                }
+            });
+        } else {
+            FutureUtils.setValue(createPromise, writeHandler);
+        }
+    }
+
+    PermitManager getLogSegmentRollingPermitManager() {
+        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);
+            }
+        });
+    }
+
+    /**
+     * Check if an end of stream marker was added to the stream
+     * A stream with an end of stream marker cannot be appended to
+     *
+     * @return true if the marker was added to the stream, false otherwise
+     */
+    @Override
+    public boolean isEndOfStreamMarked() throws IOException {
+        checkClosedOrInError("isEndOfStreamMarked");
+        long lastTxId = FutureUtils.result(getLastLogRecordAsyncInternal(false, true)).getTransactionId();
+        return lastTxId == DistributedLogConstants.MAX_TXID;
+    }
+
+    /**
+     * 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 {
+        long position;
+        try {
+            position = FutureUtils.result(getLastLogRecordAsyncInternal(true, false)).getTransactionId();
+            if (DistributedLogConstants.INVALID_TXID == position ||
+                DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID == position) {
+                position = 0;
+            }
+        } catch (LogEmptyException ex) {
+            position = 0;
+        } catch (LogNotFoundException ex) {
+            position = 0;
+        }
+        return new AppendOnlyStreamWriter(startAsyncLogSegmentNonPartitioned(), position);
+    }
+
+    /**
+     * 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 {
+        return new AppendOnlyStreamReader(this);
+    }
+
+    /**
+     * Begin writing to the log stream identified by the name
+     *
+     * @return the writer interface to generate log records
+     */
+    @Override
+    public BKSyncLogWriter startLogSegmentNonPartitioned() throws IOException {
+        checkClosedOrInError("startLogSegmentNonPartitioned");
+        BKSyncLogWriter writer = new BKSyncLogWriter(conf, dynConf, this);
+        boolean success = false;
+        try {
+            writer.createAndCacheWriteHandler();
+            BKLogWriteHandler writeHandler = writer.getWriteHandler();
+            FutureUtils.result(writeHandler.lockHandler());
+            success = true;
+            return writer;
+        } finally {
+            if (!success) {
+                writer.abort();
+            }
+        }
+    }
+
+    /**
+     * Begin writing to the log stream identified by the name
+     *
+     * @return the writer interface to generate log records
+     */
+    @Override
+    public BKAsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException {
+        return (BKAsyncLogWriter) FutureUtils.result(openAsyncLogWriter());
+    }
+
+    @Override
+    public Future<AsyncLogWriter> openAsyncLogWriter() {
+        try {
+            checkClosedOrInError("startLogSegmentNonPartitioned");
+        } catch (AlreadyClosedException e) {
+            return Future.exception(e);
+        }
+
+        Future<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;
+                            }
+                        });
+            }
+        });
+    }
+
+    @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);
+            }
+        });
+    }
+
+    private Future<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 getDLSNNotLessThanTxIdInSegment(
+                fromTxnId,
+                segmentIdx,
+                segments,
+                driver.getLogSegmentEntryStore(READER)
+        );
+    }
+
+    private Future<DLSN> getDLSNNotLessThanTxIdInSegment(final long fromTxnId,
+                                                         final int segmentIdx,
+                                                         final List<LogSegmentMetadata> segments,
+                                                         final LogSegmentEntryStore entryStore) {
+        final LogSegmentMetadata segment = segments.get(segmentIdx);
+        return ReadUtils.getLogRecordNotLessThanTxId(
+                name,
+                segment,
+                fromTxnId,
+                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);
+                }
+            }
+        });
+    }
+
+    /**
+     * 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.
+     */
+    @Override
+    public LogReader getInputStream(long fromTxnId)
+        throws IOException {
+        return getInputStreamInternal(fromTxnId);
+    }
+
+    @Override
+    public LogReader getInputStream(DLSN fromDLSN) throws IOException {
+        return getInputStreamInternal(fromDLSN, Optional.<Long>absent());
+    }
+
+    @Override
+    public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException {
+        return FutureUtils.result(openAsyncLogReader(fromTxnId));
+    }
+
+    /**
+     * Opening a log reader positioning by transaction id <code>fromTxnId</code>.
+     *
+     * <p>
+     * - retrieve log segments for the stream
+     * - if the log segment list is empty, positioning by the last dlsn
+     * - otherwise, find the first log segment that contains the records whose transaction ids are not less than
+     *   the provided transaction id <code>fromTxnId</code>
+     *   - if all log segments' records' transaction ids are more than <code>fromTxnId</code>, positioning
+     *     on the first record.
+     *   - otherwise, search the log segment to find the log record
+     *     - if the log record is found, positioning the reader by that found record's dlsn
+     *     - otherwise, positioning by the last dlsn
+     * </p>
+     *
+     * @see DLUtils#findLogSegmentNotLessThanTxnId(List, long)
+     * @see ReadUtils#getLogRecordNotLessThanTxId(String, LogSegmentMetadata, long, ExecutorService, LogSegmentEntryStore, int)
+     * @param fromTxnId
+     *          transaction id to start reading from
+     * @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>() {
+
+            @Override
+            public void onSuccess(DLSN dlsn) {
+                dlsnPromise.setValue(dlsn);
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                if (cause instanceof LogEmptyException) {
+                    dlsnPromise.setValue(DLSN.InitialDLSN);
+                } else {
+                    dlsnPromise.setException(cause);
+                }
+            }
+        });
+        return dlsnPromise.flatMap(new AbstractFunction1<DLSN, Future<AsyncLogReader>>() {
+            @Override
+            public Future<AsyncLogReader> apply(DLSN dlsn) {
+                return openAsyncLogReader(dlsn);
+            }
+        });
+    }
+
+    @Override
+    public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException {
+        return FutureUtils.result(openAsyncLogReader(fromDLSN));
+    }
+
+    @Override
+    public Future<AsyncLogReader> openAsyncLogReader(DLSN fromDLSN) {
+        Optional<String> subscriberId = Optional.absent();
+        AsyncLogReader reader = new BKAsyncLogReader(
+                this,
+                scheduler,
+                fromDLSN,
+                subscriberId,
+                false,
+                statsLogger);
+        pendingReaders.add(reader);
+        return Future.value(reader);
+    }
+
+    /**
+     * Note the lock here is a sort of elective exclusive lock. I.e. acquiring this lock will only prevent other
+     * people who try to acquire the lock from reading from the stream. Normal readers (and writers) will not be
+     * blocked.
+     */
+    @Override
+    public Future<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) {
+        return getAsyncLogReaderWithLock(Optional.of(fromDLSN), Optional.of(subscriberId));
+    }
+
+    @Override
+    public Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId) {
+        Optional<DLSN> fromDLSN = Optional.absent();
+        return getAsyncLogReaderWithLock(fromDLSN, Optional.of(subscriberId));
+    }
+
+    protected Future<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."));
+        }
+        final BKAsyncLogReader reader = new BKAsyncLogReader(
+                BKDistributedLogManager.this,
+                scheduler,
+                fromDLSN.isPresent() ? fromDLSN.get() : DLSN.InitialDLSN,
+                subscriberId,
+                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) {
+                // cancel the lock when the creation future is cancelled
+                lockFuture.cancel();
+                return BoxedUnit.UNIT;
+            }
+        });
+        // lock the stream - fetch the last commit position on success
+        lockFuture.flatMap(new Function<Void, Future<AsyncLogReader>>() {
+            @Override
+            public Future<AsyncLogReader> apply(Void complete) {
+                if (fromDLSN.isPresent()) {
+                    return Future.value((AsyncLogReader) 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;
+                    }
+                });
+            }
+        }).addEventListener(new FutureEventListener<AsyncLogReader>() {
+            @Override
+            public void onSuccess(AsyncLogReader r) {
+                pendingReaders.remove(reader);
+                FutureUtils.setValue(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;
+                    }
+                });
+            }
+        });
+        return createPromise;
+    }
+
+    /**
+     * Get the input stream starting with fromTxnId for the specified log
+     *
+     * @param fromTxnId
+     *          transaction id to start reading from
+     * @return log reader
+     * @throws IOException
+     */
+    LogReader getInputStreamInternal(long fromTxnId)
+        throws IOException {
+        DLSN fromDLSN;
+        try {
+            fromDLSN = FutureUtils.result(getDLSNNotLessThanTxId(fromTxnId));
+        } catch (LogEmptyException lee) {
+            fromDLSN = DLSN.InitialDLSN;
+        }
+        return getInputStreamInternal(fromDLSN, Optional.of(fromTxnId));
+    }
+
+    LogReader getInputStreamInternal(DLSN fromDLSN, Optional<Long> fromTxnId)
+            throws IOException {
+        LOG.info("Create sync reader starting from {}", fromDLSN);
+        checkClosedOrInError("getInputStream");
+        return new BKSyncLogReader(
+                conf,
+                this,
+                fromDLSN,
+                fromTxnId,
+                statsLogger);
+    }
+
+    /**
+     * Get the last log record in the stream
+     *
+     * @return the last log record in the stream
+     * @throws java.io.IOException if a stream cannot be found.
+     */
+    @Override
+    public LogRecordWithDLSN getLastLogRecord() throws IOException {
+        checkClosedOrInError("getLastLogRecord");
+        return FutureUtils.result(getLastLogRecordAsync());
+    }
+
+    @Override
+    public long getFirstTxId() throws IOException {
+        checkClosedOrInError("getFirstTxId");
+        return FutureUtils.result(getFirstRecordAsyncInternal()).getTransactionId();
+    }
+
+    @Override
+    public long getLastTxId() throws IOException {
+        checkClosedOrInError("getLastTxId");
+        return FutureUtils.result(getLastTxIdAsync());
+    }
+
+    @Override
+    public DLSN getLastDLSN() throws IOException {
+        checkClosedOrInError("getLastDLSN");
+        return FutureUtils.result(getLastLogRecordAsyncInternal(false, false)).getDlsn();
+    }
+
+    /**
+     * Get Latest log record in the log
+     *
+     * @return latest log record
+     */
+    @Override
+    public Future<LogRecordWithDLSN> getLastLogRecordAsync() {
+        return getLastLogRecordAsyncInternal(false, false);
+    }
+
+    private Future<LogRecordWithDLSN> getLastLogRecordAsyncInternal(final boolean recover,
+                                                                    final boolean includeEndOfStream) {
+        return processReaderOperation(new Function<BKLogReadHandler, Future<LogRecordWithDLSN>>() {
+            @Override
+            public Future<LogRecordWithDLSN> apply(final BKLogReadHandler ledgerHandler) {
+                return ledgerHandler.getLastLogRecordAsync(recover, includeEndOfStream);
+            }
+        });
+    }
+
+    /**
+     * Get Latest Transaction Id in the log
+     *
+     * @return latest transaction id
+     */
+    @Override
+    public Future<Long> getLastTxIdAsync() {
+        return getLastLogRecordAsyncInternal(false, false)
+                .map(RECORD_2_TXID_FUNCTION);
+    }
+
+    /**
+     * Get first DLSN in the log.
+     *
+     * @return first dlsn in the stream
+     */
+    @Override
+    public Future<DLSN> getFirstDLSNAsync() {
+        return getFirstRecordAsyncInternal().map(RECORD_2_DLSN_FUNCTION);
+    }
+
+    private Future<LogRecordWithDLSN> getFirstRecordAsyncInternal() {
+        return processReaderOperation(new Function<BKLogReadHandler, Future<LogRecordWithDLSN>>() {
+            @Override
+            public Future<LogRecordWithDLSN> apply(final BKLogReadHandler ledgerHandler) {
+                return ledgerHandler.asyncGetFirstLogRecord();
+            }
+        });
+    }
+
+    /**
+     * Get Latest DLSN in the log.
+     *
+     * @return latest transaction id
+     */
+    @Override
+    public Future<DLSN> getLastDLSNAsync() {
+        return getLastLogRecordAsyncInternal(false, false)
+                .map(RECORD_2_DLSN_FUNCTION);
+    }
+
+    /**
+     * 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
+     *
+     * @return number of log records
+     * @throws IOException
+     */
+    @Override
+    public long getLogRecordCount() throws IOException {
+        checkClosedOrInError("getLogRecordCount");
+        return FutureUtils.result(getLogRecordCountAsync(DLSN.InitialDLSN));
+    }
+
+    /**
+     * Get the number of log records in the active portion of the log asynchronously.
+     * Any log segments that have already been truncated will not be included
+     *
+     * @return future number of log records
+     * @throws IOException
+     */
+    @Override
+    public Future<Long> getLogRecordCountAsync(final DLSN beginDLSN) {
+        return processReaderOperation(new Function<BKLogReadHandler, Future<Long>>() {
+                    @Override
+                    public Future<Long> apply(BKLogReadHandler ledgerHandler) {
+                        return ledgerHandler.asyncGetLogRecordCount(beginDLSN);
+                    }
+                });
+    }
+
+    @Override
+    public void recover() throws IOException {
+        recoverInternal(conf.getUnpartitionedStreamName());
+    }
+
+    /**
+     * Recover a specified stream within the log container
+     * The writer implicitly recovers a topic when it resumes writing.
+     * This allows applications to recover a container explicitly so
+     * that application may read a fully recovered log before resuming
+     * the writes
+     *
+     * @throws IOException if the recovery fails
+     */
+    private void recoverInternal(String streamIdentifier) throws IOException {
+        checkClosedOrInError("recoverInternal");
+        BKLogWriteHandler ledgerHandler = createWriteHandler(true);
+        try {
+            FutureUtils.result(ledgerHandler.recoverIncompleteLogSegments());
+        } finally {
+            Utils.closeQuietly(ledgerHandler);
+        }
+    }
+
+    /**
+     * Delete all the partitions of the specified log
+     *
+     * @throws IOException if the deletion fails
+     */
+    @Override
+    public void delete() throws IOException {
+        FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
+                .deleteLog(uri, getStreamName()));
+    }
+
+    /**
+     * The DistributedLogManager may archive/purge any logs for transactionId
+     * less than or equal to minImageTxId.
+     * This is to be used only when the client explicitly manages deletion. If
+     * the cleanup policy is based on sliding time window, then this method need
+     * not be called.
+     *
+     * @param minTxIdToKeep the earliest txid that must be retained
+     * @throws IOException if purging fails
+     */
+    @Override
+    public void purgeLogsOlderThan(long minTxIdToKeep) throws IOException {
+        Preconditions.checkArgument(minTxIdToKeep > 0, "Invalid transaction id " + minTxIdToKeep);
+        checkClosedOrInError("purgeLogSegmentsOlderThan");
+        BKLogWriteHandler ledgerHandler = createWriteHandler(true);
+        try {
+            LOG.info("Purging logs for {} older than {}", ledgerHandler.getFullyQualifiedName(), minTxIdToKeep);
+            FutureUtils.result(ledgerHandler.purgeLogSegmentsOlderThanTxnId(minTxIdToKeep));
+        } finally {
+            Utils.closeQuietly(ledgerHandler);
+        }
+    }
+
+    static class PendingReaders implements AsyncCloseable {
+
+        final ExecutorService executorService;
+        final Set<AsyncCloseable> readers = new HashSet<AsyncCloseable>();
+
+        PendingReaders(ExecutorService executorService) {
+            this.executorService = executorService;
+        }
+
+        public synchronized void remove(AsyncCloseable reader) {
+            readers.remove(reader);
+        }
+
+        public synchronized void add(AsyncCloseable reader) {
+            readers.add(reader);
+        }
+
+        @Override
+        public Future<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;
+                        }
+                    });
+        }
+    };
+
+    /**
+     * Close the distributed log manager, freeing any resources it may hold.
+     */
+    @Override
+    public Future<Void> asyncClose() {
+        Promise<Void> closeFuture;
+        BKLogReadHandler readHandlerToClose;
+        synchronized (this) {
+            if (null != closePromise) {
+                return closePromise;
+            }
+            closeFuture = closePromise = new Promise<Void>();
+            readHandlerToClose = readHandlerForListener;
+        }
+
+        Future<Void> closeResult = Utils.closeSequence(null, true,
+                readHandlerToClose,
+                pendingReaders,
+                resourcesCloseable.or(AsyncCloseable.NULL));
+        closeResult.proxyTo(closeFuture);
+        return closeFuture;
+    }
+
+    @Override
+    public void close() throws IOException {
+        FutureUtils.result(asyncClose());
+    }
+
+    @Override
+    public String toString() {
+        return String.format("DLM:%s:%s", getUri(), getStreamName());
+    }
+
+    public void raiseAlert(String msg, Object... args) {
+        alertStatsLogger.raise(msg, args);
+    }
+
+    @Override
+    public SubscriptionsStore getSubscriptionsStore() {
+        return driver.getSubscriptionsStore(getStreamName());
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
new file mode 100644
index 0000000..0a4608e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
@@ -0,0 +1,320 @@
+/**
+ * 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.base.Optional;
+import com.google.common.base.Ticker;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.callback.NamespaceListener;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.InvalidStreamNameException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+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.util.Utils;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Iterator;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.distributedlog.namespace.NamespaceDriver.Role.WRITER;
+import static org.apache.distributedlog.util.DLUtils.validateName;
+
+/**
+ * BKDistributedLogNamespace is the default implementation of {@link DistributedLogNamespace}. It uses
+ * zookeeper for metadata storage and bookkeeper for data storage.
+ * <h3>Metrics</h3>
+ *
+ * <h4>ZooKeeper Client</h4>
+ * See {@link ZooKeeperClient} for detail sub-stats.
+ * <ul>
+ * <li> `scope`/dlzk_factory_writer_shared/* : stats about the zookeeper client shared by all DL writers.
+ * <li> `scope`/dlzk_factory_reader_shared/* : stats about the zookeeper client shared by all DL readers.
+ * <li> `scope`/bkzk_factory_writer_shared/* : stats about the zookeeper client used by bookkeeper client
+ * shared by all DL writers.
+ * <li> `scope`/bkzk_factory_reader_shared/* : stats about the zookeeper client used by bookkeeper client
+ * shared by all DL readers.
+ * </ul>
+ *
+ * <h4>BookKeeper Client</h4>
+ * BookKeeper client stats are exposed directly to current scope. See {@link BookKeeperClient} for detail stats.
+ *
+ * <h4>Utils</h4>
+ * <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>
+ *
+ * <h4>DistributedLogManager</h4>
+ *
+ * All the core stats about reader and writer are exposed under current scope via {@link BKDistributedLogManager}.
+ */
+public class BKDistributedLogNamespace implements DistributedLogNamespace {
+    static final Logger LOG = LoggerFactory.getLogger(BKDistributedLogNamespace.class);
+
+    private final String clientId;
+    private final int regionId;
+    private final DistributedLogConfiguration conf;
+    private final URI namespace;
+    // namespace driver
+    private final NamespaceDriver driver;
+    // resources
+    private final OrderedScheduler scheduler;
+    private final PermitLimiter writeLimiter;
+    private final AsyncFailureInjector failureInjector;
+    // log segment metadata store
+    private final LogSegmentMetadataCache logSegmentMetadataCache;
+    // feature provider
+    private final FeatureProvider featureProvider;
+    // Stats Loggers
+    private final StatsLogger statsLogger;
+    private final StatsLogger perLogStatsLogger;
+
+    protected final AtomicBoolean closed = new AtomicBoolean(false);
+
+    public BKDistributedLogNamespace(
+            DistributedLogConfiguration conf,
+            URI uri,
+            NamespaceDriver driver,
+            OrderedScheduler scheduler,
+            FeatureProvider featureProvider,
+            PermitLimiter writeLimiter,
+            AsyncFailureInjector failureInjector,
+            StatsLogger statsLogger,
+            StatsLogger perLogStatsLogger,
+            String clientId,
+            int regionId) {
+        this.conf = conf;
+        this.namespace = uri;
+        this.driver = driver;
+        this.scheduler = scheduler;
+        this.featureProvider = featureProvider;
+        this.writeLimiter = writeLimiter;
+        this.failureInjector = failureInjector;
+        this.statsLogger = statsLogger;
+        this.perLogStatsLogger = perLogStatsLogger;
+        this.clientId = clientId;
+        this.regionId = regionId;
+
+        // create a log segment metadata cache
+        this.logSegmentMetadataCache = new LogSegmentMetadataCache(conf, Ticker.systemTicker());
+    }
+
+    @Override
+    public NamespaceDriver getNamespaceDriver() {
+        return driver;
+    }
+
+    //
+    // Namespace Methods
+    //
+
+    @Override
+    public void createLog(String logName)
+            throws InvalidStreamNameException, IOException {
+        checkState();
+        validateName(logName);
+        URI uri = FutureUtils.result(driver.getLogMetadataStore().createLog(logName));
+        FutureUtils.result(driver.getLogStreamMetadataStore(WRITER).getLog(uri, logName, true, true));
+    }
+
+    @Override
+    public void deleteLog(String logName)
+            throws InvalidStreamNameException, LogNotFoundException, IOException {
+        checkState();
+        validateName(logName);
+        Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
+        if (!uri.isPresent()) {
+            throw new LogNotFoundException("Log " + logName + " isn't found.");
+        }
+        DistributedLogManager dlm = openLogInternal(
+                uri.get(),
+                logName,
+                Optional.<DistributedLogConfiguration>absent(),
+                Optional.<DynamicDistributedLogConfiguration>absent());
+        dlm.delete();
+    }
+
+    @Override
+    public DistributedLogManager openLog(String logName)
+            throws InvalidStreamNameException, IOException {
+        return openLog(logName,
+                Optional.<DistributedLogConfiguration>absent(),
+                Optional.<DynamicDistributedLogConfiguration>absent(),
+                Optional.<StatsLogger>absent());
+    }
+
+    @Override
+    public DistributedLogManager openLog(String logName,
+                                         Optional<DistributedLogConfiguration> logConf,
+                                         Optional<DynamicDistributedLogConfiguration> dynamicLogConf,
+                                         Optional<StatsLogger> perStreamStatsLogger)
+            throws InvalidStreamNameException, IOException {
+        checkState();
+        validateName(logName);
+        Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
+        if (!uri.isPresent()) {
+            throw new LogNotFoundException("Log " + logName + " isn't found.");
+        }
+        return openLogInternal(
+                uri.get(),
+                logName,
+                logConf,
+                dynamicLogConf);
+    }
+
+    @Override
+    public boolean logExists(String logName)
+        throws IOException, IllegalArgumentException {
+        checkState();
+        Optional<URI> uri = FutureUtils.result(driver.getLogMetadataStore().getLogLocation(logName));
+        if (uri.isPresent()) {
+            try {
+                FutureUtils.result(driver.getLogStreamMetadataStore(WRITER)
+                        .logExists(uri.get(), logName));
+                return true;
+            } catch (LogNotFoundException lnfe) {
+                return false;
+            }
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public Iterator<String> getLogs() throws IOException {
+        checkState();
+        return FutureUtils.result(driver.getLogMetadataStore().getLogs());
+    }
+
+    @Override
+    public void registerNamespaceListener(NamespaceListener listener) {
+        driver.getLogMetadataStore().registerNamespaceListener(listener);
+    }
+
+    @Override
+    public synchronized AccessControlManager createAccessControlManager() throws IOException {
+        checkState();
+        return driver.getAccessControlManager();
+    }
+
+    /**
+     * Open the log in location <i>uri</i>.
+     *
+     * @param uri
+     *          location to store the log
+     * @param nameOfLogStream
+     *          name of the log
+     * @param logConfiguration
+     *          optional stream configuration
+     * @param dynamicLogConfiguration
+     *          dynamic stream configuration overrides.
+     * @return distributedlog manager instance.
+     * @throws InvalidStreamNameException if the stream name is invalid
+     * @throws IOException
+     */
+    protected DistributedLogManager openLogInternal(
+            URI uri,
+            String nameOfLogStream,
+            Optional<DistributedLogConfiguration> logConfiguration,
+            Optional<DynamicDistributedLogConfiguration> dynamicLogConfiguration)
+        throws InvalidStreamNameException, IOException {
+        // Make sure the name is well formed
+        checkState();
+        validateName(nameOfLogStream);
+
+        DistributedLogConfiguration mergedConfiguration = new DistributedLogConfiguration();
+        mergedConfiguration.addConfiguration(conf);
+        mergedConfiguration.loadStreamConf(logConfiguration);
+        // If dynamic config was not provided, default to a static view of the global configuration.
+        DynamicDistributedLogConfiguration dynConf = null;
+        if (dynamicLogConfiguration.isPresent()) {
+            dynConf = dynamicLogConfiguration.get();
+        } else {
+            dynConf = ConfUtils.getConstDynConf(mergedConfiguration);
+        }
+
+        return new BKDistributedLogManager(
+                nameOfLogStream,                    /* Log Name */
+                mergedConfiguration,                /* Configuration */
+                dynConf,                            /* Dynamic Configuration */
+                uri,                                /* Namespace URI */
+                driver,                             /* Namespace Driver */
+                logSegmentMetadataCache,            /* Log Segment Metadata Cache */
+                scheduler,                          /* DL scheduler */
+                clientId,                           /* Client Id */
+                regionId,                           /* Region Id */
+                writeLimiter,                       /* Write Limiter */
+                featureProvider.scope("dl"),        /* Feature Provider */
+                failureInjector,                    /* Failure Injector */
+                statsLogger,                        /* Stats Logger */
+                perLogStatsLogger,                  /* Per Log Stats Logger */
+                Optional.<AsyncCloseable>absent()   /* shared resources, we don't need to close any resources in dlm */
+        );
+    }
+
+    /**
+     * Check the namespace state.
+     *
+     * @throws IOException
+     */
+    private void checkState() throws IOException {
+        if (closed.get()) {
+            LOG.error("BK namespace {} is already closed", namespace);
+            throw new AlreadyClosedException("BK namespace " + namespace + " is already closed");
+        }
+    }
+
+    /**
+     * Close the distributed log manager factory, freeing any resources it may hold.
+     */
+    @Override
+    public void close() {
+        if (!closed.compareAndSet(false, true)) {
+            return;
+        }
+        // shutdown the driver
+        Utils.close(driver);
+        // close the write limiter
+        this.writeLimiter.close();
+        // Shutdown the schedulers
+        SchedulerUtils.shutdownScheduler(scheduler, conf.getSchedulerShutdownTimeoutMs(),
+                TimeUnit.MILLISECONDS);
+        LOG.info("Executor Service Stopped.");
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java
new file mode 100644
index 0000000..07ae0ff
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java
@@ -0,0 +1,715 @@
+/**
+ * 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.base.Stopwatch;
+import org.apache.distributedlog.callback.LogSegmentNamesListener;
+import org.apache.distributedlog.exceptions.LogEmptyException;
+import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.metadata.LogMetadata;
+import org.apache.distributedlog.io.AsyncAbortable;
+import org.apache.distributedlog.io.AsyncCloseable;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
+import org.apache.distributedlog.logsegment.PerStreamLogSegmentCache;
+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.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.
+ *
+ * <h3>Metrics</h3>
+ * The log handler is a base class on managing log segments. so all the metrics
+ * here are related to log segments retrieval and exposed under `logsegments`.
+ * These metrics are all OpStats, in the format of <code>`scope`/logsegments/`op`</code>.
+ * <p>
+ * Those operations are:
+ * <ul>
+ * <li>get_inprogress_segment: time between the inprogress log segment created and
+ * the handler read it.
+ * <li>get_completed_segment: time between a log segment is turned to completed and
+ * the handler read it.
+ * <li>negative_get_inprogress_segment: record the negative values for `get_inprogress_segment`.
+ * <li>negative_get_completed_segment: record the negative values for `get_completed_segment`.
+ * <li>recover_last_entry: recovering last entry from a log segment
+ * <li>recover_scanned_entries: the number of entries that are scanned during recovering.
+ * </ul>
+ * @see BKLogWriteHandler
+ * @see BKLogReadHandler
+ */
+public abstract class BKLogHandler implements AsyncCloseable, AsyncAbortable {
+    static final Logger LOG = LoggerFactory.getLogger(BKLogHandler.class);
+
+    protected final LogMetadata logMetadata;
+    protected final DistributedLogConfiguration conf;
+    protected final LogStreamMetadataStore streamMetadataStore;
+    protected final LogSegmentMetadataStore metadataStore;
+    protected final LogSegmentMetadataCache metadataCache;
+    protected final LogSegmentEntryStore entryStore;
+    protected final int firstNumEntriesPerReadLastRecordScan;
+    protected final int maxNumEntriesPerReadLastRecordScan;
+    protected volatile long lastLedgerRollingTimeMillis = -1;
+    protected final OrderedScheduler scheduler;
+    protected final StatsLogger statsLogger;
+    protected final AlertStatsLogger alertStatsLogger;
+    protected volatile boolean reportGetSegmentStats = false;
+    private final String lockClientId;
+    protected final AtomicReference<IOException> metadataException = new AtomicReference<IOException>(null);
+
+    // Maintain the list of log segments per stream
+    protected final PerStreamLogSegmentCache logSegmentCache;
+
+    // trace
+    protected final long metadataLatencyWarnThresholdMillis;
+
+    // Stats
+    private final OpStatsLogger getInprogressSegmentStat;
+    private final OpStatsLogger getCompletedSegmentStat;
+    private final OpStatsLogger negativeGetInprogressSegmentStat;
+    private final OpStatsLogger negativeGetCompletedSegmentStat;
+    private final OpStatsLogger recoverLastEntryStats;
+    private final OpStatsLogger recoverScannedEntriesStats;
+
+    /**
+     * Construct a Bookkeeper journal manager.
+     */
+    BKLogHandler(LogMetadata metadata,
+                 DistributedLogConfiguration conf,
+                 LogStreamMetadataStore streamMetadataStore,
+                 LogSegmentMetadataCache metadataCache,
+                 LogSegmentEntryStore entryStore,
+                 OrderedScheduler scheduler,
+                 StatsLogger statsLogger,
+                 AlertStatsLogger alertStatsLogger,
+                 String lockClientId) {
+        this.logMetadata = metadata;
+        this.conf = conf;
+        this.scheduler = scheduler;
+        this.statsLogger = statsLogger;
+        this.alertStatsLogger = alertStatsLogger;
+        this.logSegmentCache = new PerStreamLogSegmentCache(
+                metadata.getLogName(),
+                conf.isLogSegmentSequenceNumberValidationEnabled());
+        firstNumEntriesPerReadLastRecordScan = conf.getFirstNumEntriesPerReadLastRecordScan();
+        maxNumEntriesPerReadLastRecordScan = conf.getMaxNumEntriesPerReadLastRecordScan();
+        this.streamMetadataStore = streamMetadataStore;
+        this.metadataStore = streamMetadataStore.getLogSegmentMetadataStore();
+        this.metadataCache = metadataCache;
+        this.entryStore = entryStore;
+        this.lockClientId = lockClientId;
+
+        // Traces
+        this.metadataLatencyWarnThresholdMillis = conf.getMetadataLatencyWarnThresholdMillis();
+
+        // Stats
+        StatsLogger segmentsLogger = statsLogger.scope("logsegments");
+        getInprogressSegmentStat = segmentsLogger.getOpStatsLogger("get_inprogress_segment");
+        getCompletedSegmentStat = segmentsLogger.getOpStatsLogger("get_completed_segment");
+        negativeGetInprogressSegmentStat = segmentsLogger.getOpStatsLogger("negative_get_inprogress_segment");
+        negativeGetCompletedSegmentStat = segmentsLogger.getOpStatsLogger("negative_get_completed_segment");
+        recoverLastEntryStats = segmentsLogger.getOpStatsLogger("recover_last_entry");
+        recoverScannedEntriesStats = segmentsLogger.getOpStatsLogger("recover_scanned_entries");
+    }
+
+    BKLogHandler checkMetadataException() throws IOException {
+        if (null != metadataException.get()) {
+            throw metadataException.get();
+        }
+        return this;
+    }
+
+    public void reportGetSegmentStats(boolean enabled) {
+        this.reportGetSegmentStats = enabled;
+    }
+
+    public String getLockClientId() {
+        return lockClientId;
+    }
+
+    public Future<LogRecordWithDLSN> asyncGetFirstLogRecord() {
+        final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
+        streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
+                .addEventListener(new FutureEventListener<Void>() {
+            @Override
+            public void onSuccess(Void value) {
+                readLogSegmentsFromStore(
+                        LogSegmentMetadata.COMPARATOR,
+                        LogSegmentFilter.DEFAULT_FILTER,
+                        null
+                ).addEventListener(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"));
+                            return;
+                        }
+                        Future<LogRecordWithDLSN> firstRecord = null;
+                        for (LogSegmentMetadata ledger : ledgerList.getValue()) {
+                            if (!ledger.isTruncated() && (ledger.getRecordCount() > 0 || ledger.isInProgress())) {
+                                firstRecord = asyncReadFirstUserRecord(ledger, DLSN.InitialDLSN);
+                                break;
+                            }
+                        }
+                        if (null != firstRecord) {
+                            promise.become(firstRecord);
+                        } else {
+                            promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
+                        }
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        promise.setException(cause);
+                    }
+                });
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                promise.setException(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>() {
+            @Override
+            public void onSuccess(Void value) {
+                readLogSegmentsFromStore(
+                        LogSegmentMetadata.DESC_COMPARATOR,
+                        LogSegmentFilter.DEFAULT_FILTER,
+                        null
+                ).addEventListener(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"));
+                            return;
+                        }
+                        asyncGetLastLogRecord(
+                                ledgerList.getValue().iterator(),
+                                promise,
+                                recover,
+                                false,
+                                includeEndOfStream);
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        promise.setException(cause);
+                    }
+                });
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                promise.setException(cause);
+            }
+        });
+        return promise;
+    }
+
+    private void asyncGetLastLogRecord(final Iterator<LogSegmentMetadata> ledgerIter,
+                                       final Promise<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);
+                        }
+                    }
+            );
+        } else {
+            promise.setException(new LogEmptyException("Log " + getFullyQualifiedName() + " has no records"));
+        }
+    }
+
+    private Future<LogRecordWithDLSN> asyncReadFirstUserRecord(LogSegmentMetadata ledger, DLSN beginDLSN) {
+        return ReadUtils.asyncReadFirstUserRecord(
+                getFullyQualifiedName(),
+                ledger,
+                firstNumEntriesPerReadLastRecordScan,
+                maxNumEntriesPerReadLastRecordScan,
+                new AtomicInteger(0),
+                scheduler,
+                entryStore,
+                beginDLSN
+        );
+    }
+
+    /**
+     * This is a helper method to compactly return the record count between two records, the first denoted by
+     * 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;
+            }
+        });
+    }
+
+    /**
+     * Ledger metadata tells us how many records are in each completed segment, but for the first and last segments
+     * we may have to crack open the entry and count. For the first entry, we need to do so because beginDLSN may be
+     * 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) {
+        if (ledger.isInProgress() && ledger.isDLSNinThisSegment(beginDLSN)) {
+            return asyncReadLastUserRecord(ledger).flatMap(new Function<LogRecordWithDLSN, Future<Long>>() {
+                public Future<Long> apply(final LogRecordWithDLSN endRecord) {
+                    if (null != endRecord) {
+                        return asyncGetLogRecordCount(ledger, beginDLSN, endRecord.getLastPositionWithinLogSegment() /* end position */);
+                    } else {
+                        return Future.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;
+                    }
+                }
+            });
+        } else if (ledger.isDLSNinThisSegment(beginDLSN)) {
+            return asyncGetLogRecordCount(ledger, beginDLSN, ledger.getRecordCount() /* end position */);
+        } else {
+            return Future.value((long) ledger.getRecordCount());
+        }
+    }
+
+    /**
+     * Get a count of records between beginDLSN and the end of the stream.
+     *
+     * @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) {
+        return streamMetadataStore.logExists(logMetadata.getUri(), logMetadata.getLogName())
+                .flatMap(new Function<Void, Future<Long>>() {
+            public Future<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) {
+
+                        List<Future<Long>> futureCounts = new ArrayList<Future<Long>>(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);
+                            }
+                        });
+                    }
+                });
+            }
+        });
+    }
+
+    private Long sum(List<Long> values) {
+        long sum = 0;
+        for (Long value : values) {
+            sum += value;
+        }
+        return sum;
+    }
+
+    @Override
+    public Future<Void> asyncAbort() {
+        return asyncClose();
+    }
+
+    public Future<LogRecordWithDLSN> asyncReadLastUserRecord(final LogSegmentMetadata l) {
+        return asyncReadLastRecord(l, false, false, false);
+    }
+
+    public Future<LogRecordWithDLSN> asyncReadLastRecord(final LogSegmentMetadata l,
+                                                         final boolean fence,
+                                                         final boolean includeControl,
+                                                         final boolean includeEndOfStream) {
+        final AtomicInteger numRecordsScanned = new AtomicInteger(0);
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+        return ReadUtils.asyncReadLastRecord(
+                getFullyQualifiedName(),
+                l,
+                fence,
+                includeControl,
+                includeEndOfStream,
+                firstNumEntriesPerReadLastRecordScan,
+                maxNumEntriesPerReadLastRecordScan,
+                numRecordsScanned,
+                scheduler,
+                entryStore
+        ).addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+            @Override
+            public void onSuccess(LogRecordWithDLSN value) {
+                recoverLastEntryStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+                recoverScannedEntriesStats.registerSuccessfulEvent(numRecordsScanned.get());
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                recoverLastEntryStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            }
+        });
+    }
+
+    protected void setLastLedgerRollingTimeMillis(long rollingTimeMillis) {
+        if (lastLedgerRollingTimeMillis < rollingTimeMillis) {
+            lastLedgerRollingTimeMillis = rollingTimeMillis;
+        }
+    }
+
+    public String getFullyQualifiedName() {
+        return logMetadata.getFullyQualifiedName();
+    }
+
+    // Log Segments Related Functions
+    //
+    // ***Note***
+    // Get log segment list should go through #getCachedLogSegments as we need to assign start sequence id
+    // for inprogress log segment so the reader could generate the right sequence id.
+    //
+    // ***PerStreamCache vs LogSegmentMetadataCache **
+    // The per stream cache maintains the list of segments per stream, while the metadata cache
+    // maintains log segments. The metadata cache is just to reduce the access to zookeeper, it is
+    // okay that some of the log segments are not in the cache; however the per stream cache can not
+    // have any gaps between log segment sequence numbers which it has to be accurate.
+
+    /**
+     * Get the cached log segments.
+     *
+     * @param comparator the comparator to sort the returned log segments.
+     * @return list of sorted log segments
+     * @throws UnexpectedException if unexpected condition detected.
+     */
+    protected List<LogSegmentMetadata> getCachedLogSegments(Comparator<LogSegmentMetadata> comparator)
+        throws UnexpectedException {
+        try {
+            return logSegmentCache.getLogSegments(comparator);
+        } catch (UnexpectedException ue) {
+            // the log segments cache went wrong
+            LOG.error("Unexpected exception on getting log segments from the cache for stream {}",
+                    getFullyQualifiedName(), ue);
+            metadataException.compareAndSet(null, ue);
+            throw ue;
+        }
+    }
+
+    /**
+     * Add the segment <i>metadata</i> for <i>name</i> in the cache.
+     *
+     * @param name
+     *          segment znode name.
+     * @param metadata
+     *          segment metadata.
+     */
+    protected void addLogSegmentToCache(String name, LogSegmentMetadata metadata) {
+        metadataCache.put(metadata.getZkPath(), metadata);
+        logSegmentCache.add(name, metadata);
+        // update the last ledger rolling time
+        if (!metadata.isInProgress() && (lastLedgerRollingTimeMillis < metadata.getCompletionTime())) {
+            lastLedgerRollingTimeMillis = metadata.getCompletionTime();
+        }
+
+        if (reportGetSegmentStats) {
+            // update stats
+            long ts = System.currentTimeMillis();
+            if (metadata.isInProgress()) {
+                // as we used timestamp as start tx id we could take it as start time
+                // NOTE: it is a hack here.
+                long elapsedMillis = ts - metadata.getFirstTxId();
+                long elapsedMicroSec = TimeUnit.MILLISECONDS.toMicros(elapsedMillis);
+                if (elapsedMicroSec > 0) {
+                    if (elapsedMillis > metadataLatencyWarnThresholdMillis) {
+                        LOG.warn("{} received inprogress log segment in {} millis: {}",
+                                 new Object[] { getFullyQualifiedName(), elapsedMillis, metadata });
+                    }
+                    getInprogressSegmentStat.registerSuccessfulEvent(elapsedMicroSec);
+                } else {
+                    negativeGetInprogressSegmentStat.registerSuccessfulEvent(-elapsedMicroSec);
+                }
+            } else {
+                long elapsedMillis = ts - metadata.getCompletionTime();
+                long elapsedMicroSec = TimeUnit.MILLISECONDS.toMicros(elapsedMillis);
+                if (elapsedMicroSec > 0) {
+                    if (elapsedMillis > metadataLatencyWarnThresholdMillis) {
+                        LOG.warn("{} received completed log segment in {} millis : {}",
+                                 new Object[] { getFullyQualifiedName(), elapsedMillis, metadata });
+                    }
+                    getCompletedSegmentStat.registerSuccessfulEvent(elapsedMicroSec);
+                } else {
+                    negativeGetCompletedSegmentStat.registerSuccessfulEvent(-elapsedMicroSec);
+                }
+            }
+        }
+    }
+
+    /**
+     * Read log segment <i>name</i> from the cache.
+     *
+     * @param name name of the log segment
+     * @return log segment metadata
+     */
+    protected LogSegmentMetadata readLogSegmentFromCache(String name) {
+        return logSegmentCache.get(name);
+    }
+
+    /**
+     * Remove the log segment <i>name</i> from the cache.
+     *
+     * @param name name of the log segment.
+     * @return log segment metadata
+     */
+    protected LogSegmentMetadata removeLogSegmentFromCache(String name) {
+        metadataCache.invalidate(name);
+        return logSegmentCache.remove(name);
+    }
+
+    /**
+     * Update the log segment cache with updated mapping
+     *
+     * @param logSegmentsRemoved log segments removed
+     * @param logSegmentsAdded log segments added
+     */
+    protected void updateLogSegmentCache(Set<String> logSegmentsRemoved,
+                                         Map<String, LogSegmentMetadata> logSegmentsAdded) {
+        for (String segmentName : logSegmentsRemoved) {
+            metadataCache.invalidate(segmentName);
+        }
+        for (Map.Entry<String, LogSegmentMetadata> entry : logSegmentsAdded.entrySet()) {
+            metadataCache.put(entry.getKey(), entry.getValue());
+        }
+        logSegmentCache.update(logSegmentsRemoved, logSegmentsAdded);
+    }
+
+    /**
+     * Read the log segments from the store and register a listener
+     * @param comparator
+     * @param segmentFilter
+     * @param logSegmentNamesListener
+     * @return future represents the result of log segments
+     */
+    public Future<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>>>();
+        metadataStore.getLogSegmentNames(logMetadata.getLogSegmentsPath(), logSegmentNamesListener)
+                .addEventListener(new FutureEventListener<Versioned<List<String>>>() {
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        FutureUtils.setException(readResult, cause);
+                    }
+
+                    @Override
+                    public void onSuccess(Versioned<List<String>> logSegmentNames) {
+                        readLogSegmentsFromStore(logSegmentNames, comparator, segmentFilter, readResult);
+                    }
+                });
+        return readResult;
+    }
+
+    protected void readLogSegmentsFromStore(final Versioned<List<String>> logSegmentNames,
+                                            final Comparator<LogSegmentMetadata> comparator,
+                                            final LogSegmentFilter segmentFilter,
+                                            final Promise<Versioned<List<LogSegmentMetadata>>> readResult) {
+        Set<String> segmentsReceived = new HashSet<String>();
+        segmentsReceived.addAll(segmentFilter.filter(logSegmentNames.getValue()));
+        Set<String> segmentsAdded;
+        final Set<String> removedSegments = Collections.synchronizedSet(new HashSet<String>());
+        final Map<String, LogSegmentMetadata> addedSegments =
+                Collections.synchronizedMap(new HashMap<String, LogSegmentMetadata>());
+        Pair<Set<String>, Set<String>> segmentChanges = logSegmentCache.diff(segmentsReceived);
+        segmentsAdded = segmentChanges.getLeft();
+        removedSegments.addAll(segmentChanges.getRight());
+
+        if (segmentsAdded.isEmpty()) {
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("No segments added for {}.", getFullyQualifiedName());
+            }
+
+            // update the cache before #getCachedLogSegments to return
+            updateLogSegmentCache(removedSegments, addedSegments);
+
+            List<LogSegmentMetadata> segmentList;
+            try {
+                segmentList = getCachedLogSegments(comparator);
+            } catch (UnexpectedException e) {
+                FutureUtils.setException(readResult, e);
+                return;
+            }
+
+            FutureUtils.setValue(readResult,
+                    new Versioned<List<LogSegmentMetadata>>(segmentList, logSegmentNames.getVersion()));
+            return;
+        }
+
+        final AtomicInteger numChildren = new AtomicInteger(segmentsAdded.size());
+        final AtomicInteger numFailures = new AtomicInteger(0);
+        for (final String segment: segmentsAdded) {
+            String logSegmentPath = logMetadata.getLogSegmentPath(segment);
+            LogSegmentMetadata cachedSegment = metadataCache.get(logSegmentPath);
+            if (null != cachedSegment) {
+                addedSegments.put(segment, cachedSegment);
+                completeReadLogSegmentsFromStore(
+                        removedSegments,
+                        addedSegments,
+                        comparator,
+                        readResult,
+                        logSegmentNames.getVersion(),
+                        numChildren,
+                        numFailures);
+                continue;
+            }
+            metadataStore.getLogSegment(logSegmentPath)
+                    .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+
+                        @Override
+                        public void onSuccess(LogSegmentMetadata result) {
+                            addedSegments.put(segment, result);
+                            complete();
+                        }
+
+                        @Override
+                        public void onFailure(Throwable cause) {
+                            // LogSegmentNotFoundException exception is possible in two cases
+                            // 1. A log segment was deleted by truncation between the call to getChildren and read
+                            // attempt on the znode corresponding to the segment
+                            // 2. In progress segment has been completed => inprogress ZNode does not exist
+                            if (cause instanceof LogSegmentNotFoundException) {
+                                removedSegments.add(segment);
+                                complete();
+                            } else {
+                                // fail fast
+                                if (1 == numFailures.incrementAndGet()) {
+                                    FutureUtils.setException(readResult, cause);
+                                    return;
+                                }
+                            }
+                        }
+
+                        private void complete() {
+                            completeReadLogSegmentsFromStore(
+                                    removedSegments,
+                                    addedSegments,
+                                    comparator,
+                                    readResult,
+                                    logSegmentNames.getVersion(),
+                                    numChildren,
+                                    numFailures);
+                        }
+                    });
+        }
+    }
+
+    private void completeReadLogSegmentsFromStore(final Set<String> removedSegments,
+                                                  final Map<String, LogSegmentMetadata> addedSegments,
+                                                  final Comparator<LogSegmentMetadata> comparator,
+                                                  final Promise<Versioned<List<LogSegmentMetadata>>> readResult,
+                                                  final Version logSegmentNamesVersion,
+                                                  final AtomicInteger numChildren,
+                                                  final AtomicInteger numFailures) {
+        if (0 != numChildren.decrementAndGet()) {
+            return;
+        }
+        if (numFailures.get() > 0) {
+            return;
+        }
+        // update the cache only when fetch completed and before #getCachedLogSegments
+        updateLogSegmentCache(removedSegments, addedSegments);
+        List<LogSegmentMetadata> segmentList;
+        try {
+            segmentList = getCachedLogSegments(comparator);
+        } catch (UnexpectedException e) {
+            FutureUtils.setException(readResult, e);
+            return;
+        }
+        FutureUtils.setValue(readResult,
+            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
new file mode 100644
index 0000000..c6e2e07
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
@@ -0,0 +1,431 @@
+/**
+ * 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 java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.callback.LogSegmentNamesListener;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.metadata.LogMetadataForReader;
+import org.apache.distributedlog.lock.DistributedLock;
+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.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.
+ * <h3>Metrics</h3>
+ *
+ * <h4>ReadAhead Worker</h4>
+ * Most of readahead stats are exposed under scope `readahead_worker`. Only readahead exceptions are exposed
+ * in parent scope via <code>readAheadExceptionsLogger</code>.
+ * <ul>
+ * <li> `readahead_worker`/wait: counter. number of waits that readahead worker is waiting. If this keeps increasing,
+ * it usually means readahead keep getting full because of reader slows down reading.
+ * <li> `readahead_worker`/repositions: counter. number of repositions that readhead worker encounters. reposition
+ * means that a readahead worker finds that it isn't advancing to a new log segment and force re-positioning.
+ * <li> `readahead_worker`/entry_piggy_back_hits: counter. it increases when the last add confirmed being advanced
+ * because of the piggy-back lac.
+ * <li> `readahead_worker`/entry_piggy_back_misses: counter. it increases when the last add confirmed isn't advanced
+ * by a read entry because it doesn't piggy back a newer lac.
+ * <li> `readahead_worker`/read_entries: opstats. stats on number of entries read per readahead read batch.
+ * <li> `readahead_worker`/read_lac_counter: counter. stats on the number of readLastConfirmed operations
+ * <li> `readahead_worker`/read_lac_and_entry_counter: counter. stats on the number of readLastConfirmedAndEntry
+ * operations.
+ * <li> `readahead_worker`/cache_full: counter. it increases each time readahead worker finds cache become full.
+ * If it keeps increasing, that means reader slows down reading.
+ * <li> `readahead_worker`/resume: opstats. stats on readahead worker resuming reading from wait state.
+ * <li> `readahead_worker`/read_lac_lag: opstats. stats on the number of entries diff between the lac reader knew
+ * last time and the lac that it received. if `lag` between two subsequent lacs is high, that might means delay
+ * might be high. because reader is only allowed to read entries after lac is advanced.
+ * <li> `readahead_worker`/long_poll_interruption: opstats. stats on the number of interruptions happened to long
+ * poll. the interruptions are usually because of receiving zookeeper notifications.
+ * <li> `readahead_worker`/notification_execution: opstats. stats on executions over the notifications received from
+ * zookeeper.
+ * <li> `readahead_worker`/metadata_reinitialization: opstats. stats on metadata reinitialization after receiving
+ * notifcation from log segments updates.
+ * <li> `readahead_worker`/idle_reader_warn: counter. it increases each time the readahead worker detects itself
+ * becoming idle.
+ * </ul>
+ * <h4>Read Lock</h4>
+ * All read lock related stats are exposed under scope `read_lock`.
+ * for detail stats.
+ */
+class BKLogReadHandler extends BKLogHandler implements LogSegmentNamesListener {
+    static final Logger LOG = LoggerFactory.getLogger(BKLogReadHandler.class);
+
+    protected final LogMetadataForReader logMetadataForReader;
+
+    protected final DynamicDistributedLogConfiguration dynConf;
+
+    private final Optional<String> subscriberId;
+    private DistributedLock readLock;
+    private Future<Void> lockAcquireFuture;
+
+    // notify the state change about the read handler
+    protected final AsyncNotification readerStateNotification;
+
+    // log segments listener
+    protected boolean logSegmentsNotificationDisabled = false;
+    protected final CopyOnWriteArraySet<LogSegmentListener> listeners =
+            new CopyOnWriteArraySet<LogSegmentListener>();
+    protected Versioned<List<LogSegmentMetadata>> lastNotifiedLogSegments =
+            new Versioned<List<LogSegmentMetadata>>(null, Version.NEW);
+
+    // stats
+    private final StatsLogger perLogStatsLogger;
+
+    /**
+     * Construct a Bookkeeper journal manager.
+     */
+    BKLogReadHandler(LogMetadataForReader logMetadata,
+                     Optional<String> subscriberId,
+                     DistributedLogConfiguration conf,
+                     DynamicDistributedLogConfiguration dynConf,
+                     LogStreamMetadataStore streamMetadataStore,
+                     LogSegmentMetadataCache metadataCache,
+                     LogSegmentEntryStore entryStore,
+                     OrderedScheduler scheduler,
+                     AlertStatsLogger alertStatsLogger,
+                     StatsLogger statsLogger,
+                     StatsLogger perLogStatsLogger,
+                     String clientId,
+                     AsyncNotification readerStateNotification,
+                     boolean isHandleForReading) {
+        super(logMetadata,
+                conf,
+                streamMetadataStore,
+                metadataCache,
+                entryStore,
+                scheduler,
+                statsLogger,
+                alertStatsLogger,
+                clientId);
+        this.logMetadataForReader = logMetadata;
+        this.dynConf = dynConf;
+        this.perLogStatsLogger =
+                isHandleForReading ? perLogStatsLogger : NullStatsLogger.INSTANCE;
+        this.readerStateNotification = readerStateNotification;
+        this.subscriberId = subscriberId;
+    }
+
+    @VisibleForTesting
+    String getReadLockPath() {
+        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() {
+        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() {
+        if (null == lockAcquireFuture) {
+            lockAcquireFuture = streamMetadataStore.createReadLock(logMetadataForReader, subscriberId)
+                    .flatMap(new ExceptionalFunction<DistributedLock, Future<Void>>() {
+                        @Override
+                        public Future<Void> applyE(DistributedLock lock) throws Throwable {
+                            BKLogReadHandler.this.readLock = lock;
+                            LOG.info("acquiring readlock {} at {}", getLockClientId(), getReadLockPath());
+                            return acquireLockOnExecutorThread(lock);
+                        }
+                    });
+        }
+        return lockAcquireFuture;
+    }
+
+    /**
+     * 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();
+
+        // 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;
+            }
+        });
+        acquireFuture.addEventListener(new FutureEventListener<DistributedLock>() {
+            @Override
+            public void onSuccess(DistributedLock lock) {
+                LOG.info("acquired readlock {} at {}", getLockClientId(), getReadLockPath());
+                satisfyPromiseAsync(threadAcquirePromise, new Return<Void>(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));
+            }
+        });
+        return threadAcquirePromise;
+    }
+
+    /**
+     * Check ownership of elective stream lock.
+     */
+    void checkReadLock() throws DLIllegalStateException, LockingException {
+        synchronized (this) {
+            if ((null == lockAcquireFuture) ||
+                (!lockAcquireFuture.isDefined())) {
+                throw new DLIllegalStateException("Attempt to check for lock before it has been acquired successfully");
+            }
+        }
+
+        readLock.checkOwnership();
+    }
+
+    public Future<Void> asyncClose() {
+        DistributedLock lockToClose;
+        synchronized (this) {
+            if (null != lockAcquireFuture && !lockAcquireFuture.isDefined()) {
+                FutureUtils.cancel(lockAcquireFuture);
+            }
+            lockToClose = readLock;
+        }
+        return Utils.closeSequence(scheduler, lockToClose)
+                .flatMap(new AbstractFunction1<Void, Future<Void>>() {
+            @Override
+            public Future<Void> apply(Void result) {
+                // unregister the log segment listener
+                metadataStore.unregisterLogSegmentListener(logMetadata.getLogSegmentsPath(), BKLogReadHandler.this);
+                return Future.Void();
+            }
+        });
+    }
+
+    @Override
+    public Future<Void> asyncAbort() {
+        return asyncClose();
+    }
+
+    /**
+     * Start fetch the log segments and register the {@link LogSegmentNamesListener}.
+     * The future is satisfied only on a successful fetch or encountered a fatal failure.
+     *
+     * @return future represents the fetch result
+     */
+    Future<Versioned<List<LogSegmentMetadata>>> asyncStartFetchLogSegments() {
+        Promise<Versioned<List<LogSegmentMetadata>>> promise =
+                new Promise<Versioned<List<LogSegmentMetadata>>>();
+        asyncStartFetchLogSegments(promise);
+        return promise;
+    }
+
+    void asyncStartFetchLogSegments(final Promise<Versioned<List<LogSegmentMetadata>>> promise) {
+        readLogSegmentsFromStore(
+                LogSegmentMetadata.COMPARATOR,
+                LogSegmentFilter.DEFAULT_FILTER,
+                this).addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                if (cause instanceof LogNotFoundException ||
+                        cause instanceof LogSegmentNotFoundException ||
+                        cause instanceof UnexpectedException) {
+                    // indicate some inconsistent behavior, abort
+                    metadataException.compareAndSet(null, (IOException) cause);
+                    // notify the reader that read handler is in error state
+                    notifyReaderOnError(cause);
+                    FutureUtils.setException(promise, cause);
+                    return;
+                }
+                scheduler.schedule(new Runnable() {
+                    @Override
+                    public void run() {
+                        asyncStartFetchLogSegments(promise);
+                    }
+                }, conf.getZKRetryBackoffMaxMillis(), TimeUnit.MILLISECONDS);
+            }
+
+            @Override
+            public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
+                // no-op
+                FutureUtils.setValue(promise, segments);
+            }
+        });
+    }
+
+    @VisibleForTesting
+    void disableReadAheadLogSegmentsNotification() {
+        logSegmentsNotificationDisabled = true;
+    }
+
+    @Override
+    public void onSegmentsUpdated(final Versioned<List<String>> segments) {
+        synchronized (this) {
+            if (lastNotifiedLogSegments.getVersion() != Version.NEW &&
+                    lastNotifiedLogSegments.getVersion().compare(segments.getVersion()) != Version.Occurred.BEFORE) {
+                // the log segments has been read, and it is possibly a retry from last segments update
+                return;
+            }
+        }
+
+        Promise<Versioned<List<LogSegmentMetadata>>> readLogSegmentsPromise =
+                new Promise<Versioned<List<LogSegmentMetadata>>>();
+        readLogSegmentsPromise.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                if (cause instanceof LogNotFoundException ||
+                        cause instanceof LogSegmentNotFoundException ||
+                        cause instanceof UnexpectedException) {
+                    // indicate some inconsistent behavior, abort
+                    metadataException.compareAndSet(null, (IOException) cause);
+                    // notify the reader that read handler is in error state
+                    notifyReaderOnError(cause);
+                    return;
+                }
+                scheduler.schedule(new Runnable() {
+                    @Override
+                    public void run() {
+                        onSegmentsUpdated(segments);
+                    }
+                }, conf.getZKRetryBackoffMaxMillis(), TimeUnit.MILLISECONDS);
+            }
+
+            @Override
+            public void onSuccess(Versioned<List<LogSegmentMetadata>> logSegments) {
+                List<LogSegmentMetadata> segmentsToNotify = null;
+                synchronized (BKLogReadHandler.this) {
+                    Versioned<List<LogSegmentMetadata>> lastLogSegments = lastNotifiedLogSegments;
+                    if (lastLogSegments.getVersion() == Version.NEW ||
+                            lastLogSegments.getVersion().compare(logSegments.getVersion()) == Version.Occurred.BEFORE) {
+                        lastNotifiedLogSegments = logSegments;
+                        segmentsToNotify = logSegments.getValue();
+                    }
+                }
+                if (null != segmentsToNotify) {
+                    notifyUpdatedLogSegments(segmentsToNotify);
+                }
+            }
+        });
+        // log segments list is updated, read their metadata
+        readLogSegmentsFromStore(
+                segments,
+                LogSegmentMetadata.COMPARATOR,
+                LogSegmentFilter.DEFAULT_FILTER,
+                readLogSegmentsPromise);
+    }
+
+    @Override
+    public void onLogStreamDeleted() {
+        notifyLogStreamDeleted();
+    }
+
+    //
+    // Listener for log segments
+    //
+
+    protected void registerListener(@Nullable LogSegmentListener listener) {
+        if (null != listener) {
+            listeners.add(listener);
+        }
+    }
+
+    protected void unregisterListener(@Nullable LogSegmentListener listener) {
+        if (null != listener) {
+            listeners.remove(listener);
+        }
+    }
+
+    protected void notifyUpdatedLogSegments(List<LogSegmentMetadata> segments) {
+        if (logSegmentsNotificationDisabled) {
+            return;
+        }
+
+        for (LogSegmentListener listener : listeners) {
+            List<LogSegmentMetadata> listToReturn =
+                    new ArrayList<LogSegmentMetadata>(segments);
+            Collections.sort(listToReturn, LogSegmentMetadata.COMPARATOR);
+            listener.onSegmentsUpdated(listToReturn);
+        }
+    }
+
+    protected void notifyLogStreamDeleted() {
+        if (logSegmentsNotificationDisabled) {
+            return;
+        }
+
+        for (LogSegmentListener listener : listeners) {
+            listener.onLogStreamDeleted();
+        }
+    }
+
+    // notify the errors
+    protected void notifyReaderOnError(Throwable cause) {
+        if (null != readerStateNotification) {
+            readerStateNotification.notifyOnError(cause);
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
new file mode 100644
index 0000000..6b60c77
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
@@ -0,0 +1,1348 @@
+/**
+ * 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 java.io.IOException;
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.concurrent.locks.ReentrantLock;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.BKTransmitException;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.FlushException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException;
+import org.apache.distributedlog.exceptions.WriteCancelledException;
+import org.apache.distributedlog.exceptions.WriteException;
+import org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException;
+import org.apache.distributedlog.feature.CoreFeatureKeys;
+import org.apache.distributedlog.injector.FailureInjector;
+import org.apache.distributedlog.injector.RandomDelayFailureInjector;
+import org.apache.distributedlog.io.Buffer;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.io.CompressionUtils;
+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.util.FailpointUtils;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.util.PermitLimiter;
+import org.apache.distributedlog.util.SafeQueueingFuturePool;
+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.bookkeeper.client.AsyncCallback.AddCallback;
+import org.apache.bookkeeper.client.AsyncCallback.CloseCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.AlertStatsLogger;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.MathUtils;
+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;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
+
+/**
+ * BookKeeper Based Log Segment Writer.
+ *
+ * Multiple log records are packed into a single bookkeeper
+ * entry before sending it over the network. The fact that the log record entries
+ * are complete in the bookkeeper entries means that each bookkeeper log entry
+ * can be read as a complete edit log. This is useful for reading, as we don't
+ * need to read through the entire log segment to get the last written entry.
+ *
+ * <h3>Metrics</h3>
+ *
+ * <ul>
+ * <li> flush/periodic/{success,miss}: counters for periodic flushes.
+ * <li> data/{success,miss}: counters for data transmits.
+ * <li> transmit/packetsize: opstats. characteristics of packet size for transmits.
+ * <li> control/success: counter of success transmit of control records
+ * <li> seg_writer/write: opstats. latency characteristics of write operations in segment writer.
+ * <li> seg_writer/add_complete/{callback,queued,deferred}: opstats. latency components of add completions.
+ * <li> seg_writer/pendings: counter. the number of records pending by the segment writers.
+ * <li> transmit/outstanding/requests: per stream gauge. the number of outstanding transmits each stream.
+ * </ul>
+ */
+class BKLogSegmentWriter implements LogSegmentWriter, AddCallback, Runnable, Sizable {
+    static final Logger LOG = LoggerFactory.getLogger(BKLogSegmentWriter.class);
+
+    private final String fullyQualifiedLogSegment;
+    private final String streamName;
+    private final int logSegmentMetadataVersion;
+    private BKTransmitPacket packetPrevious;
+    private Entry.Writer recordSetWriter;
+    private final AtomicInteger outstandingTransmits;
+    private final int transmissionThreshold;
+    protected final LogSegmentEntryWriter entryWriter;
+    private final CompressionCodec.Type compressionType;
+    private final ReentrantLock transmitLock = new ReentrantLock();
+    private final AtomicInteger transmitResult
+        = new AtomicInteger(BKException.Code.OK);
+    private final DistributedLock lock;
+    private final boolean isDurableWriteEnabled;
+    private DLSN lastDLSN = DLSN.InvalidDLSN;
+    private final long startTxId;
+    private long lastTxId = DistributedLogConstants.INVALID_TXID;
+    private long lastTxIdAcknowledged = DistributedLogConstants.INVALID_TXID;
+    private long outstandingBytes = 0;
+    private long numFlushesSinceRestart = 0;
+    private long numBytes = 0;
+    private long lastEntryId = Long.MIN_VALUE;
+    private long lastTransmitNanos = Long.MIN_VALUE;
+    private final int periodicKeepAliveMs;
+
+    // Indicates whether there are writes that have been successfully transmitted that would need
+    // a control record to be transmitted to make them visible to the readers by updating the last
+    // add confirmed
+    volatile private boolean controlFlushNeeded = false;
+    private boolean immediateFlushEnabled = false;
+    private int minDelayBetweenImmediateFlushMs = 0;
+    private Stopwatch lastTransmit;
+    private boolean streamEnded = false;
+    private final ScheduledFuture<?> periodicFlushSchedule;
+    private final ScheduledFuture<?> periodicKeepAliveSchedule;
+    final private AtomicReference<ScheduledFuture<?>> transmitSchedFutureRef = new AtomicReference<ScheduledFuture<?>>(null);
+    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 final boolean enableRecordCounts;
+    private int positionWithinLogSegment = 0;
+    private final long logSegmentSequenceNumber;
+    // Used only for values that *could* change (e.g. buffer size etc.)
+    private final DistributedLogConfiguration conf;
+    private final OrderedScheduler scheduler;
+
+    // stats
+    private final StatsLogger envelopeStatsLogger;
+    private final StatsLogger transmitOutstandingLogger;
+    private final Counter transmitDataSuccesses;
+    private final Counter transmitDataMisses;
+    private final Gauge<Number> transmitOutstandingGauge;
+    private final OpStatsLogger transmitDataPacketSize;
+    private final Counter transmitControlSuccesses;
+    private final Counter pFlushSuccesses;
+    private final Counter pFlushMisses;
+    private final OpStatsLogger writeTime;
+    private final OpStatsLogger addCompleteTime;
+    private final OpStatsLogger addCompleteQueuedTime;
+    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 AlertStatsLogger alertStatsLogger;
+    private final WriteLimiter writeLimiter;
+    private final FailureInjector writeDelayInjector;
+
+    /**
+     * Construct an edit log output stream which writes to a ledger.
+     */
+    protected BKLogSegmentWriter(String streamName,
+                                 String logSegmentName,
+                                 DistributedLogConfiguration conf,
+                                 int logSegmentMetadataVersion,
+                                 LogSegmentEntryWriter entryWriter,
+                                 DistributedLock lock, /** the lock needs to be acquired **/
+                                 long startTxId,
+                                 long logSegmentSequenceNumber,
+                                 OrderedScheduler scheduler,
+                                 StatsLogger statsLogger,
+                                 StatsLogger perLogStatsLogger,
+                                 AlertStatsLogger alertStatsLogger,
+                                 PermitLimiter globalWriteLimiter,
+                                 FeatureProvider featureProvider,
+                                 DynamicDistributedLogConfiguration dynConf)
+        throws IOException {
+        super();
+
+        // set up a write limiter
+        PermitLimiter streamWriteLimiter = null;
+        if (conf.getPerWriterOutstandingWriteLimit() < 0) {
+            streamWriteLimiter = PermitLimiter.NULL_PERMIT_LIMITER;
+        } else {
+            Feature disableWriteLimitFeature = featureProvider.getFeature(
+                CoreFeatureKeys.DISABLE_WRITE_LIMIT.name().toLowerCase());
+            streamWriteLimiter = new SimplePermitLimiter(
+                conf.getOutstandingWriteLimitDarkmode(),
+                conf.getPerWriterOutstandingWriteLimit(),
+                statsLogger.scope("streamWriteLimiter"),
+                false,
+                disableWriteLimitFeature);
+        }
+        this.writeLimiter = new WriteLimiter(streamName, streamWriteLimiter, globalWriteLimiter);
+        this.alertStatsLogger = alertStatsLogger;
+        this.envelopeStatsLogger = BroadCastStatsLogger.masterslave(statsLogger, perLogStatsLogger);
+
+        StatsLogger flushStatsLogger = statsLogger.scope("flush");
+        StatsLogger pFlushStatsLogger = flushStatsLogger.scope("periodic");
+        pFlushSuccesses = pFlushStatsLogger.getCounter("success");
+        pFlushMisses = pFlushStatsLogger.getCounter("miss");
+
+        // transmit
+        StatsLogger transmitDataStatsLogger = statsLogger.scope("data");
+        transmitDataSuccesses = transmitDataStatsLogger.getCounter("success");
+        transmitDataMisses = transmitDataStatsLogger.getCounter("miss");
+        StatsLogger transmitStatsLogger = statsLogger.scope("transmit");
+        transmitDataPacketSize =  transmitStatsLogger.getOpStatsLogger("packetsize");
+        StatsLogger transmitControlStatsLogger = statsLogger.scope("control");
+        transmitControlSuccesses = transmitControlStatsLogger.getCounter("success");
+        StatsLogger segWriterStatsLogger = statsLogger.scope("seg_writer");
+        writeTime = segWriterStatsLogger.getOpStatsLogger("write");
+        addCompleteTime = segWriterStatsLogger.scope("add_complete").getOpStatsLogger("callback");
+        addCompleteQueuedTime = segWriterStatsLogger.scope("add_complete").getOpStatsLogger("queued");
+        addCompleteDeferredTime = segWriterStatsLogger.scope("add_complete").getOpStatsLogger("deferred");
+        pendingWrites = segWriterStatsLogger.getCounter("pending");
+
+        // outstanding transmit requests
+        transmitOutstandingLogger = perLogStatsLogger.scope("transmit").scope("outstanding");
+        transmitOutstandingGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+            @Override
+            public Number getSample() {
+                return outstandingTransmits.get();
+            }
+        };
+        transmitOutstandingLogger.registerGauge("requests", transmitOutstandingGauge);
+
+        outstandingTransmits = new AtomicInteger(0);
+        this.fullyQualifiedLogSegment = streamName + ":" + logSegmentName;
+        this.streamName = streamName;
+        this.logSegmentMetadataVersion = logSegmentMetadataVersion;
+        this.entryWriter = entryWriter;
+        this.lock = lock;
+        this.lock.checkOwnershipAndReacquire();
+
+        final int configuredTransmissionThreshold = dynConf.getOutputBufferSize();
+        if (configuredTransmissionThreshold > MAX_LOGRECORDSET_SIZE) {
+            LOG.warn("Setting output buffer size {} greater than max transmission size {} for log segment {}",
+                new Object[] {configuredTransmissionThreshold, MAX_LOGRECORDSET_SIZE, fullyQualifiedLogSegment});
+            this.transmissionThreshold = MAX_LOGRECORDSET_SIZE;
+        } else {
+            this.transmissionThreshold = configuredTransmissionThreshold;
+        }
+        this.compressionType = CompressionUtils.stringToType(conf.getCompressionType());
+
+        this.logSegmentSequenceNumber = logSegmentSequenceNumber;
+        this.recordSetWriter = Entry.newEntry(
+                streamName,
+                Math.max(transmissionThreshold, 1024),
+                envelopeBeforeTransmit(),
+                compressionType,
+                envelopeStatsLogger);
+        this.packetPrevious = null;
+        this.startTxId = startTxId;
+        this.lastTxId = startTxId;
+        this.lastTxIdAcknowledged = startTxId;
+        this.enableRecordCounts = conf.getEnableRecordCounts();
+        this.immediateFlushEnabled = conf.getImmediateFlushEnabled();
+        this.isDurableWriteEnabled = dynConf.isDurableWriteEnabled();
+        this.scheduler = scheduler;
+
+        // Failure injection
+        if (conf.getEIInjectWriteDelay()) {
+            this.writeDelayInjector = new RandomDelayFailureInjector(dynConf);
+        } else {
+            this.writeDelayInjector = FailureInjector.NULL;
+        }
+
+        // If we are transmitting immediately (threshold == 0) and if immediate
+        // flush is enabled, we don't need the periodic flush task
+        final int configuredPeriodicFlushFrequency = dynConf.getPeriodicFlushFrequencyMilliSeconds();
+        if (!immediateFlushEnabled || (0 != this.transmissionThreshold)) {
+            int periodicFlushFrequency = configuredPeriodicFlushFrequency;
+            if (periodicFlushFrequency > 0 && scheduler != null) {
+                periodicFlushSchedule = scheduler.scheduleAtFixedRate(this,
+                        periodicFlushFrequency/2, periodicFlushFrequency/2, TimeUnit.MILLISECONDS);
+            } else {
+                periodicFlushSchedule = null;
+            }
+        } else {
+            // Min delay heuristic applies only when immediate flush is enabled
+            // and transmission threshold is zero
+            minDelayBetweenImmediateFlushMs = conf.getMinDelayBetweenImmediateFlushMs();
+            periodicFlushSchedule = null;
+        }
+        this.periodicKeepAliveMs = conf.getPeriodicKeepAliveMilliSeconds();
+        if (periodicKeepAliveMs > 0 && scheduler != null) {
+            periodicKeepAliveSchedule = scheduler.scheduleAtFixedRate(new Runnable() {
+                @Override
+                public void run() {
+                    keepAlive();
+                }
+            }, periodicKeepAliveMs, periodicKeepAliveMs, TimeUnit.MILLISECONDS);
+        } else {
+            periodicKeepAliveSchedule = null;
+        }
+
+        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();
+    }
+
+    String getFullyQualifiedLogSegment() {
+        return fullyQualifiedLogSegment;
+    }
+
+    @VisibleForTesting
+    DistributedLock getLock() {
+        return this.lock;
+    }
+
+    @VisibleForTesting
+    FuturePool getFuturePool() {
+        if (null == scheduler) {
+            return null;
+        }
+        return scheduler.getFuturePool(streamName);
+    }
+
+    @VisibleForTesting
+    void setTransmitResult(int rc) {
+        transmitResult.set(rc);
+    }
+
+    @VisibleForTesting
+    protected final LogSegmentEntryWriter getEntryWriter() {
+        return this.entryWriter;
+    }
+
+    @Override
+    public long getLogSegmentId() {
+        return this.entryWriter.getLogSegmentId();
+    }
+
+    protected final long getLogSegmentSequenceNumber() {
+        return logSegmentSequenceNumber;
+    }
+
+    /**
+     * Get the start tx id of the log segment.
+     *
+     * @return start tx id of the log segment.
+     */
+    protected final long getStartTxId() {
+        return startTxId;
+    }
+
+    /**
+     * Get the last tx id that has been written to the log segment buffer but not committed yet.
+     *
+     * @return last tx id that has been written to the log segment buffer but not committed yet.
+     * @see #getLastTxIdAcknowledged()
+     */
+    synchronized long getLastTxId() {
+        return lastTxId;
+    }
+
+    /**
+     * Get the last tx id that has been acknowledged.
+     *
+     * @return last tx id that has been acknowledged.
+     * @see #getLastTxId()
+     */
+    synchronized long getLastTxIdAcknowledged() {
+        return lastTxIdAcknowledged;
+    }
+
+    /**
+     * Get the position-within-logsemgnet of the last written log record.
+     *
+     * @return position-within-logsegment of the last written log record.
+     */
+    synchronized int getPositionWithinLogSegment() {
+        return positionWithinLogSegment;
+    }
+
+    @VisibleForTesting
+    long getLastEntryId() {
+        return lastEntryId;
+    }
+
+    /**
+     * Get the last dlsn of the last acknowledged record.
+     *
+     * @return last dlsn of the last acknowledged record.
+     */
+    synchronized DLSN getLastDLSN() {
+        return lastDLSN;
+    }
+
+    @Override
+    public long size() {
+        return entryWriter.size();
+    }
+
+    private synchronized int getAverageTransmitSize() {
+        if (numFlushesSinceRestart > 0) {
+            long ret = numBytes/numFlushesSinceRestart;
+
+            if (ret < Integer.MIN_VALUE || ret > Integer.MAX_VALUE) {
+                throw new IllegalArgumentException
+                    (ret + " transmit size should never exceed max transmit size");
+            }
+            return (int) ret;
+        }
+
+        return 0;
+    }
+
+    private Entry.Writer newRecordSetWriter() {
+        return Entry.newEntry(
+                streamName,
+                Math.max(transmissionThreshold, getAverageTransmitSize()),
+                envelopeBeforeTransmit(),
+                compressionType,
+                envelopeStatsLogger);
+    }
+
+    private boolean envelopeBeforeTransmit() {
+        return LogSegmentMetadata.supportsEnvelopedEntries(logSegmentMetadataVersion);
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        return closeInternal(false);
+    }
+
+    @Override
+    public Future<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) {
+            EntryBuffer recordSet = packet.getRecordSet();
+            numRecords = recordSet.getNumRecords();
+            int rc = transmitResult.get();
+            if (BKException.Code.OK == rc) {
+                rc = BKException.Code.InterruptedException;
+            }
+            Throwable reason = new WriteCancelledException(streamName, FutureUtils.transmitException(rc));
+            recordSet.abortTransmit(reason);
+        }
+        LOG.info("Stream {} aborted {} writes", fullyQualifiedLogSegment, numRecords);
+    }
+
+    private synchronized long getWritesPendingTransmit() {
+        if (null != recordSetWriter) {
+            return recordSetWriter.getNumRecords();
+        } else {
+            return 0;
+        }
+    }
+
+    private synchronized long getPendingAddCompleteCount() {
+        if (null != addCompleteFuturePool) {
+            return addCompleteFuturePool.size();
+        } else {
+            return 0;
+        }
+    }
+
+    private Future<Void> closeInternal(boolean abort) {
+        Promise<Void> closePromise;
+        synchronized (this) {
+            if (null != closeFuture) {
+                return closeFuture;
+            }
+            closePromise = closeFuture = new Promise<Void>();
+        }
+
+        AtomicReference<Throwable> throwExc = new AtomicReference<Throwable>(null);
+        closeInternal(abort, throwExc, closePromise);
+        return closePromise;
+    }
+
+    private void closeInternal(final boolean abort,
+                               final AtomicReference<Throwable> throwExc,
+                               final Promise<Void> closePromise) {
+        // clean stats resources
+        this.transmitOutstandingLogger.unregisterGauge("requests", transmitOutstandingGauge);
+        this.writeLimiter.close();
+
+        // Cancel the periodic keep alive schedule first
+        if (null != periodicKeepAliveSchedule) {
+            if (!periodicKeepAliveSchedule.cancel(false)) {
+                LOG.info("Periodic keepalive for log segment {} isn't cancelled.", getFullyQualifiedLogSegment());
+            }
+        }
+
+        // Cancel the periodic flush schedule first
+        // The task is allowed to exit gracefully
+        if (null != periodicFlushSchedule) {
+            // we don't need to care about the cancel result here. if the periodicl flush task couldn't
+            // be cancelled, it means that it is doing flushing. So following flushes would be synchronized
+            // to wait until background flush completed.
+            if (!periodicFlushSchedule.cancel(false)) {
+                LOG.info("Periodic flush for log segment {} isn't cancelled.", getFullyQualifiedLogSegment());
+            }
+        }
+
+        // If it is a normal close and the stream isn't in an error state, we attempt to flush any buffered data
+        if (!abort && !isLogSegmentInError()) {
+            this.enforceLock = false;
+            LOG.info("Flushing before closing log segment {}", getFullyQualifiedLogSegment());
+            flushAndCommit().addEventListener(new FutureEventListener<Long>() {
+                @Override
+                public void onSuccess(Long value) {
+                    abortTransmitPacketOnClose(abort, throwExc, closePromise);
+                }
+
+                @Override
+                public void onFailure(Throwable cause) {
+                    throwExc.set(cause);
+                    abortTransmitPacketOnClose(abort, throwExc, closePromise);
+                }
+            });
+        } else {
+            abortTransmitPacketOnClose(abort, throwExc, closePromise);
+        }
+
+    }
+
+    private void abortTransmitPacketOnClose(final boolean abort,
+                                            final AtomicReference<Throwable> throwExc,
+                                            final Promise<Void> closePromise) {
+        LOG.info("Closing BKPerStreamLogWriter (abort={}) for {} :" +
+                        " lastDLSN = {} outstandingTransmits = {} writesPendingTransmit = {} addCompletesPending = {}",
+                new Object[]{abort, fullyQualifiedLogSegment, getLastDLSN(),
+                        outstandingTransmits.get(), getWritesPendingTransmit(), getPendingAddCompleteCount()});
+
+        // Save the current packet to reset, leave a new empty packet to avoid a race with
+        // addCompleteDeferredProcessing.
+        final BKTransmitPacket packetPreviousSaved;
+        final BKTransmitPacket packetCurrentSaved;
+        synchronized (this) {
+            packetPreviousSaved = packetPrevious;
+            packetCurrentSaved = new BKTransmitPacket(recordSetWriter);
+            recordSetWriter = newRecordSetWriter();
+        }
+
+        // Once the last packet been transmitted, apply any remaining promises asynchronously
+        // to avoid blocking close if bk client is slow for some reason.
+        if (null != packetPreviousSaved) {
+            packetPreviousSaved.addTransmitCompleteListener(new FutureEventListener<Integer>() {
+                @Override
+                public void onSuccess(Integer transmitResult) {
+                    flushAddCompletes();
+                    abortPacket(packetCurrentSaved);
+                }
+                @Override
+                public void onFailure(Throwable cause) {
+                    LOG.error("Unexpected error on transmit completion ", cause);
+                }
+            });
+        } else {
+            // In this case there are no pending add completes, but we still need to abort the
+            // current packet.
+            abortPacket(packetCurrentSaved);
+        }
+        closeLedgerOnClose(abort, throwExc, closePromise);
+    }
+
+    private void closeLedgerOnClose(final boolean abort,
+                                    final AtomicReference<Throwable> throwExc,
+                                    final Promise<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.
+            // we should throw the exception to #closeToFinalize, so it would fail completing a log segment.
+            entryWriter.asyncClose(new CloseCallback() {
+                @Override
+                public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
+                    if (BKException.Code.OK != rc && BKException.Code.LedgerClosedException != rc) {
+                        if (!abort) {
+                            throwExc.set(new IOException("Failed to close ledger for " + fullyQualifiedLogSegment + " : " +
+                                    BKException.getMessage(rc)));
+                        }
+                    }
+                    completeClosePromise(abort, throwExc, closePromise);
+                }
+            }, null);
+        } else {
+            completeClosePromise(abort, throwExc, closePromise);
+        }
+    }
+
+    private void completeClosePromise(final boolean abort,
+                                      final AtomicReference<Throwable> throwExc,
+                                      final Promise<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);
+        } else {
+            FutureUtils.setException(closePromise, throwExc.get());
+        }
+    }
+
+    @Override
+    synchronized public void write(LogRecord record) throws IOException {
+        writeUserRecord(record);
+        flushIfNeeded();
+    }
+
+    @Override
+    synchronized public Future<DLSN> asyncWrite(LogRecord record) {
+        return asyncWrite(record, true);
+    }
+
+    synchronized public Future<DLSN> asyncWrite(LogRecord record, boolean flush) {
+        Future<DLSN> result = null;
+        try {
+            if (record.isControl()) {
+                // we don't pack control records with user records together
+                // so transmit current output buffer if possible
+                try {
+                    transmit();
+                } catch (IOException ioe) {
+                    return Future.exception(new WriteCancelledException(fullyQualifiedLogSegment, ioe));
+                }
+                result = writeControlLogRecord(record);
+                transmit();
+            } else {
+                result = writeUserRecord(record);
+                if (!isDurableWriteEnabled) {
+                    // we have no idea about the DLSN if durability is turned off.
+                    result = Future.value(DLSN.InvalidDLSN);
+                }
+                if (flush) {
+                    flushIfNeeded();
+                }
+            }
+        } catch (IOException ioe) {
+            // We may incorrectly report transmit failure here, but only if we happened to hit
+            // packet/xmit size limit conditions AND fail flush above, which should happen rarely
+            if (null != result) {
+                LOG.error("Overriding first result with flush failure {}", result);
+            }
+            result = Future.exception(ioe);
+
+            // Flush to ensure any prev. writes with flush=false are flushed despite failure.
+            flushIfNeededNoThrow();
+        }
+        return result;
+    }
+
+    synchronized private Future<DLSN> writeUserRecord(LogRecord record) throws IOException {
+        if (null != closeFuture) {
+            throw new WriteException(fullyQualifiedLogSegment, BKException.getMessage(BKException.Code.LedgerClosedException));
+        }
+
+        if (BKException.Code.OK != transmitResult.get()) {
+            // Failfast if the stream already encountered error with safe retry on the client
+            throw new WriteException(fullyQualifiedLogSegment, BKException.getMessage(transmitResult.get()));
+        }
+
+        if (streamEnded) {
+            throw new EndOfStreamException("Writing to a stream after it has been marked as completed");
+        }
+
+        if ((record.getTransactionId() < 0) ||
+            (record.getTransactionId() == DistributedLogConstants.MAX_TXID)) {
+            throw new TransactionIdOutOfOrderException(record.getTransactionId());
+        }
+
+        // Inject write delay if configured to do so
+        writeDelayInjector.inject();
+
+        // Will check write rate limits and throw if exceeded.
+        writeLimiter.acquire();
+        pendingWrites.inc();
+
+        // The count represents the number of user records up to the
+        // current record
+        // Increment the record count only when writing a user log record
+        // 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;
+        try {
+            // increment the position for the record to write
+            // if the record is failed to write, it would be decremented.
+            positionWithinLogSegment++;
+            int numRecords = 1;
+            if (record.isRecordSet()) {
+                numRecords = LogRecordSet.numRecords(record);
+            }
+            future = writeInternal(record);
+            // after the record (record set) is written, the position should be
+            // moved for {numRecords}, but since we already moved the record by 1
+            // so advance the position for other {numRecords - 1}.
+            positionWithinLogSegment += (numRecords - 1);
+        } catch (IOException ex) {
+            writeLimiter.release();
+            pendingWrites.dec();
+            positionWithinLogSegment--;
+            throw ex;
+        }
+
+        // Track outstanding requests and return the future.
+        return future.ensure(new Function0<BoxedUnit>() {
+            public BoxedUnit apply() {
+                pendingWrites.dec();
+                writeLimiter.release();
+                return null;
+            }
+        });
+    }
+
+    boolean isLogSegmentInError() {
+        return (transmitResult.get() != BKException.Code.OK);
+    }
+
+    boolean shouldFailCompleteLogSegment() {
+        return (transmitResult.get() != BKException.Code.OK) &&
+                (transmitResult.get() != BKException.Code.LedgerClosedException);
+    }
+
+    synchronized public Future<DLSN> writeInternal(LogRecord record)
+            throws LogRecordTooLongException, LockingException, BKTransmitException,
+                   WriteException, InvalidEnvelopedEntryException {
+        int logRecordSize = record.getPersistentSize();
+
+        if (logRecordSize > MAX_LOGRECORD_SIZE) {
+            throw new LogRecordTooLongException(String.format(
+                    "Log Record of size %d written when only %d is allowed",
+                    logRecordSize, MAX_LOGRECORD_SIZE));
+        }
+
+        // If we will exceed the max number of bytes allowed per entry
+        // initiate a transmit before accepting the new log record
+        if ((recordSetWriter.getNumBytes() + logRecordSize) > MAX_LOGRECORDSET_SIZE) {
+            checkStateAndTransmit();
+        }
+
+        checkWriteLock();
+
+        if (enableRecordCounts) {
+            // Set the count here. The caller would appropriately increment it
+            // if this log record is to be counted
+            record.setPositionWithinLogSegment(positionWithinLogSegment);
+        }
+
+        Promise<DLSN> writePromise = new Promise<DLSN>();
+        writePromise.addEventListener(new OpStatsListener<DLSN>(writeTime));
+        recordSetWriter.writeRecord(record, writePromise);
+
+        if (record.getTransactionId() < lastTxId) {
+            LOG.info("Log Segment {} TxId decreased Last: {} Record: {}",
+                    new Object[] {fullyQualifiedLogSegment, lastTxId, record.getTransactionId()});
+        }
+        if (!record.isControl()) {
+            // only update last tx id for user records
+            lastTxId = record.getTransactionId();
+            outstandingBytes += (20 + record.getPayload().length);
+        }
+        return writePromise;
+    }
+
+    synchronized private Future<DLSN> writeControlLogRecord()
+            throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
+                   LockingException, LogRecordTooLongException {
+        LogRecord controlRec = new LogRecord(lastTxId, DistributedLogConstants.CONTROL_RECORD_CONTENT);
+        controlRec.setControl();
+        return writeControlLogRecord(controlRec);
+    }
+
+    synchronized private Future<DLSN> writeControlLogRecord(LogRecord record)
+            throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
+                   LockingException, LogRecordTooLongException {
+        return writeInternal(record);
+    }
+
+    /**
+     * We write a special log record that marks the end of the stream. Since this is the last
+     * log record in the stream, it is marked with MAX_TXID. MAX_TXID also has the useful
+     * side-effect of disallowing future startLogSegment calls through the MaxTxID check
+     *
+     * @throws IOException
+     */
+    synchronized private void writeEndOfStreamMarker() throws IOException {
+        LogRecord endOfStreamRec = new LogRecord(DistributedLogConstants.MAX_TXID, "endOfStream".getBytes(UTF_8));
+        endOfStreamRec.setEndOfStream();
+        writeInternal(endOfStreamRec);
+    }
+
+    /**
+     * Flushes all the data up to this point,
+     * adds the end of stream marker and marks the stream
+     * as read-only in the metadata. No appends to the
+     * stream will be allowed after this point
+     */
+    public Future<Long> markEndOfStream() {
+        synchronized (this) {
+            try {
+                writeEndOfStreamMarker();
+            } catch (IOException e) {
+                return Future.exception(e);
+            }
+            streamEnded = true;
+        }
+        return flushAndCommit();
+    }
+
+    /**
+     * Write bulk of records.
+     *
+     * (TODO: moved this method to log writer level)
+     *
+     * @param records list of records to write
+     * @return number of records that has been written
+     * @throws IOException when there is I/O errors during writing records.
+     */
+    synchronized public int writeBulk(List<LogRecord> records) throws IOException {
+        int numRecords = 0;
+        for (LogRecord r : records) {
+            write(r);
+            numRecords++;
+        }
+        return numRecords;
+    }
+
+    private void checkStateBeforeTransmit() throws WriteException {
+        try {
+            FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_TransmitBeforeAddEntry);
+        } catch (IOException e) {
+            throw new WriteException(streamName, "Fail transmit before adding entries");
+        }
+    }
+
+    /**
+     * Transmit the output buffer data to the backend.
+     *
+     * @return last txn id that already acknowledged
+     * @throws BKTransmitException if the segment writer is already in error state
+     * @throws LockingException if the segment writer lost lock before transmit
+     * @throws WriteException if failed to create the envelope for the data to transmit
+     * @throws InvalidEnvelopedEntryException when built an invalid enveloped entry
+     */
+    synchronized void checkStateAndTransmit()
+            throws BKTransmitException, WriteException, InvalidEnvelopedEntryException, LockingException {
+        checkStateBeforeTransmit();
+        transmit();
+    }
+
+    @Override
+    public synchronized Future<Long> flush() {
+        try {
+            checkStateBeforeTransmit();
+        } catch (WriteException e) {
+            return Future.exception(e);
+        }
+
+        Future<Integer> transmitFuture;
+        try {
+            transmitFuture = transmit();
+        } catch (BKTransmitException e) {
+            return Future.exception(e);
+        } catch (LockingException e) {
+            return Future.exception(e);
+        } catch (WriteException e) {
+            return Future.exception(e);
+        } catch (InvalidEnvelopedEntryException e) {
+            return Future.exception(e);
+        }
+
+        if (null == transmitFuture) {
+            if (null != packetPrevious) {
+                transmitFuture = packetPrevious.getTransmitFuture();
+            }  else {
+                return Future.value(getLastTxIdAcknowledged());
+            }
+        }
+
+        return transmitFuture.flatMap(GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC);
+    }
+
+    @Override
+    public synchronized Future<Long> commit() {
+        // we don't pack control records with user records together
+        // so transmit current output buffer if possible
+        Future<Integer> transmitFuture;
+        try {
+            try {
+                transmitFuture = transmit();
+            } catch (IOException ioe) {
+                return Future.exception(ioe);
+            }
+            if (null == transmitFuture) {
+                writeControlLogRecord();
+                return flush();
+            }
+        } catch (IOException ioe) {
+            return Future.exception(ioe);
+        }
+        return transmitFuture.flatMap(GET_LAST_TXID_ACKNOWLEDGED_AFTER_TRANSMIT_FUNC);
+    }
+
+    Future<Long> flushAndCommit() {
+        return flush().flatMap(COMMIT_AFTER_FLUSH_FUNC);
+    }
+
+    void flushIfNeededNoThrow() {
+        try {
+            flushIfNeeded();
+        } catch (IOException ioe) {
+            LOG.error("Encountered exception while flushing log records to stream {}",
+                fullyQualifiedLogSegment, ioe);
+        }
+    }
+
+    void scheduleFlushWithDelayIfNeeded(final Callable<?> callable,
+                                        final AtomicReference<ScheduledFuture<?>> scheduledFutureRef) {
+        final long delayMs = Math.max(0, minDelayBetweenImmediateFlushMs - lastTransmit.elapsed(TimeUnit.MILLISECONDS));
+        final ScheduledFuture<?> scheduledFuture = scheduledFutureRef.get();
+        if ((null == scheduledFuture) || scheduledFuture.isDone()) {
+            scheduledFutureRef.set(scheduler.schedule(new Runnable() {
+                @Override
+                public void run() {
+                    synchronized(this) {
+                        scheduledFutureRef.set(null);
+                        try {
+                            callable.call();
+
+                            // Flush was successful or wasn't needed, the exception should be unset.
+                            scheduledFlushException.set(null);
+                        } catch (Exception exc) {
+                            scheduledFlushException.set(exc);
+                            LOG.error("Delayed flush failed", exc);
+                        }
+                    }
+                }
+            }, delayMs, TimeUnit.MILLISECONDS));
+        }
+    }
+
+    // Based on transmit buffer size, immediate flush, etc., should we flush the current
+    // packet now.
+    void flushIfNeeded() throws BKTransmitException, WriteException, InvalidEnvelopedEntryException,
+            LockingException, FlushException {
+        if (outstandingBytes > transmissionThreshold) {
+            // If flush delay is disabled, flush immediately, else schedule appropriately.
+            if (0 == minDelayBetweenImmediateFlushMs) {
+                checkStateAndTransmit();
+            } else {
+                scheduleFlushWithDelayIfNeeded(new Callable<Void>() {
+                    @Override
+                    public Void call() throws Exception {
+                        checkStateAndTransmit();
+                        return null;
+                    }
+                }, transmitSchedFutureRef);
+
+                // Timing here is not very important--the last flush failed and we should
+                // indicate this to the caller. The next flush may succeed and unset the
+                // scheduledFlushException in which case the next write will succeed (if the caller
+                // hasn't already closed the writer).
+                if (scheduledFlushException.get() != null) {
+                    throw new FlushException("Last flush encountered an error while writing data to the backend",
+                        getLastTxId(), getLastTxIdAcknowledged(), scheduledFlushException.get());
+                }
+            }
+        }
+    }
+
+    private void checkWriteLock() throws LockingException {
+        try {
+            if (FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_WriteInternalLostLock)) {
+                throw new LockingException("/failpoint/lockpath", "failpoint is simulating a lost lock"
+                        + getFullyQualifiedLogSegment());
+            }
+        } catch (IOException e) {
+            throw new LockingException("/failpoint/lockpath", "failpoint is simulating a lost lock for "
+                    + getFullyQualifiedLogSegment());
+        }
+        if (enforceLock) {
+            lock.checkOwnershipAndReacquire();
+        }
+    }
+
+    /**
+     * Transmit the current buffer to bookkeeper.
+     * Synchronised at the class. #write() and #setReadyToFlush()
+     * are never called at the same time.
+     *
+     * NOTE: This method should only throw known exceptions so that we don't accidentally
+     *       add new code that throws in an inappropriate place.
+     *
+     * @return a transmit future for caller to wait for transmit result if we transmit successfully,
+     *         null if no data to transmit
+     * @throws BKTransmitException if the segment writer is already in error state
+     * @throws LockingException if the segment writer lost lock before transmit
+     * @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()
+        throws BKTransmitException, LockingException, WriteException, InvalidEnvelopedEntryException {
+        EntryBuffer recordSetToTransmit;
+        transmitLock.lock();
+        try {
+            synchronized (this) {
+                checkWriteLock();
+                // If transmitResult is anything other than BKException.Code.OK, it means that the
+                // stream has encountered an error and cannot be written to.
+                if (!transmitResult.compareAndSet(BKException.Code.OK,
+                                                  BKException.Code.OK)) {
+                    LOG.error("Log Segment {} Trying to write to an errored stream; Error is {}",
+                              fullyQualifiedLogSegment,
+                              BKException.getMessage(transmitResult.get()));
+                    throw new BKTransmitException("Trying to write to an errored stream;"
+                                                          + " Error code : (" + transmitResult.get()
+                                                          + ") " + BKException.getMessage(transmitResult.get()), transmitResult.get());
+                }
+
+                if (recordSetWriter.getNumRecords() == 0) {
+                    // Control flushes always have at least the control record to flush
+                    transmitDataMisses.inc();
+                    return null;
+                }
+
+                recordSetToTransmit = recordSetWriter;
+                recordSetWriter = newRecordSetWriter();
+                outstandingBytes = 0;
+
+                if (recordSetToTransmit.hasUserRecords()) {
+                    numBytes += recordSetToTransmit.getNumBytes();
+                    numFlushesSinceRestart++;
+                }
+            }
+
+            Buffer toSend;
+            try {
+                toSend = recordSetToTransmit.getBuffer();
+                FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_TransmitFailGetBuffer);
+            } catch (IOException e) {
+                if (e instanceof InvalidEnvelopedEntryException) {
+                    alertStatsLogger.raise("Invalid enveloped entry for segment {} : ", fullyQualifiedLogSegment, e);
+                }
+                LOG.error("Exception while enveloping entries for segment: {}",
+                          new Object[] {fullyQualifiedLogSegment}, e);
+                // If a write fails here, we need to set the transmit result to an error so that
+                // no future writes go through and violate ordering guarantees.
+                transmitResult.set(BKException.Code.WriteException);
+                if (e instanceof InvalidEnvelopedEntryException) {
+                    alertStatsLogger.raise("Invalid enveloped entry for segment {} : ", fullyQualifiedLogSegment, e);
+                    throw (InvalidEnvelopedEntryException) e;
+                } else {
+                    throw new WriteException(streamName, "Envelope Error");
+                }
+            }
+
+            synchronized (this) {
+                // update the transmit timestamp
+                lastTransmitNanos = MathUtils.nowInNano();
+
+                BKTransmitPacket packet = new BKTransmitPacket(recordSetToTransmit);
+                packetPrevious = packet;
+                entryWriter.asyncAddEntry(toSend.getData(), 0, toSend.size(),
+                                          this, packet);
+
+                if (recordSetToTransmit.hasUserRecords()) {
+                    transmitDataSuccesses.inc();
+                } else {
+                    transmitControlSuccesses.inc();
+                }
+
+                lastTransmit.reset().start();
+                outstandingTransmits.incrementAndGet();
+                controlFlushNeeded = false;
+                return packet.getTransmitFuture();
+            }
+        } finally {
+            transmitLock.unlock();
+        }
+    }
+
+    /**
+     *  Checks if there is any data to transmit so that the periodic flush
+     *  task can determine if there is anything it needs to do
+     */
+    synchronized private boolean haveDataToTransmit() {
+        if (!transmitResult.compareAndSet(BKException.Code.OK, BKException.Code.OK)) {
+            // Even if there is data it cannot be transmitted, so effectively nothing to send
+            return false;
+        }
+
+        return (recordSetWriter.getNumRecords() > 0);
+    }
+
+    @Override
+    public void addComplete(final int rc, LedgerHandle handle,
+                            final long entryId, final Object ctx) {
+        final AtomicReference<Integer> effectiveRC = new AtomicReference<Integer>(rc);
+        try {
+            if (FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_TransmitComplete)) {
+                effectiveRC.set(BKException.Code.UnexpectedConditionException);
+            }
+        } catch (Exception exc) {
+            effectiveRC.set(BKException.Code.UnexpectedConditionException);
+        }
+
+        // Sanity check to make sure we're receiving these callbacks in order.
+        if (entryId > -1 && lastEntryId >= entryId) {
+            LOG.error("Log segment {} saw out of order entry {} lastEntryId {}",
+                new Object[] {fullyQualifiedLogSegment, entryId, lastEntryId});
+        }
+        lastEntryId = entryId;
+
+        assert (ctx instanceof BKTransmitPacket);
+        final BKTransmitPacket transmitPacket = (BKTransmitPacket) ctx;
+
+        // Time from transmit until receipt of addComplete callback
+        addCompleteTime.registerSuccessfulEvent(TimeUnit.MICROSECONDS.convert(
+            System.nanoTime() - transmitPacket.getTransmitTime(), TimeUnit.NANOSECONDS));
+
+        if (BKException.Code.OK == rc) {
+            EntryBuffer recordSet = transmitPacket.getRecordSet();
+            if (recordSet.hasUserRecords()) {
+                synchronized (this) {
+                    lastTxIdAcknowledged = Math.max(lastTxIdAcknowledged, recordSet.getMaxTxId());
+                }
+            }
+        }
+
+        if (null != addCompleteFuturePool) {
+            final Stopwatch queuedTime = Stopwatch.createStarted();
+            addCompleteFuturePool.apply(new Function0<Void>() {
+                public Void apply() {
+                    final Stopwatch deferredTime = Stopwatch.createStarted();
+                    addCompleteQueuedTime.registerSuccessfulEvent(queuedTime.elapsed(TimeUnit.MICROSECONDS));
+                    addCompleteDeferredProcessing(transmitPacket, entryId, effectiveRC.get());
+                    addCompleteDeferredTime.registerSuccessfulEvent(deferredTime.elapsed(TimeUnit.MICROSECONDS));
+                    return null;
+                }
+                @Override
+                public String toString() {
+                    return String.format("AddComplete(Stream=%s, entryId=%d, rc=%d)",
+                            fullyQualifiedLogSegment, entryId, rc);
+                }
+            }).addEventListener(new FutureEventListener<Void>() {
+                @Override
+                public void onSuccess(Void done) {
+                }
+                @Override
+                public void onFailure(Throwable cause) {
+                    LOG.error("addComplete processing failed for {} entry {} lastTxId {} rc {} with error",
+                        new Object[] {fullyQualifiedLogSegment, entryId, transmitPacket.getRecordSet().getMaxTxId(), rc, cause});
+                }
+            });
+            // Race condition if we notify before the addComplete is enqueued.
+            transmitPacket.notifyTransmitComplete(effectiveRC.get());
+            outstandingTransmits.getAndDecrement();
+        } else {
+            // Notify transmit complete must be called before deferred processing in the
+            // sync case since otherwise callbacks in deferred processing may deadlock.
+            transmitPacket.notifyTransmitComplete(effectiveRC.get());
+            outstandingTransmits.getAndDecrement();
+            addCompleteDeferredProcessing(transmitPacket, entryId, effectiveRC.get());
+        }
+    }
+
+    private void addCompleteDeferredProcessing(final BKTransmitPacket transmitPacket,
+                                               final long entryId,
+                                               final int rc) {
+        boolean cancelPendingPromises = false;
+        EntryBuffer recordSet = transmitPacket.getRecordSet();
+        synchronized (this) {
+            if (transmitResult.compareAndSet(BKException.Code.OK, rc)) {
+                // If this is the first time we are setting an error code in the transmitResult then
+                // we must cancel pending promises; once this error has been set, more records will not
+                // be enqueued; they will be failed with WriteException
+                cancelPendingPromises = (BKException.Code.OK != rc);
+            } else {
+                LOG.warn("Log segment {} entryId {}: Tried to set transmit result to ({}) but is already ({})",
+                    new Object[] {fullyQualifiedLogSegment, entryId, rc, transmitResult.get()});
+            }
+
+            if (transmitResult.get() != BKException.Code.OK) {
+                if (recordSet.hasUserRecords()) {
+                    transmitDataPacketSize.registerFailedEvent(recordSet.getNumBytes());
+                }
+            } else {
+                // If we had data that we flushed then we need it to make sure that
+                // background flush in the next pass will make the previous writes
+                // visible by advancing the lastAck
+                if (recordSet.hasUserRecords()) {
+                    transmitDataPacketSize.registerSuccessfulEvent(recordSet.getNumBytes());
+                    controlFlushNeeded = true;
+                    if (immediateFlushEnabled) {
+                        if (0 == minDelayBetweenImmediateFlushMs) {
+                            backgroundFlush(true);
+                        } else {
+                            scheduleFlushWithDelayIfNeeded(new Callable<Void>() {
+                                @Override
+                                public Void call() throws Exception {
+                                    backgroundFlush(true);
+                                    return null;
+                                }
+                            }, immFlushSchedFutureRef);
+                        }
+                    }
+                }
+            }
+
+            // update last dlsn before satisifying future
+            if (BKException.Code.OK == transmitResult.get()) {
+                DLSN lastDLSNInPacket = recordSet.finalizeTransmit(
+                        logSegmentSequenceNumber, entryId);
+                if (recordSet.hasUserRecords()) {
+                    if (null != lastDLSNInPacket && lastDLSN.compareTo(lastDLSNInPacket) < 0) {
+                        lastDLSN = lastDLSNInPacket;
+                    }
+                }
+            }
+        }
+
+        if (BKException.Code.OK == transmitResult.get()) {
+            recordSet.completeTransmit(logSegmentSequenceNumber, entryId);
+        } else {
+            recordSet.abortTransmit(FutureUtils.transmitException(transmitResult.get()));
+        }
+
+        if (cancelPendingPromises) {
+            // Since the writer is in a bad state no more packets will be tramsitted, and its safe to
+            // assign a new empty packet. This is to avoid a race with closeInternal which may also
+            // try to cancel the current packet;
+            final BKTransmitPacket packetCurrentSaved;
+            synchronized (this) {
+                packetCurrentSaved = new BKTransmitPacket(recordSetWriter);
+                recordSetWriter = newRecordSetWriter();
+            }
+            packetCurrentSaved.getRecordSet().abortTransmit(
+                    new WriteCancelledException(streamName,
+                            FutureUtils.transmitException(transmitResult.get())));
+        }
+    }
+
+    @Override
+    synchronized public void run()  {
+        backgroundFlush(false);
+    }
+
+    synchronized private void backgroundFlush(boolean controlFlushOnly)  {
+        if (null != closeFuture) {
+            // if the log segment is closing, skip any background flushing
+            LOG.debug("Skip background flushing since log segment {} is closing.", getFullyQualifiedLogSegment());
+            return;
+        }
+        try {
+            boolean newData = haveDataToTransmit();
+
+            if (controlFlushNeeded || (!controlFlushOnly && newData)) {
+                // If we need this periodic transmit to persist previously written data but
+                // there is no new data (which would cause the transmit to be skipped) generate
+                // a control record
+                if (!newData) {
+                    writeControlLogRecord();
+                }
+
+                transmit();
+                pFlushSuccesses.inc();
+            } else {
+                pFlushMisses.inc();
+            }
+        } catch (IOException exc) {
+            LOG.error("Log Segment {}: Error encountered by the periodic flush", fullyQualifiedLogSegment, exc);
+        }
+    }
+
+    synchronized private void keepAlive() {
+        if (null != closeFuture) {
+            // if the log segment is closing, skip sending any keep alive records.
+            LOG.debug("Skip sending keepAlive control record since log segment {} is closing.",
+                    getFullyQualifiedLogSegment());
+            return;
+        }
+
+        if (MathUtils.elapsedMSec(lastTransmitNanos) < periodicKeepAliveMs) {
+            return;
+        }
+
+        LogRecord controlRec = new LogRecord(lastTxId, DistributedLogConstants.KEEPALIVE_RECORD_CONTENT);
+        controlRec.setControl();
+        asyncWrite(controlRec);
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
new file mode 100644
index 0000000..fdb29f3
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
@@ -0,0 +1,1325 @@
+/**
+ * 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 com.google.common.base.Stopwatch;
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
+import org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.function.GetLastTxIdFunction;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
+import org.apache.distributedlog.metadata.LogMetadataForWriter;
+import org.apache.distributedlog.lock.DistributedLock;
+import org.apache.distributedlog.logsegment.LogSegmentFilter;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataCache;
+import org.apache.distributedlog.logsegment.RollingPolicy;
+import org.apache.distributedlog.logsegment.SizeBasedRollingPolicy;
+import org.apache.distributedlog.logsegment.TimeBasedRollingPolicy;
+import org.apache.distributedlog.metadata.LogStreamMetadataStore;
+import org.apache.distributedlog.metadata.MetadataUpdater;
+import org.apache.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
+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.util.OrderedScheduler;
+import org.apache.distributedlog.util.Transaction;
+import org.apache.distributedlog.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.
+ *
+ * <h3>Metrics</h3>
+ * All the metrics about log write handler are exposed under scope `segments`.
+ * <ul>
+ * <li> `segments`/open : opstats. latency characteristics on starting a new log segment.
+ * <li> `segments`/close : opstats. latency characteristics on completing an inprogress log segment.
+ * <li> `segments`/recover : opstats. latency characteristics on recovering a log segment.
+ * <li> `segments`/delete : opstats. latency characteristics on deleting a log segment.
+ * </ul>
+ */
+class BKLogWriteHandler extends BKLogHandler {
+    static final Logger LOG = LoggerFactory.getLogger(BKLogReadHandler.class);
+
+    private static Transaction.OpListener<LogSegmentEntryWriter> NULL_OP_LISTENER =
+            new Transaction.OpListener<LogSegmentEntryWriter>() {
+        @Override
+        public void onCommit(LogSegmentEntryWriter r) {
+            // no-op
+        }
+
+        @Override
+        public void onAbort(Throwable t) {
+            // no-op
+        }
+    };
+
+    protected final LogMetadataForWriter logMetadataForWriter;
+    protected final Allocator<LogSegmentEntryWriter, Object> logSegmentAllocator;
+    protected final DistributedLock lock;
+    protected final MaxTxId maxTxId;
+    protected final MaxLogSegmentSequenceNo maxLogSegmentSequenceNo;
+    protected final boolean validateLogSegmentSequenceNumber;
+    protected final int regionId;
+    protected final RollingPolicy rollingPolicy;
+    protected Future<? extends DistributedLock> lockFuture = null;
+    protected final PermitLimiter writeLimiter;
+    protected final FeatureProvider featureProvider;
+    protected final DynamicDistributedLogConfiguration dynConf;
+    protected final MetadataUpdater metadataUpdater;
+    // tracking the inprogress log segments
+    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;
+
+    // Recover Functions
+    private final RecoverLogSegmentFunction recoverLogSegmentFunction =
+            new RecoverLogSegmentFunction();
+    private final AbstractFunction1<List<LogSegmentMetadata>, Future<Long>> recoverLogSegmentsFunction =
+            new AbstractFunction1<List<LogSegmentMetadata>, Future<Long>>() {
+                @Override
+                public Future<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) {
+                        if (lastLedgerRollingTimeMillis < 0) {
+                            lastLedgerRollingTimeMillis = Utils.nowInMillis();
+                        }
+                    }
+
+                    if (validateLogSegmentSequenceNumber) {
+                        synchronized (inprogressLSSNs) {
+                            for (LogSegmentMetadata segment : segmentList) {
+                                if (segment.isInProgress()) {
+                                    inprogressLSSNs.addLast(segment.getLogSegmentSequenceNumber());
+                                }
+                            }
+                        }
+                    }
+
+                    return FutureUtils.processList(segmentList, recoverLogSegmentFunction, scheduler).map(
+                            GetLastTxIdFunction.INSTANCE);
+                }
+            };
+
+    // Stats
+    private final StatsLogger perLogStatsLogger;
+    private final OpStatsLogger closeOpStats;
+    private final OpStatsLogger openOpStats;
+    private final OpStatsLogger recoverOpStats;
+    private final OpStatsLogger deleteOpStats;
+
+    /**
+     * Construct a Bookkeeper journal manager.
+     */
+    BKLogWriteHandler(LogMetadataForWriter logMetadata,
+                      DistributedLogConfiguration conf,
+                      LogStreamMetadataStore streamMetadataStore,
+                      LogSegmentMetadataCache metadataCache,
+                      LogSegmentEntryStore entryStore,
+                      OrderedScheduler scheduler,
+                      Allocator<LogSegmentEntryWriter, Object> segmentAllocator,
+                      StatsLogger statsLogger,
+                      StatsLogger perLogStatsLogger,
+                      AlertStatsLogger alertStatsLogger,
+                      String clientId,
+                      int regionId,
+                      PermitLimiter writeLimiter,
+                      FeatureProvider featureProvider,
+                      DynamicDistributedLogConfiguration dynConf,
+                      DistributedLock lock /** owned by handler **/) {
+        super(logMetadata,
+                conf,
+                streamMetadataStore,
+                metadataCache,
+                entryStore,
+                scheduler,
+                statsLogger,
+                alertStatsLogger,
+                clientId);
+        this.logMetadataForWriter = logMetadata;
+        this.logSegmentAllocator = segmentAllocator;
+        this.perLogStatsLogger = perLogStatsLogger;
+        this.writeLimiter = writeLimiter;
+        this.featureProvider = featureProvider;
+        this.dynConf = dynConf;
+        this.lock = lock;
+        this.metadataUpdater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
+
+        if (conf.getEncodeRegionIDInLogSegmentMetadata()) {
+            this.regionId = regionId;
+        } else {
+            this.regionId = DistributedLogConstants.LOCAL_REGION_ID;
+        }
+        this.validateLogSegmentSequenceNumber = conf.isLogSegmentSequenceNumberValidationEnabled();
+
+        // Construct the max sequence no
+        maxLogSegmentSequenceNo = new MaxLogSegmentSequenceNo(logMetadata.getMaxLSSNData());
+        inprogressLSSNs = new LinkedList<Long>();
+        // Construct the max txn id.
+        maxTxId = new MaxTxId(logMetadata.getMaxTxIdData());
+
+        // Schedule fetching log segment list in background before we access it.
+        // We don't need to watch the log segment list changes for writer, as it manages log segment list.
+        fetchForWrite = readLogSegmentsFromStore(
+                LogSegmentMetadata.COMPARATOR,
+                WRITE_HANDLE_FILTER,
+                null);
+
+        // Initialize other parameters.
+        setLastLedgerRollingTimeMillis(Utils.nowInMillis());
+
+        // Rolling Policy
+        if (conf.getLogSegmentRollingIntervalMinutes() > 0) {
+            rollingPolicy = new TimeBasedRollingPolicy(conf.getLogSegmentRollingIntervalMinutes() * 60 * 1000L);
+        } else {
+            rollingPolicy = new SizeBasedRollingPolicy(conf.getMaxLogSegmentBytes());
+        }
+
+        // Stats
+        StatsLogger segmentsStatsLogger = statsLogger.scope("segments");
+        openOpStats = segmentsStatsLogger.getOpStatsLogger("open");
+        closeOpStats = segmentsStatsLogger.getOpStatsLogger("close");
+        recoverOpStats = segmentsStatsLogger.getOpStatsLogger("recover");
+        deleteOpStats = segmentsStatsLogger.getOpStatsLogger("delete");
+    }
+
+    private Future<List<LogSegmentMetadata>> getCachedLogSegmentsAfterFirstFetch(
+            final Comparator<LogSegmentMetadata> comparator) {
+        final Promise<List<LogSegmentMetadata>> promise = new Promise<List<LogSegmentMetadata>>();
+        fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                FutureUtils.setException(promise, cause);
+            }
+
+            @Override
+            public void onSuccess(Versioned<List<LogSegmentMetadata>> result) {
+                try {
+                    FutureUtils.setValue(promise, getCachedLogSegments(comparator));
+                } catch (UnexpectedException e) {
+                    FutureUtils.setException(promise, e);
+                }
+            }
+        });
+        return promise;
+    }
+
+    private Future<List<LogSegmentMetadata>> getCachedLogSegmentsAfterFirstFullFetch(
+            final Comparator<LogSegmentMetadata> comparator) {
+        Future<Versioned<List<LogSegmentMetadata>>> result;
+        synchronized (this) {
+            if (null == fetchForTruncation) {
+                fetchForTruncation = readLogSegmentsFromStore(
+                        LogSegmentMetadata.COMPARATOR,
+                        LogSegmentFilter.DEFAULT_FILTER,
+                        null);
+            }
+            result = fetchForTruncation;
+        }
+
+        final Promise<List<LogSegmentMetadata>> promise = new Promise<List<LogSegmentMetadata>>();
+        result.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                FutureUtils.setException(promise, cause);
+            }
+
+            @Override
+            public void onSuccess(Versioned<List<LogSegmentMetadata>> result) {
+                try {
+                    FutureUtils.setValue(promise, getCachedLogSegments(comparator));
+                } catch (UnexpectedException e) {
+                    FutureUtils.setException(promise, e);
+                }
+            }
+        });
+        return promise;
+    }
+
+    // Transactional operations for MaxLogSegmentSequenceNo
+    void storeMaxSequenceNumber(final Transaction<Object> txn,
+                                final MaxLogSegmentSequenceNo maxSeqNo,
+                                final long seqNo,
+                                final boolean isInprogress) {
+        metadataStore.storeMaxLogSegmentSequenceNumber(txn, logMetadata, maxSeqNo.getVersionedData(seqNo),
+                new Transaction.OpListener<Version>() {
+            @Override
+            public void onCommit(Version version) {
+                if (validateLogSegmentSequenceNumber) {
+                    synchronized (inprogressLSSNs) {
+                        if (isInprogress) {
+                            inprogressLSSNs.add(seqNo);
+                        } else {
+                            inprogressLSSNs.removeFirst();
+                        }
+                    }
+                }
+                maxSeqNo.update(version, seqNo);
+            }
+
+            @Override
+            public void onAbort(Throwable t) {
+                // no-op
+            }
+        });
+    }
+
+    // Transactional operations for MaxTxId
+    void storeMaxTxId(final Transaction<Object> txn,
+                      final MaxTxId maxTxId,
+                      final long txId) {
+        metadataStore.storeMaxTxnId(txn, logMetadataForWriter, maxTxId.getVersionedData(txId),
+                new Transaction.OpListener<Version>() {
+                    @Override
+                    public void onCommit(Version version) {
+                                                        maxTxId.update(version, txId);
+                                                                                      }
+
+                    @Override
+                    public void onAbort(Throwable t) {
+                        // no-op
+                    }
+                });
+    }
+
+    // Transactional operations for logsegment
+    void writeLogSegment(final Transaction<Object> txn,
+                         final LogSegmentMetadata metadata) {
+        metadataStore.createLogSegment(txn, metadata, new Transaction.OpListener<Void>() {
+            @Override
+            public void onCommit(Void r) {
+                addLogSegmentToCache(metadata.getSegmentName(), metadata);
+            }
+
+            @Override
+            public void onAbort(Throwable t) {
+                // no-op
+            }
+        });
+    }
+
+    void deleteLogSegment(final Transaction<Object> txn,
+                          final LogSegmentMetadata metadata) {
+        metadataStore.deleteLogSegment(txn, metadata, new Transaction.OpListener<Void>() {
+            @Override
+            public void onCommit(Void r) {
+                removeLogSegmentFromCache(metadata.getSegmentName());
+            }
+
+            @Override
+            public void onAbort(Throwable t) {
+                // no-op
+            }
+        });
+    }
+
+    /**
+     * The caller could call this before any actions, which to hold the lock for
+     * the write handler of its whole lifecycle. The lock will only be released
+     * when closing the write handler.
+     *
+     * This method is useful to prevent releasing underlying zookeeper lock during
+     * recovering/completing log segments. Releasing underlying zookeeper lock means
+     * 1) increase latency when re-lock on starting new log segment. 2) increase the
+     * possibility of a stream being re-acquired by other instances.
+     *
+     * @return future represents the lock result
+     */
+    Future<? extends DistributedLock> lockHandler() {
+        if (null != lockFuture) {
+            return lockFuture;
+        }
+        lockFuture = lock.asyncAcquire();
+        return lockFuture;
+    }
+
+    Future<Void> unlockHandler() {
+        if (null != lockFuture) {
+            return lock.asyncClose();
+        } else {
+            return Future.Void();
+        }
+    }
+
+    /**
+     * Start a new log segment in a BookKeeper ledger.
+     * First ensure that we have the write lock for this journal.
+     * Then create a ledger and stream based on that ledger.
+     * The ledger id is written to the inprogress znode, so that in the
+     * case of a crash, a recovery process can find the ledger we were writing
+     * to when we crashed.
+     *
+     * @param txId First transaction id to be written to the stream
+     * @return
+     * @throws IOException
+     */
+    public BKLogSegmentWriter startLogSegment(long txId) throws IOException {
+        return startLogSegment(txId, false, false);
+    }
+
+    /**
+     * Start a new log segment in a BookKeeper ledger.
+     * First ensure that we have the write lock for this journal.
+     * Then create a ledger and stream based on that ledger.
+     * The ledger id is written to the inprogress znode, so that in the
+     * case of a crash, a recovery process can find the ledger we were writing
+     * to when we crashed.
+     *
+     * @param txId First transaction id to be written to the stream
+     * @param bestEffort
+     * @param allowMaxTxID
+     *          allow using max tx id to start log segment
+     * @return
+     * @throws IOException
+     */
+    public BKLogSegmentWriter startLogSegment(long txId, boolean bestEffort, boolean allowMaxTxID)
+            throws IOException {
+        Stopwatch stopwatch = Stopwatch.createStarted();
+        boolean success = false;
+        try {
+            BKLogSegmentWriter writer = doStartLogSegment(txId, bestEffort, allowMaxTxID);
+            success = true;
+            return writer;
+        } finally {
+            if (success) {
+                openOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            } else {
+                openOpStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            }
+        }
+    }
+
+    protected long assignLogSegmentSequenceNumber() throws IOException {
+        // For any active stream we will always make sure that there is at least one
+        // active ledger (except when the stream first starts out). Therefore when we
+        // see no ledger metadata for a stream, we assume that this is the first ledger
+        // in the stream
+        long logSegmentSeqNo = DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
+        boolean logSegmentsFound = false;
+
+        if (LogSegmentMetadata.supportsLogSegmentSequenceNo(conf.getDLLedgerMetadataLayoutVersion())) {
+            List<LogSegmentMetadata> ledgerListDesc = getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR);
+            Long nextLogSegmentSeqNo = DLUtils.nextLogSegmentSequenceNumber(ledgerListDesc);
+
+            if (null == nextLogSegmentSeqNo) {
+                logSegmentsFound = false;
+                // we don't find last assigned log segment sequence number
+                // then we start the log segment with configured FirstLogSegmentSequenceNumber.
+                logSegmentSeqNo = conf.getFirstLogSegmentSequenceNumber();
+            } else {
+                logSegmentsFound = true;
+                // latest log segment is assigned with a sequence number, start with next sequence number
+                logSegmentSeqNo = nextLogSegmentSeqNo;
+            }
+        }
+
+        // We only skip log segment sequence number validation only when no log segments found &
+        // the maximum log segment sequence number is "UNASSIGNED".
+        if (!logSegmentsFound &&
+            (DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO == maxLogSegmentSequenceNo.getSequenceNumber())) {
+            // no ledger seqno stored in /ledgers before
+            LOG.info("No max ledger sequence number found while creating log segment {} for {}.",
+                logSegmentSeqNo, getFullyQualifiedName());
+        } else if (maxLogSegmentSequenceNo.getSequenceNumber() + 1 != logSegmentSeqNo) {
+            LOG.warn("Unexpected max log segment sequence number {} for {} : list of cached segments = {}",
+                new Object[]{maxLogSegmentSequenceNo.getSequenceNumber(), getFullyQualifiedName(),
+                    getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR)});
+            // there is max log segment number recorded there and it isn't match. throw exception.
+            throw new DLIllegalStateException("Unexpected max log segment sequence number "
+                + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName()
+                + ", expected " + (logSegmentSeqNo - 1));
+        }
+
+        return logSegmentSeqNo;
+    }
+
+    protected BKLogSegmentWriter doStartLogSegment(long txId, boolean bestEffort, boolean allowMaxTxID) throws IOException {
+        return FutureUtils.result(asyncStartLogSegment(txId, bestEffort, allowMaxTxID));
+    }
+
+    protected Future<BKLogSegmentWriter> asyncStartLogSegment(final long txId,
+                                                              final boolean bestEffort,
+                                                              final boolean allowMaxTxID) {
+        final Promise<BKLogSegmentWriter> promise = new Promise<BKLogSegmentWriter>();
+        try {
+            lock.checkOwnershipAndReacquire();
+        } catch (LockingException e) {
+            FutureUtils.setException(promise, e);
+            return promise;
+        }
+        fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                FutureUtils.setException(promise, cause);
+            }
+
+            @Override
+            public void onSuccess(Versioned<List<LogSegmentMetadata>> list) {
+                doStartLogSegment(txId, bestEffort, allowMaxTxID, promise);
+            }
+        });
+        return promise;
+    }
+
+    protected void doStartLogSegment(final long txId,
+                                     final boolean bestEffort,
+                                     final boolean allowMaxTxID,
+                                     final Promise<BKLogSegmentWriter> promise) {
+        // validate the tx id
+        if ((txId < 0) ||
+                (!allowMaxTxID && (txId == DistributedLogConstants.MAX_TXID))) {
+            FutureUtils.setException(promise, new IOException("Invalid Transaction Id " + txId));
+            return;
+        }
+
+        long highestTxIdWritten = maxTxId.get();
+        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"));
+                return;
+            } else {
+                LOG.error("We've already seen TxId {} the max TXId is {}", txId, highestTxIdWritten);
+                FutureUtils.setException(promise, new TransactionIdOutOfOrderException(txId, highestTxIdWritten));
+                return;
+            }
+        }
+
+        try {
+            logSegmentAllocator.allocate();
+        } catch (IOException e) {
+            // failed to issue an allocation request
+            failStartLogSegment(promise, bestEffort, e);
+            return;
+        }
+
+        // start the transaction from zookeeper
+        final Transaction<Object> txn = streamMetadataStore.newTransaction();
+
+        // failpoint injected before creating ledger
+        try {
+            FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_StartLogSegmentBeforeLedgerCreate);
+        } catch (IOException ioe) {
+            failStartLogSegment(promise, bestEffort, ioe);
+            return;
+        }
+
+        logSegmentAllocator.tryObtain(txn, NULL_OP_LISTENER)
+                .addEventListener(new FutureEventListener<LogSegmentEntryWriter>() {
+
+            @Override
+            public void onSuccess(LogSegmentEntryWriter entryWriter) {
+                // try-obtain succeed
+                createInprogressLogSegment(
+                        txn,
+                        txId,
+                        entryWriter,
+                        bestEffort,
+                        promise);
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                failStartLogSegment(promise, bestEffort, cause);
+            }
+        });
+    }
+
+    private void failStartLogSegment(Promise<BKLogSegmentWriter> promise,
+                                     boolean bestEffort,
+                                     Throwable cause) {
+        if (bestEffort) {
+            FutureUtils.setValue(promise, null);
+        } else {
+            FutureUtils.setException(promise, cause);
+        }
+    }
+
+    // once the ledger handle is obtained from allocator, this function should guarantee
+    // either the transaction is executed or aborted. Otherwise, the ledger handle will
+    // just leak from the allocation pool - hence cause "No Ledger Allocator"
+    private void createInprogressLogSegment(Transaction<Object> txn,
+                                            final long txId,
+                                            final LogSegmentEntryWriter entryWriter,
+                                            boolean bestEffort,
+                                            final Promise<BKLogSegmentWriter> promise) {
+        final long logSegmentSeqNo;
+        try {
+            FailpointUtils.checkFailPoint(
+                    FailpointUtils.FailPointName.FP_StartLogSegmentOnAssignLogSegmentSequenceNumber);
+            logSegmentSeqNo = assignLogSegmentSequenceNumber();
+        } catch (IOException e) {
+            // abort the current prepared transaction
+            txn.abort(e);
+            failStartLogSegment(promise, bestEffort, e);
+            return;
+        }
+
+        final String inprogressZnodePath = inprogressZNode(
+                entryWriter.getLogSegmentId(), txId, logSegmentSeqNo);
+        final LogSegmentMetadata l =
+            new LogSegmentMetadata.LogSegmentMetadataBuilder(inprogressZnodePath,
+                conf.getDLLedgerMetadataLayoutVersion(), entryWriter.getLogSegmentId(), txId)
+                    .setLogSegmentSequenceNo(logSegmentSeqNo)
+                    .setRegionId(regionId)
+                    .setEnvelopeEntries(
+                            LogSegmentMetadata.supportsEnvelopedEntries(conf.getDLLedgerMetadataLayoutVersion()))
+                    .build();
+
+        // Create an inprogress segment
+        writeLogSegment(txn, l);
+
+        // Try storing max sequence number.
+        LOG.debug("Try storing max sequence number in startLogSegment {} : {}", inprogressZnodePath, logSegmentSeqNo);
+        storeMaxSequenceNumber(txn, maxLogSegmentSequenceNo, logSegmentSeqNo, true);
+
+        // Try storing max tx id.
+        LOG.debug("Try storing MaxTxId in startLogSegment  {} {}", inprogressZnodePath, txId);
+        storeMaxTxId(txn, maxTxId, txId);
+
+        txn.execute().addEventListener(FutureEventListenerRunnable.of(new FutureEventListener<Void>() {
+
+            @Override
+            public void onSuccess(Void value) {
+                try {
+                    FutureUtils.setValue(promise, new BKLogSegmentWriter(
+                            getFullyQualifiedName(),
+                            l.getSegmentName(),
+                            conf,
+                            conf.getDLLedgerMetadataLayoutVersion(),
+                            entryWriter,
+                            lock,
+                            txId,
+                            logSegmentSeqNo,
+                            scheduler,
+                            statsLogger,
+                            perLogStatsLogger,
+                            alertStatsLogger,
+                            writeLimiter,
+                            featureProvider,
+                            dynConf));
+                } catch (IOException ioe) {
+                    failStartLogSegment(promise, false, ioe);
+                }
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                failStartLogSegment(promise, false, cause);
+            }
+        }, scheduler));
+    }
+
+    boolean shouldStartNewSegment(BKLogSegmentWriter writer) {
+        return rollingPolicy.shouldRollover(writer, lastLedgerRollingTimeMillis);
+    }
+
+    /**
+     * Finalize a log segment. If the journal manager is currently
+     * writing to a ledger, ensure that this is the ledger of the log segment
+     * being finalized.
+     * <p/>
+     * Otherwise this is the recovery case. In the recovery case, ensure that
+     * 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>();
+        completeAndCloseLogSegment(writer, promise);
+        return promise;
+    }
+
+    private void completeAndCloseLogSegment(final BKLogSegmentWriter writer,
+                                            final Promise<LogSegmentMetadata> promise) {
+        writer.asyncClose().addEventListener(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,
+                            new IOException("LogSegmentWriter for " + writer.getFullyQualifiedLogSegment() + " is already in error."));
+                    return;
+                }
+                doCompleteAndCloseLogSegment(
+                        inprogressZNodeName(writer.getLogSegmentId(), writer.getStartTxId(), writer.getLogSegmentSequenceNumber()),
+                        writer.getLogSegmentSequenceNumber(),
+                        writer.getLogSegmentId(),
+                        writer.getStartTxId(),
+                        writer.getLastTxId(),
+                        writer.getPositionWithinLogSegment(),
+                        writer.getLastDLSN().getEntryId(),
+                        writer.getLastDLSN().getSlotId(),
+                        promise);
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                FutureUtils.setException(promise, cause);
+            }
+        });
+    }
+
+    @VisibleForTesting
+    LogSegmentMetadata completeAndCloseLogSegment(long logSegmentSeqNo,
+                                                  long logSegmentId,
+                                                  long firstTxId,
+                                                  long lastTxId,
+                                                  int recordCount)
+        throws IOException {
+        return completeAndCloseLogSegment(inprogressZNodeName(logSegmentId, firstTxId, logSegmentSeqNo), logSegmentSeqNo,
+            logSegmentId, firstTxId, lastTxId, recordCount, -1, -1);
+    }
+
+    /**
+     * Finalize a log segment. If the journal manager is currently
+     * writing to a ledger, ensure that this is the ledger of the log segment
+     * being finalized.
+     * <p/>
+     * Otherwise this is the recovery case. In the recovery case, ensure that
+     * the firstTxId of the ledger matches firstTxId for the segment we are
+     * trying to finalize.
+     */
+    LogSegmentMetadata completeAndCloseLogSegment(String inprogressZnodeName, long logSegmentSeqNo,
+                                                  long logSegmentId, long firstTxId, long lastTxId,
+                                                  int recordCount, long lastEntryId, long lastSlotId)
+            throws IOException {
+        Stopwatch stopwatch = Stopwatch.createStarted();
+        boolean success = false;
+        try {
+            LogSegmentMetadata completedLogSegment =
+                    doCompleteAndCloseLogSegment(inprogressZnodeName, logSegmentSeqNo,
+                            logSegmentId, firstTxId, lastTxId, recordCount,
+                            lastEntryId, lastSlotId);
+            success = true;
+            return completedLogSegment;
+        } finally {
+            if (success) {
+                closeOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            } else {
+                closeOpStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            }
+        }
+    }
+
+    protected long computeStartSequenceId(LogSegmentMetadata segment) throws IOException {
+        if (!segment.isInProgress()) {
+            return segment.getStartSequenceId();
+        }
+
+        long startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
+
+        // we only record sequence id when both write version and logsegment's version support sequence id
+        if (LogSegmentMetadata.supportsSequenceId(conf.getDLLedgerMetadataLayoutVersion())
+                && segment.supportsSequenceId()) {
+            List<LogSegmentMetadata> logSegmentDescList =
+                    getCachedLogSegments(LogSegmentMetadata.DESC_COMPARATOR);
+            startSequenceId = DLUtils.computeStartSequenceId(logSegmentDescList, segment);
+        }
+
+        return startSequenceId;
+    }
+
+    /**
+     * Close log segment
+     *
+     * @param inprogressZnodeName
+     * @param logSegmentSeqNo
+     * @param logSegmentId
+     * @param firstTxId
+     * @param lastTxId
+     * @param recordCount
+     * @param lastEntryId
+     * @param lastSlotId
+     * @throws IOException
+     */
+    protected LogSegmentMetadata doCompleteAndCloseLogSegment(
+            String inprogressZnodeName,
+            long logSegmentSeqNo,
+            long logSegmentId,
+            long firstTxId,
+            long lastTxId,
+            int recordCount,
+            long lastEntryId,
+            long lastSlotId) throws IOException {
+        Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
+        doCompleteAndCloseLogSegment(
+                inprogressZnodeName,
+                logSegmentSeqNo,
+                logSegmentId,
+                firstTxId,
+                lastTxId,
+                recordCount,
+                lastEntryId,
+                lastSlotId,
+                promise);
+        return FutureUtils.result(promise);
+    }
+
+    protected void doCompleteAndCloseLogSegment(final String inprogressZnodeName,
+                                                final long logSegmentSeqNo,
+                                                final long logSegmentId,
+                                                final long firstTxId,
+                                                final long lastTxId,
+                                                final int recordCount,
+                                                final long lastEntryId,
+                                                final long lastSlotId,
+                                                final Promise<LogSegmentMetadata> promise) {
+        fetchForWrite.addEventListener(new FutureEventListener<Versioned<List<LogSegmentMetadata>>>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                FutureUtils.setException(promise, cause);
+            }
+
+            @Override
+            public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
+                doCompleteAndCloseLogSegmentAfterLogSegmentListFetched(
+                        inprogressZnodeName,
+                        logSegmentSeqNo,
+                        logSegmentId,
+                        firstTxId,
+                        lastTxId,
+                        recordCount,
+                        lastEntryId,
+                        lastSlotId,
+                        promise);
+            }
+        });
+    }
+
+    private void doCompleteAndCloseLogSegmentAfterLogSegmentListFetched(
+            final String inprogressZnodeName,
+            long logSegmentSeqNo,
+            long logSegmentId,
+            long firstTxId,
+            long lastTxId,
+            int recordCount,
+            long lastEntryId,
+            long lastSlotId,
+            final Promise<LogSegmentMetadata> promise) {
+        try {
+            lock.checkOwnershipAndReacquire();
+        } catch (IOException ioe) {
+            FutureUtils.setException(promise, ioe);
+            return;
+        }
+
+        LOG.debug("Completing and Closing Log Segment {} {}", firstTxId, lastTxId);
+        LogSegmentMetadata inprogressLogSegment = readLogSegmentFromCache(inprogressZnodeName);
+
+        // validate log segment
+        if (inprogressLogSegment.getLogSegmentId() != logSegmentId) {
+            FutureUtils.setException(promise, new IOException(
+                "Active ledger has different ID to inprogress. "
+                    + inprogressLogSegment.getLogSegmentId() + " found, "
+                    + logSegmentId + " expected"));
+            return;
+        }
+        // validate the transaction id
+        if (inprogressLogSegment.getFirstTxId() != firstTxId) {
+            FutureUtils.setException(promise, new IOException("Transaction id not as expected, "
+                + inprogressLogSegment.getFirstTxId() + " found, " + firstTxId + " expected"));
+            return;
+        }
+        // validate the log sequence number
+        if (validateLogSegmentSequenceNumber) {
+            synchronized (inprogressLSSNs) {
+                if (inprogressLSSNs.isEmpty()) {
+                    FutureUtils.setException(promise, new UnexpectedException(
+                            "Didn't find matched inprogress log segments when completing inprogress "
+                                    + inprogressLogSegment));
+                    return;
+                }
+                long leastInprogressLSSN = inprogressLSSNs.getFirst();
+                // the log segment sequence number in metadata {@link inprogressLogSegment.getLogSegmentSequenceNumber()}
+                // should be same as the sequence number we are completing (logSegmentSeqNo)
+                // and
+                // 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(
+                            "Didn't find matched inprogress log segments when completing inprogress "
+                                    + inprogressLogSegment));
+                    return;
+                }
+            }
+        }
+
+        // store max sequence number.
+        long maxSeqNo= Math.max(logSegmentSeqNo, maxLogSegmentSequenceNo.getSequenceNumber());
+        if (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo ||
+                (maxLogSegmentSequenceNo.getSequenceNumber() == logSegmentSeqNo + 1)) {
+            // ignore the case that a new inprogress log segment is pre-allocated
+            // before completing current inprogress one
+            LOG.info("Try storing max sequence number {} in completing {}.",
+                    new Object[] { logSegmentSeqNo, inprogressLogSegment.getZkPath() });
+        } else {
+            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 "
+                        + maxLogSegmentSequenceNo.getSequenceNumber() + " for " + getFullyQualifiedName()
+                        + ", expected " + (logSegmentSeqNo - 1)));
+                return;
+            }
+        }
+
+        // Prepare the completion
+        final String pathForCompletedLedger = completedLedgerZNode(firstTxId, lastTxId, logSegmentSeqNo);
+        long startSequenceId;
+        try {
+            startSequenceId = computeStartSequenceId(inprogressLogSegment);
+        } catch (IOException ioe) {
+            FutureUtils.setException(promise, ioe);
+            return;
+        }
+        // write completed ledger znode
+        final LogSegmentMetadata completedLogSegment =
+                inprogressLogSegment.completeLogSegment(
+                        pathForCompletedLedger,
+                        lastTxId,
+                        recordCount,
+                        lastEntryId,
+                        lastSlotId,
+                        startSequenceId);
+        setLastLedgerRollingTimeMillis(completedLogSegment.getCompletionTime());
+
+        // prepare the transaction
+        Transaction<Object> txn = streamMetadataStore.newTransaction();
+
+        // create completed log segment
+        writeLogSegment(txn, completedLogSegment);
+        // delete inprogress log segment
+        deleteLogSegment(txn, inprogressLogSegment);
+        // store max sequence number
+        storeMaxSequenceNumber(txn, maxLogSegmentSequenceNo, maxSeqNo, false);
+        // update max txn id.
+        LOG.debug("Trying storing LastTxId in Finalize Path {} LastTxId {}", pathForCompletedLedger, lastTxId);
+        storeMaxTxId(txn, maxTxId, lastTxId);
+
+        txn.execute().addEventListener(FutureEventListenerRunnable.of(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);
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                FutureUtils.setException(promise, cause);
+            }
+        }, scheduler));
+    }
+
+    public Future<Long> recoverIncompleteLogSegments() {
+        try {
+            FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_RecoverIncompleteLogSegments);
+        } catch (IOException ioe) {
+            return Future.exception(ioe);
+        }
+        return getCachedLogSegmentsAfterFirstFetch(LogSegmentMetadata.COMPARATOR).flatMap(recoverLogSegmentsFunction);
+    }
+
+    class RecoverLogSegmentFunction extends Function<LogSegmentMetadata, Future<LogSegmentMetadata>> {
+
+        @Override
+        public Future<LogSegmentMetadata> apply(final LogSegmentMetadata l) {
+            if (!l.isInProgress()) {
+                return Future.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);
+                        }
+                    });
+        }
+
+        private Future<LogSegmentMetadata> completeLogSegment(LogSegmentMetadata l,
+                                                              LogRecordWithDLSN lastRecord) {
+            LOG.info("Recovered last record in log segment {} for {}.", l, getFullyQualifiedName());
+
+            long endTxId = DistributedLogConstants.EMPTY_LOGSEGMENT_TX_ID;
+            int recordCount = 0;
+            long lastEntryId = -1;
+            long lastSlotId = -1;
+
+            if (null != lastRecord) {
+                endTxId = lastRecord.getTransactionId();
+                recordCount = lastRecord.getLastPositionWithinLogSegment();
+                lastEntryId = lastRecord.getDlsn().getEntryId();
+                lastSlotId = lastRecord.getDlsn().getSlotId();
+            }
+
+            if (endTxId == DistributedLogConstants.INVALID_TXID) {
+                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,"
+                    + " 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>();
+            doCompleteAndCloseLogSegment(
+                    l.getZNodeName(),
+                    l.getLogSegmentSequenceNumber(),
+                    l.getLogSegmentId(),
+                    l.getFirstTxId(),
+                    endTxId,
+                    recordCount,
+                    lastEntryId,
+                    lastSlotId,
+                    promise);
+            return promise;
+        }
+
+    }
+
+    Future<List<LogSegmentMetadata>> setLogSegmentsOlderThanDLSNTruncated(final DLSN dlsn) {
+        if (DLSN.InvalidDLSN == dlsn) {
+            List<LogSegmentMetadata> emptyList = new ArrayList<LogSegmentMetadata>(0);
+            return Future.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);
+                    }
+                });
+    }
+
+    private Future<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()});
+        List<LogSegmentMetadata> truncateList = new ArrayList<LogSegmentMetadata>(logSegments.size());
+        LogSegmentMetadata partialTruncate = null;
+        LOG.info("{}: Truncating log segments older than {}", getFullyQualifiedName(), dlsn);
+        for (int i = 0; i < logSegments.size(); i++) {
+            LogSegmentMetadata l = logSegments.get(i);
+            if (!l.isInProgress()) {
+                if (l.getLastDLSN().compareTo(dlsn) < 0) {
+                    LOG.debug("{}: Truncating log segment {} ", getFullyQualifiedName(), l);
+                    truncateList.add(l);
+                } else if (l.getFirstDLSN().compareTo(dlsn) < 0) {
+                    // Can be satisfied by at most one segment
+                    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));
+                    }
+                    LOG.info("{}: Partially truncating log segment {} older than {}.", new Object[] {getFullyQualifiedName(), l, dlsn});
+                    partialTruncate = l;
+                } else {
+                    break;
+                }
+            } else {
+                break;
+            }
+        }
+        return setLogSegmentTruncationStatus(truncateList, partialTruncate, dlsn);
+    }
+
+    private int getNumCandidateLogSegmentsToPurge(List<LogSegmentMetadata> logSegments) {
+        if (logSegments.isEmpty()) {
+            return 0;
+        } else {
+            // we have to keep at least one completed log segment for sequence id
+            int numCandidateLogSegments = 0;
+            for (LogSegmentMetadata segment : logSegments) {
+                if (segment.isInProgress()) {
+                    break;
+                } else {
+                    ++numCandidateLogSegments;
+                }
+            }
+
+            return numCandidateLogSegments - 1;
+        }
+    }
+
+    Future<List<LogSegmentMetadata>> purgeLogSegmentsOlderThanTimestamp(final long minTimestampToKeep) {
+        if (minTimestampToKeep >= Utils.nowInMillis()) {
+            return Future.exception(new IllegalArgumentException(
+                    "Invalid timestamp " + minTimestampToKeep + " to purge logs for " + getFullyQualifiedName()));
+        }
+        return getCachedLogSegmentsAfterFirstFullFetch(LogSegmentMetadata.COMPARATOR).flatMap(
+                new Function<List<LogSegmentMetadata>, Future<List<LogSegmentMetadata>>>() {
+            @Override
+            public Future<List<LogSegmentMetadata>> apply(List<LogSegmentMetadata> logSegments) {
+                List<LogSegmentMetadata> purgeList = new ArrayList<LogSegmentMetadata>(logSegments.size());
+
+                int numCandidates = getNumCandidateLogSegmentsToPurge(logSegments);
+
+                for (int iterator = 0; iterator < numCandidates; iterator++) {
+                    LogSegmentMetadata l = logSegments.get(iterator);
+                    // When application explicitly truncates segments; timestamp based purge is
+                    // only used to cleanup log segments that have been marked for truncation
+                    if ((l.isTruncated() || !conf.getExplicitTruncationByApplication()) &&
+                        !l.isInProgress() && (l.getCompletionTime() < minTimestampToKeep)) {
+                        purgeList.add(l);
+                    } else {
+                        // stop truncating log segments if we find either an inprogress or a partially
+                        // truncated log segment
+                        break;
+                    }
+                }
+                LOG.info("Deleting log segments older than {} for {} : {}",
+                        new Object[] { minTimestampToKeep, getFullyQualifiedName(), purgeList });
+                return deleteLogSegments(purgeList);
+            }
+        });
+    }
+
+    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;
+
+                    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(
+            final List<LogSegmentMetadata> truncateList,
+            LogSegmentMetadata partialTruncate,
+            DLSN minActiveDLSN) {
+        final List<LogSegmentMetadata> listToTruncate = Lists.newArrayListWithCapacity(truncateList.size() + 1);
+        final List<LogSegmentMetadata> listAfterTruncated = Lists.newArrayListWithCapacity(truncateList.size() + 1);
+        Transaction<Object> updateTxn = metadataUpdater.transaction();
+        for(LogSegmentMetadata l : truncateList) {
+            if (!l.isTruncated()) {
+                LogSegmentMetadata newSegment = metadataUpdater.setLogSegmentTruncated(updateTxn, l);
+                listToTruncate.add(l);
+                listAfterTruncated.add(newSegment);
+            }
+        }
+
+        if (null != partialTruncate && (partialTruncate.isNonTruncated() ||
+                (partialTruncate.isPartiallyTruncated() && (partialTruncate.getMinActiveDLSN().compareTo(minActiveDLSN) < 0)))) {
+            LogSegmentMetadata newSegment = metadataUpdater.setLogSegmentPartiallyTruncated(
+                    updateTxn, partialTruncate, minActiveDLSN);
+            listToTruncate.add(partialTruncate);
+            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;
+            }
+        });
+    }
+
+    private Future<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);
+    }
+
+    private Future<LogSegmentMetadata> deleteLogSegment(
+            final LogSegmentMetadata ledgerMetadata) {
+        LOG.info("Deleting ledger {} for {}", ledgerMetadata, getFullyQualifiedName());
+        final Promise<LogSegmentMetadata> promise = new Promise<LogSegmentMetadata>();
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+        promise.addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+            @Override
+            public void onSuccess(LogSegmentMetadata segment) {
+                deleteOpStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                deleteOpStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            }
+        });
+        entryStore.deleteLogSegment(ledgerMetadata)
+                .addEventListener(new FutureEventListener<LogSegmentMetadata>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                FutureUtils.setException(promise, cause);
+            }
+
+            @Override
+            public void onSuccess(LogSegmentMetadata segment) {
+                deleteLogSegmentMetadata(segment, promise);
+            }
+        });
+        return promise;
+    }
+
+    private void deleteLogSegmentMetadata(final LogSegmentMetadata segmentMetadata,
+                                          final Promise<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);
+            }
+
+            @Override
+            public void onAbort(Throwable t) {
+                if (t instanceof LogSegmentNotFoundException) {
+                    // purge log segment
+                    removeLogSegmentFromCache(segmentMetadata.getZNodeName());
+                    promise.setValue(segmentMetadata);
+                    return;
+                } else {
+                    LOG.error("Couldn't purge {} for {}: with error {}",
+                            new Object[]{ segmentMetadata, getFullyQualifiedName(), t });
+                    promise.setException(t);
+                }
+            }
+        });
+        deleteTxn.execute();
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        return Utils.closeSequence(scheduler,
+                lock,
+                logSegmentAllocator);
+    }
+
+    @Override
+    public Future<Void> asyncAbort() {
+        return asyncClose();
+    }
+
+    String completedLedgerZNodeName(long firstTxId, long lastTxId, long logSegmentSeqNo) {
+        if (DistributedLogConstants.LOGSEGMENT_NAME_VERSION == conf.getLogSegmentNameVersion()) {
+            return String.format("%s_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX, logSegmentSeqNo);
+        } else {
+            return String.format("%s_%018d_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX,
+                    firstTxId, lastTxId);
+        }
+    }
+
+    /**
+     * Get the znode path for a finalize ledger
+     */
+    String completedLedgerZNode(long firstTxId, long lastTxId, long logSegmentSeqNo) {
+        return String.format("%s/%s", logMetadata.getLogSegmentsPath(),
+                completedLedgerZNodeName(firstTxId, lastTxId, logSegmentSeqNo));
+    }
+
+    /**
+     * Get the name of the inprogress znode.
+     *
+     * @return name of the inprogress znode.
+     */
+    String inprogressZNodeName(long logSegmentId, long firstTxId, long logSegmentSeqNo) {
+        if (DistributedLogConstants.LOGSEGMENT_NAME_VERSION == conf.getLogSegmentNameVersion()) {
+            // Lots of the problems are introduced due to different inprogress names with same ledger sequence number.
+            return String.format("%s_%018d", DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX, logSegmentSeqNo);
+        } else {
+            return DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX + "_" + Long.toString(firstTxId, 16);
+        }
+    }
+
+    /**
+     * Get the znode path for the inprogressZNode
+     */
+    String inprogressZNode(long logSegmentId, long firstTxId, long logSegmentSeqNo) {
+        return logMetadata.getLogSegmentsPath() + "/" + inprogressZNodeName(logSegmentId, firstTxId, logSegmentSeqNo);
+    }
+
+    String inprogressZNode(String inprogressZNodeName) {
+        return logMetadata.getLogSegmentsPath() + "/" + inprogressZNodeName;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
new file mode 100644
index 0000000..bf89823
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
@@ -0,0 +1,276 @@
+/**
+ * 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 com.google.common.base.Optional;
+import com.google.common.base.Ticker;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.IdleReaderException;
+import org.apache.distributedlog.util.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;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Synchronous Log Reader based on {@link AsyncLogReader}
+ */
+class BKSyncLogReader implements LogReader, AsyncNotification {
+
+    private final BKDistributedLogManager bkdlm;
+    private final BKLogReadHandler readHandler;
+    private final AtomicReference<IOException> readerException =
+            new AtomicReference<IOException>(null);
+    private final int maxReadAheadWaitTime;
+    private Promise<Void> closeFuture;
+    private final Optional<Long> startTransactionId;
+    private boolean positioned = false;
+    private Entry.Reader currentEntry = null;
+
+    // readahead reader
+    ReadAheadEntryReader readAheadReader = null;
+
+    // idle reader settings
+    private final boolean shouldCheckIdleReader;
+    private final int idleErrorThresholdMillis;
+
+    // Stats
+    private final Counter idleReaderError;
+
+    BKSyncLogReader(DistributedLogConfiguration conf,
+                    BKDistributedLogManager bkdlm,
+                    DLSN startDLSN,
+                    Optional<Long> startTransactionId,
+                    StatsLogger statsLogger) throws IOException {
+        this.bkdlm = bkdlm;
+        this.readHandler = bkdlm.createReadHandler(
+                Optional.<String>absent(),
+                this,
+                true);
+        this.maxReadAheadWaitTime = conf.getReadAheadWaitTime();
+        this.idleErrorThresholdMillis = conf.getReaderIdleErrorThresholdMillis();
+        this.shouldCheckIdleReader = idleErrorThresholdMillis > 0 && idleErrorThresholdMillis < Integer.MAX_VALUE;
+        this.startTransactionId = startTransactionId;
+
+        // start readahead
+        startReadAhead(startDLSN);
+        if (!startTransactionId.isPresent()) {
+            positioned = true;
+        }
+
+        // Stats
+        StatsLogger syncReaderStatsLogger = statsLogger.scope("sync_reader");
+        idleReaderError = syncReaderStatsLogger.getCounter("idle_reader_error");
+    }
+
+    private void startReadAhead(DLSN startDLSN) throws IOException {
+        readAheadReader = new ReadAheadEntryReader(
+                    bkdlm.getStreamName(),
+                    startDLSN,
+                    bkdlm.getConf(),
+                    readHandler,
+                    bkdlm.getReaderEntryStore(),
+                    bkdlm.getScheduler(),
+                    Ticker.systemTicker(),
+                    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;
+                    }
+                });
+    }
+
+    @VisibleForTesting
+    ReadAheadEntryReader getReadAheadReader() {
+        return readAheadReader;
+    }
+
+    @VisibleForTesting
+    BKLogReadHandler getReadHandler() {
+        return readHandler;
+    }
+
+    private Entry.Reader readNextEntry(boolean nonBlocking) throws IOException {
+        Entry.Reader entry = null;
+        if (nonBlocking) {
+            return readAheadReader.getNextReadAheadEntry(0L, TimeUnit.MILLISECONDS);
+        } else {
+            while (!readAheadReader.isReadAheadCaughtUp()
+                    && null == readerException.get()
+                    && null == entry) {
+                entry = readAheadReader.getNextReadAheadEntry(maxReadAheadWaitTime, TimeUnit.MILLISECONDS);
+            }
+            if (null != entry) {
+                return entry;
+            }
+            // reader is caught up
+            if (readAheadReader.isReadAheadCaughtUp()
+                    && null == readerException.get()) {
+                entry = readAheadReader.getNextReadAheadEntry(maxReadAheadWaitTime, TimeUnit.MILLISECONDS);
+            }
+            return entry;
+        }
+    }
+
+    private void markReaderAsIdle() throws IdleReaderException {
+        idleReaderError.inc();
+        IdleReaderException ire = new IdleReaderException("Sync reader on stream "
+                + readHandler.getFullyQualifiedName()
+                + " is idle for more than " + idleErrorThresholdMillis + " ms");
+        readerException.compareAndSet(null, ire);
+        throw ire;
+    }
+
+    @Override
+    public synchronized LogRecordWithDLSN readNext(boolean nonBlocking)
+            throws IOException {
+        if (null != readerException.get()) {
+            throw readerException.get();
+        }
+        LogRecordWithDLSN record = doReadNext(nonBlocking);
+        // no record is returned, check if the reader becomes idle
+        if (null == record && shouldCheckIdleReader) {
+            if (readAheadReader.getNumCachedEntries() <= 0 &&
+                    readAheadReader.isReaderIdle(idleErrorThresholdMillis, TimeUnit.MILLISECONDS)) {
+                markReaderAsIdle();
+            }
+        }
+        return record;
+    }
+
+    private LogRecordWithDLSN doReadNext(boolean nonBlocking) throws IOException {
+        LogRecordWithDLSN record = null;
+
+        do {
+            // fetch one record until we don't find any entry available in the readahead cache
+            while (null == record) {
+                if (null == currentEntry) {
+                    currentEntry = readNextEntry(nonBlocking);
+                    if (null == currentEntry) {
+                        return null;
+                    }
+                }
+                record = currentEntry.nextRecord();
+                if (null == record) {
+                    currentEntry = null;
+                }
+            }
+
+            // check if we reached the end of stream
+            if (record.isEndOfStream()) {
+                EndOfStreamException eos = new EndOfStreamException("End of Stream Reached for "
+                        + readHandler.getFullyQualifiedName());
+                readerException.compareAndSet(null, eos);
+                throw eos;
+            }
+            // skip control records
+            if (record.isControl()) {
+                record = null;
+                continue;
+            }
+            if (!positioned) {
+                if (record.getTransactionId() < startTransactionId.get()) {
+                    record = null;
+                    continue;
+                } else {
+                    positioned = true;
+                    break;
+                }
+            } else {
+                break;
+            }
+        } while (true);
+        return record;
+    }
+
+    @Override
+    public synchronized List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords)
+            throws IOException {
+        LinkedList<LogRecordWithDLSN> retList =
+                new LinkedList<LogRecordWithDLSN>();
+
+        int numRead = 0;
+        LogRecordWithDLSN record = readNext(nonBlocking);
+        while ((null != record)) {
+            retList.add(record);
+            numRead++;
+            if (numRead >= numLogRecords) {
+                break;
+            }
+            record = readNext(nonBlocking);
+        }
+        return retList;
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        Promise<Void> closePromise;
+        synchronized (this) {
+            if (null != closeFuture) {
+                return closeFuture;
+            }
+            closeFuture = closePromise = new Promise<Void>();
+        }
+        readHandler.unregisterListener(readAheadReader);
+        readAheadReader.removeStateChangeNotification(this);
+        Utils.closeSequence(bkdlm.getScheduler(), true,
+                readAheadReader,
+                readHandler
+        ).proxyTo(closePromise);
+        return closePromise;
+    }
+
+    @Override
+    public void close() throws IOException {
+        FutureUtils.result(asyncClose());
+    }
+
+    //
+    // Notification From ReadHandler
+    //
+
+    @Override
+    public void notifyOnError(Throwable cause) {
+        if (cause instanceof IOException) {
+            readerException.compareAndSet(null, (IOException) cause);
+        } else {
+            readerException.compareAndSet(null, new IOException(cause));
+        }
+    }
+
+    @Override
+    public void notifyOnOperationComplete() {
+        // no-op
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java
new file mode 100644
index 0000000..7d33d12
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java
@@ -0,0 +1,113 @@
+/**
+ * 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 org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.util.FutureUtils;
+
+import java.io.IOException;
+import java.util.List;
+
+class BKSyncLogWriter extends BKAbstractLogWriter implements LogWriter {
+
+    public BKSyncLogWriter(DistributedLogConfiguration conf,
+                           DynamicDistributedLogConfiguration dynConf,
+                           BKDistributedLogManager bkdlm) {
+        super(conf, dynConf, bkdlm);
+    }
+    /**
+     * Write log records to the stream.
+     *
+     * @param record operation
+     */
+    @Override
+    public void write(LogRecord record) throws IOException {
+        getLedgerWriter(record.getTransactionId(), false).write(record);
+    }
+
+    /**
+     * Write edits logs operation to the stream.
+     *
+     * @param records list of records
+     */
+    @Override
+    @Deprecated
+    public int writeBulk(List<LogRecord> records) throws IOException {
+        return getLedgerWriter(records.get(0).getTransactionId(), false).writeBulk(records);
+    }
+
+    /**
+     * Flushes all the data up to this point,
+     * adds the end of stream marker and marks the stream
+     * as read-only in the metadata. No appends to the
+     * stream will be allowed after this point
+     */
+    @Override
+    public void markEndOfStream() throws IOException {
+        FutureUtils.result(getLedgerWriter(DistributedLogConstants.MAX_TXID, true).markEndOfStream());
+        closeAndComplete();
+    }
+
+    /**
+     * All data that has been written to the stream so far will be flushed.
+     * New data can be still written to the stream while flush is ongoing.
+     */
+    @Override
+    public long setReadyToFlush() throws IOException {
+        checkClosedOrInError("setReadyToFlush");
+        long highestTransactionId = 0;
+        BKLogSegmentWriter writer = getCachedLogWriter();
+        if (null != writer) {
+            highestTransactionId = Math.max(highestTransactionId, FutureUtils.result(writer.flush()));
+        }
+        return highestTransactionId;
+    }
+
+    /**
+     * Commit data that is already flushed.
+     * <p/>
+     * This API is optional as the writer implements a policy for automatically syncing
+     * the log records in the buffer. The buffered edits can be flushed when the buffer
+     * becomes full or a certain period of time is elapsed.
+     */
+    @Override
+    public long flushAndSync() throws IOException {
+        checkClosedOrInError("flushAndSync");
+
+        LOG.debug("FlushAndSync Started");
+        long highestTransactionId = 0;
+        BKLogSegmentWriter writer = getCachedLogWriter();
+        if (null != writer) {
+            highestTransactionId = Math.max(highestTransactionId, FutureUtils.result(writer.commit()));
+            LOG.debug("FlushAndSync Completed");
+        } else {
+            LOG.debug("FlushAndSync Completed - Nothing to Flush");
+        }
+        return highestTransactionId;
+    }
+
+    /**
+     * Close the stream without necessarily flushing immediately.
+     * This may be called if the stream is in error such as after a
+     * previous write or close threw an exception.
+     */
+    @Override
+    public void abort() throws IOException {
+        super.abort();
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
new file mode 100644
index 0000000..6ed662b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
@@ -0,0 +1,90 @@
+/**
+ * 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.twitter.util.Await;
+import com.twitter.util.Duration;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+
+import java.util.concurrent.TimeUnit;
+
+class BKTransmitPacket {
+
+    private final EntryBuffer recordSet;
+    private final long transmitTime;
+    private final Promise<Integer> transmitComplete;
+
+    BKTransmitPacket(EntryBuffer recordSet) {
+        this.recordSet = recordSet;
+        this.transmitTime = System.nanoTime();
+        this.transmitComplete = new Promise<Integer>();
+    }
+
+    EntryBuffer getRecordSet() {
+        return recordSet;
+    }
+
+    Promise<Integer> getTransmitFuture() {
+        return transmitComplete;
+    }
+
+    /**
+     * Complete the transmit with result code <code>transmitRc</code>.
+     * <p>It would notify all the waiters that are waiting via {@link #awaitTransmitComplete(long, TimeUnit)}
+     * or {@link #addTransmitCompleteListener(FutureEventListener)}.
+     *
+     * @param transmitResult
+     *          transmit result code.
+     */
+    public void notifyTransmitComplete(int transmitResult) {
+        transmitComplete.setValue(transmitResult);
+    }
+
+    /**
+     * Register a transmit complete listener.
+     * <p>The listener will be triggered with transmit result when transmit completes.
+     * The method should be non-blocking.
+     *
+     * @param transmitCompleteListener
+     *          listener on transmit completion
+     * @see #awaitTransmitComplete(long, TimeUnit)
+     */
+    void addTransmitCompleteListener(FutureEventListener<Integer> transmitCompleteListener) {
+        transmitComplete.addEventListener(transmitCompleteListener);
+    }
+
+    /**
+     * Await for the transmit to be complete
+     *
+     * @param timeout
+     *          wait timeout
+     * @param unit
+     *          wait timeout unit
+     */
+    int awaitTransmitComplete(long timeout, TimeUnit unit)
+        throws Exception {
+        return Await.result(transmitComplete,
+                Duration.fromTimeUnit(timeout, unit));
+    }
+
+    public long getTransmitTime() {
+        return transmitTime;
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
new file mode 100644
index 0000000..a7b17f4
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
@@ -0,0 +1,289 @@
+/**
+ * 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.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 org.apache.bookkeeper.client.AsyncCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.RegionAwareEnsemblePlacementPolicy;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.net.DNSToSwitchMapping;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.zookeeper.KeeperException;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * BookKeeper Client wrapper over {@link BookKeeper}.
+ *
+ * <h3>Metrics</h3>
+ * <ul>
+ * <li> bookkeeper operation stats are exposed under current scope by {@link BookKeeper}
+ * </ul>
+ */
+public class BookKeeperClient {
+    static final Logger LOG = LoggerFactory.getLogger(BookKeeperClient.class);
+
+    // Parameters to build bookkeeper client
+    private final DistributedLogConfiguration conf;
+    private final String name;
+    private final String zkServers;
+    private final String ledgersPath;
+    private final byte[] passwd;
+    private final ClientSocketChannelFactory channelFactory;
+    private final HashedWheelTimer requestTimer;
+    private final StatsLogger statsLogger;
+
+    // bookkeeper client state
+    private boolean closed = false;
+    private BookKeeper bkc = null;
+    private ZooKeeperClient zkc;
+    private final boolean ownZK;
+    // feature provider
+    private final Optional<FeatureProvider> featureProvider;
+
+    @SuppressWarnings("deprecation")
+    private synchronized void commonInitialization(
+            DistributedLogConfiguration conf, String ledgersPath,
+            ClientSocketChannelFactory channelFactory, StatsLogger statsLogger, HashedWheelTimer requestTimer)
+        throws IOException, InterruptedException, KeeperException {
+        ClientConfiguration bkConfig = new ClientConfiguration();
+        bkConfig.setAddEntryTimeout(conf.getBKClientWriteTimeout());
+        bkConfig.setReadTimeout(conf.getBKClientReadTimeout());
+        bkConfig.setZkLedgersRootPath(ledgersPath);
+        bkConfig.setZkTimeout(conf.getBKClientZKSessionTimeoutMilliSeconds());
+        bkConfig.setNumWorkerThreads(conf.getBKClientNumberWorkerThreads());
+        bkConfig.setEnsemblePlacementPolicy(RegionAwareEnsemblePlacementPolicy.class);
+        bkConfig.setZkRequestRateLimit(conf.getBKClientZKRequestRateLimit());
+        bkConfig.setProperty(RegionAwareEnsemblePlacementPolicy.REPP_DISALLOW_BOOKIE_PLACEMENT_IN_REGION_FEATURE_NAME,
+                DistributedLogConstants.DISALLOW_PLACEMENT_IN_REGION_FEATURE_NAME);
+        // reload configuration from dl configuration with settings prefixed with 'bkc.'
+        ConfUtils.loadConfiguration(bkConfig, conf, "bkc.");
+
+        Class<? extends DNSToSwitchMapping> dnsResolverCls;
+        try {
+            dnsResolverCls = conf.getEnsemblePlacementDnsResolverClass();
+        } catch (ConfigurationException e) {
+            LOG.error("Failed to load bk dns resolver : ", e);
+            throw new IOException("Failed to load bk dns resolver : ", e);
+        }
+        final DNSToSwitchMapping dnsResolver =
+                NetUtils.getDNSResolver(dnsResolverCls, conf.getBkDNSResolverOverrides());
+
+        this.bkc = BookKeeper.newBuilder()
+            .config(bkConfig)
+            .zk(zkc.get())
+            .channelFactory(channelFactory)
+            .statsLogger(statsLogger)
+            .dnsResolver(dnsResolver)
+            .requestTimer(requestTimer)
+            .featureProvider(featureProvider.orNull())
+            .build();
+    }
+
+    BookKeeperClient(DistributedLogConfiguration conf,
+                     String name,
+                     String zkServers,
+                     ZooKeeperClient zkc,
+                     String ledgersPath,
+                     ClientSocketChannelFactory channelFactory,
+                     HashedWheelTimer requestTimer,
+                     StatsLogger statsLogger,
+                     Optional<FeatureProvider> featureProvider) {
+        this.conf = conf;
+        this.name = name;
+        this.zkServers = zkServers;
+        this.ledgersPath = ledgersPath;
+        this.passwd = conf.getBKDigestPW().getBytes(UTF_8);
+        this.channelFactory = channelFactory;
+        this.requestTimer = requestTimer;
+        this.statsLogger = statsLogger;
+        this.featureProvider = featureProvider;
+        this.ownZK = null == zkc;
+        if (null != zkc) {
+            // reference the passing zookeeper client
+            this.zkc = zkc;
+        }
+    }
+
+    private synchronized void initialize() throws IOException {
+        if (null != this.bkc) {
+            return;
+        }
+        if (null == this.zkc) {
+            int zkSessionTimeout = conf.getBKClientZKSessionTimeoutMilliSeconds();
+            RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
+                        conf.getBKClientZKRetryBackoffStartMillis(),
+                        conf.getBKClientZKRetryBackoffMaxMillis(), conf.getBKClientZKNumRetries());
+            Credentials credentials = Credentials.NONE;
+            if (conf.getZkAclId() != null) {
+                credentials = new DigestCredentials(conf.getZkAclId(), conf.getZkAclId());
+            }
+
+            this.zkc = new ZooKeeperClient(name + ":zk", zkSessionTimeout, 2 * zkSessionTimeout, zkServers,
+                                           retryPolicy, statsLogger.scope("bkc_zkc"), conf.getZKClientNumberRetryThreads(),
+                                           conf.getBKClientZKRequestRateLimit(), credentials);
+        }
+
+        try {
+            commonInitialization(conf, ledgersPath, channelFactory, statsLogger, requestTimer);
+        } catch (InterruptedException e) {
+            throw new DLInterruptedException("Interrupted on creating bookkeeper client " + name + " : ", e);
+        } catch (KeeperException e) {
+            throw new ZKException("Error on creating bookkeeper client " + name + " : ", e);
+        }
+
+        if (ownZK) {
+            LOG.info("BookKeeper Client created {} with its own ZK Client : ledgersPath = {}, numRetries = {}, " +
+                    "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}",
+                    new Object[] { name, ledgersPath,
+                    conf.getBKClientZKNumRetries(), conf.getBKClientZKSessionTimeoutMilliSeconds(),
+                    conf.getBKClientZKRetryBackoffStartMillis(), conf.getBKClientZKRetryBackoffMaxMillis(),
+                    conf.getBkDNSResolverOverrides() });
+        } else {
+            LOG.info("BookKeeper Client created {} with shared zookeeper client : ledgersPath = {}, numRetries = {}, " +
+                    "sessionTimeout = {}, backoff = {}, maxBackoff = {}, dnsResolver = {}",
+                    new Object[] { name, ledgersPath,
+                    conf.getZKNumRetries(), conf.getZKSessionTimeoutMilliseconds(),
+                    conf.getZKRetryBackoffStartMillis(), conf.getZKRetryBackoffMaxMillis(),
+                    conf.getBkDNSResolverOverrides() });
+        }
+    }
+
+
+    public synchronized BookKeeper get() throws IOException {
+        checkClosedOrInError();
+        if (null == bkc) {
+            initialize();
+        }
+        return bkc;
+    }
+
+    // Util functions
+    public Future<LedgerHandle> createLedger(int ensembleSize,
+                                             int writeQuorumSize,
+                                             int ackQuorumSize) {
+        BookKeeper bk;
+        try {
+            bk = get();
+        } catch (IOException ioe) {
+            return Future.exception(ioe);
+        }
+        final Promise<LedgerHandle> promise = new Promise<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));
+                        } else {
+                            promise.updateIfEmpty(new Throw<LedgerHandle>(BKException.create(rc)));
+                        }
+                    }
+                }, null);
+        return promise;
+    }
+
+    public Future<Void> deleteLedger(long lid,
+                                     final boolean ignoreNonExistentLedger) {
+        BookKeeper bk;
+        try {
+            bk = get();
+        } catch (IOException ioe) {
+            return Future.exception(ioe);
+        }
+        final Promise<Void> promise = new Promise<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));
+                } else if (BKException.Code.NoSuchLedgerExistsException == rc) {
+                    if (ignoreNonExistentLedger) {
+                        promise.updateIfEmpty(new Return<Void>(null));
+                    } else {
+                        promise.updateIfEmpty(new Throw<Void>(BKException.create(rc)));
+                    }
+                } else {
+                    promise.updateIfEmpty(new Throw<Void>(BKException.create(rc)));
+                }
+            }
+        }, null);
+        return promise;
+    }
+
+    public void close() {
+        BookKeeper bkcToClose;
+        ZooKeeperClient zkcToClose;
+        synchronized (this) {
+            if (closed) {
+                return;
+            }
+            closed = true;
+            bkcToClose = bkc;
+            zkcToClose = zkc;
+        }
+
+        LOG.info("BookKeeper Client closed {}", name);
+        if (null != bkcToClose) {
+            try {
+                bkcToClose.close();
+            } catch (InterruptedException e) {
+                LOG.warn("Interrupted on closing bookkeeper client {} : ", name, e);
+                Thread.currentThread().interrupt();
+            } catch (BKException e) {
+                LOG.warn("Error on closing bookkeeper client {} : ", name, e);
+            }
+        }
+        if (null != zkcToClose) {
+            if (ownZK) {
+                zkcToClose.close();
+            }
+        }
+    }
+
+    public synchronized void checkClosedOrInError() throws AlreadyClosedException {
+        if (closed) {
+            LOG.error("BookKeeper Client {} is already closed", name);
+            throw new AlreadyClosedException("BookKeeper Client " + name + " is already closed");
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java
new file mode 100644
index 0000000..a356f9f
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java
@@ -0,0 +1,209 @@
+/**
+ * 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.base.Optional;
+import com.google.common.base.Preconditions;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.apache.bookkeeper.feature.FeatureProvider;
+
+import org.apache.bookkeeper.feature.Feature;
+
+/**
+ * Builder to build bookkeeper client.
+ */
+public class BookKeeperClientBuilder {
+
+    /**
+     * Create a bookkeeper client builder to build bookkeeper clients.
+     *
+     * @return bookkeeper client builder.
+     */
+    public static BookKeeperClientBuilder newBuilder() {
+        return new BookKeeperClientBuilder();
+    }
+
+    // client name
+    private String name = null;
+    // dl config
+    private DistributedLogConfiguration dlConfig = null;
+    // bookkeeper settings
+    // zookeeper client
+    private ZooKeeperClient zkc = null;
+    // or zookeeper servers
+    private String zkServers = null;
+    // ledgers path
+    private String ledgersPath = null;
+    // statsLogger
+    private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
+    // client channel factory
+    private ClientSocketChannelFactory channelFactory = null;
+    // request timer
+    private HashedWheelTimer requestTimer = null;
+    // feature provider
+    private Optional<FeatureProvider> featureProvider = Optional.absent();
+
+    // Cached BookKeeper Client
+    private BookKeeperClient cachedClient = null;
+
+    /**
+     * Private bookkeeper builder.
+     */
+    private BookKeeperClientBuilder() {}
+
+    /**
+     * Set client name.
+     *
+     * @param name
+     *          client name.
+     * @return builder
+     */
+    public synchronized BookKeeperClientBuilder name(String name) {
+        this.name = name;
+        return this;
+    }
+
+    /**
+     * <i>dlConfig</i> used to configure bookkeeper client.
+     *
+     * @param dlConfig
+     *          distributedlog config.
+     * @return builder.
+     */
+    public synchronized BookKeeperClientBuilder dlConfig(DistributedLogConfiguration dlConfig) {
+        this.dlConfig = dlConfig;
+        return this;
+    }
+
+    /**
+     * Set the zkc used to build bookkeeper client. If a zookeeper client is provided in this
+     * method, bookkeeper client will use it rather than creating a brand new one.
+     *
+     * @param zkc
+     *          zookeeper client.
+     * @return builder
+     * @see #zkServers(String)
+     */
+    public synchronized BookKeeperClientBuilder zkc(ZooKeeperClient zkc) {
+        this.zkc = zkc;
+        return this;
+    }
+
+    /**
+     * Set the zookeeper servers that bookkeeper client would connect to. If no zookeeper client
+     * is provided by {@link #zkc(ZooKeeperClient)}, bookkeeper client will use the given string
+     * to create a brand new zookeeper client.
+     *
+     * @param zkServers
+     *          zookeeper servers that bookkeeper client would connect to.
+     * @return builder
+     * @see #zkc(ZooKeeperClient)
+     */
+    public synchronized BookKeeperClientBuilder zkServers(String zkServers) {
+        this.zkServers = zkServers;
+        return this;
+    }
+
+    /**
+     * Set the ledgers path that bookkeeper client is going to access.
+     *
+     * @param ledgersPath
+     *          ledgers path
+     * @return builder
+     * @see org.apache.bookkeeper.conf.ClientConfiguration#getZkLedgersRootPath()
+     */
+    public synchronized BookKeeperClientBuilder ledgersPath(String ledgersPath) {
+        this.ledgersPath = ledgersPath;
+        return this;
+    }
+
+    /**
+     * Build BookKeeper client using existing <i>bkc</i> client.
+     *
+     * @param bkc
+     *          bookkeeper client.
+     * @return builder
+     */
+    public synchronized BookKeeperClientBuilder bkc(BookKeeperClient bkc) {
+        this.cachedClient = bkc;
+        return this;
+    }
+
+    /**
+     * Build BookKeeper client using existing <i>channelFactory</i>.
+     *
+     * @param channelFactory
+     *          Channel Factory used to build bookkeeper client.
+     * @return bookkeeper client builder.
+     */
+    public synchronized BookKeeperClientBuilder channelFactory(ClientSocketChannelFactory channelFactory) {
+        this.channelFactory = channelFactory;
+        return this;
+    }
+
+    /**
+     * Build BookKeeper client using existing <i>request timer</i>.
+     *
+     * @param requestTimer
+     *          HashedWheelTimer used to build bookkeeper client.
+     * @return bookkeeper client builder.
+     */
+    public synchronized BookKeeperClientBuilder requestTimer(HashedWheelTimer requestTimer) {
+        this.requestTimer = requestTimer;
+        return this;
+    }
+
+    /**
+     * Build BookKeeper Client using given stats logger <i>statsLogger</i>.
+     *
+     * @param statsLogger
+     *          stats logger to report stats
+     * @return builder.
+     */
+    public synchronized BookKeeperClientBuilder statsLogger(StatsLogger statsLogger) {
+        this.statsLogger = statsLogger;
+        return this;
+    }
+
+    public synchronized BookKeeperClientBuilder featureProvider(Optional<FeatureProvider> featureProvider) {
+        this.featureProvider = featureProvider;
+        return this;
+    }
+
+    private void validateParameters() {
+        Preconditions.checkNotNull(name, "Missing client name.");
+        Preconditions.checkNotNull(dlConfig, "Missing DistributedLog Configuration.");
+        Preconditions.checkArgument(null == zkc || null == zkServers, "Missing zookeeper setting.");
+        Preconditions.checkNotNull(ledgersPath, "Missing Ledgers Root Path.");
+    }
+
+    public synchronized BookKeeperClient build() {
+        if (null == cachedClient) {
+            cachedClient = buildClient();
+        }
+        return cachedClient;
+    }
+
+    private BookKeeperClient buildClient() {
+        validateParameters();
+        return new BookKeeperClient(dlConfig, name, zkServers, zkc, ledgersPath, channelFactory, requestTimer, statsLogger, featureProvider);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
new file mode 100644
index 0000000..0cb608f
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
@@ -0,0 +1,3528 @@
+/**
+ * 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.base.Optional;
+import com.google.common.base.Preconditions;
+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.net.DNSResolverForRacks;
+import org.apache.distributedlog.net.DNSResolverForRows;
+import org.apache.bookkeeper.conf.ClientConfiguration;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.net.DNSToSwitchMapping;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.Configuration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.PropertiesConfiguration;
+import org.apache.commons.configuration.SystemConfiguration;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URL;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.Set;
+
+/**
+ * DistributedLog Configuration.
+ * <p>
+ * DistributedLog configuration is basically a properties based configuration, which extends from
+ * Apache commons {@link CompositeConfiguration}. All the DL settings are in camel case and prefixed
+ * with a meaningful component name. for example, `zkSessionTimeoutSeconds` means <i>SessionTimeoutSeconds</i>
+ * for component `zk`.
+ *
+ * <h3>BookKeeper Configuration</h3>
+ *
+ * BookKeeper client configuration settings could be loaded via DistributedLog configuration. All those
+ * settings are prefixed with <i>`bkc.`</i>. For example, <i>bkc.zkTimeout</i> in distributedlog configuration
+ * will be applied as <i>`zkTimeout`</i> in bookkeeper client configuration.
+ *
+ * <h3>How to load configuration</h3>
+ *
+ * The default distributedlog configuration is constructed by instantiated a new instance. This
+ * distributedlog configuration will automatically load the settings that specified via
+ * {@link SystemConfiguration}.
+ *
+ * <pre>
+ *      DistributedLogConfiguration conf = new DistributedLogConfiguration();
+ * </pre>
+ *
+ * The recommended way is to load configuration from URL that points to a configuration file
+ * ({@link #loadConf(URL)}).
+ *
+ * <pre>
+ *      String configFile = "/path/to/distributedlog/conf/file";
+ *      DistributedLogConfiguration conf = new DistributedLogConfiguration();
+ *      conf.loadConf(new File(configFile).toURI().toURL());
+ * </pre>
+ *
+ * @see org.apache.bookkeeper.conf.ClientConfiguration
+ */
+public class DistributedLogConfiguration extends CompositeConfiguration {
+    static final Logger LOG = LoggerFactory.getLogger(DistributedLogConfiguration.class);
+
+    private static ClassLoader defaultLoader;
+
+    static {
+        defaultLoader = Thread.currentThread().getContextClassLoader();
+        if (null == defaultLoader) {
+            defaultLoader = DistributedLogConfiguration.class.getClassLoader();
+        }
+    }
+
+    //
+    // ZooKeeper Related Settings
+    //
+
+    public static final String BKDL_ZK_ACL_ID = "zkAclId";
+    public static final String BKDL_ZK_ACL_ID_DEFAULT = null;
+    public static final String BKDL_ZK_SESSION_TIMEOUT_SECONDS = "zkSessionTimeoutSeconds";
+    public static final int BKDL_ZK_SESSION_TIMEOUT_SECONDS_DEFAULT = 30;
+    public static final String BKDL_ZK_REQUEST_RATE_LIMIT = "zkRequestRateLimit";
+    public static final double BKDL_ZK_REQUEST_RATE_LIMIT_DEFAULT = 0;
+    public static final String BKDL_ZK_NUM_RETRIES = "zkNumRetries";
+    public static final int BKDL_ZK_NUM_RETRIES_DEFAULT = 3;
+    public static final String BKDL_ZK_RETRY_BACKOFF_START_MILLIS = "zkRetryStartBackoffMillis";
+    public static final int BKDL_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT = 5000;
+    public static final String BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS = "zkRetryMaxBackoffMillis";
+    public static final int BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT = 30000;
+    public static final String BKDL_ZKCLIENT_NUM_RETRY_THREADS = "zkcNumRetryThreads";
+    public static final int BKDL_ZKCLIENT_NUM_RETRY_THREADS_DEFAULT = 1;
+
+    //
+    // BookKeeper Related Settings
+    //
+
+    // BookKeeper zookeeper settings
+    public static final String BKDL_BKCLIENT_ZK_SESSION_TIMEOUT = "bkcZKSessionTimeoutSeconds";
+    public static final int BKDL_BKCLIENT_ZK_SESSION_TIMEOUT_DEFAULT = 30;
+    public static final String BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT = "bkcZKRequestRateLimit";
+    public static final double BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT_DEFAULT = 0;
+    public static final String BKDL_BKCLIENT_ZK_NUM_RETRIES = "bkcZKNumRetries";
+    public static final int BKDL_BKCLIENT_ZK_NUM_RETRIES_DEFAULT = 3;
+    public static final String BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS = "bkcZKRetryStartBackoffMillis";
+    public static final int BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT = 5000;
+    public static final String BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS = "bkcZKRetryMaxBackoffMillis";
+    public static final int BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT = 30000;
+
+    // Bookkeeper ensemble placement settings
+    // Bookkeeper ensemble size
+    public static final String BKDL_BOOKKEEPER_ENSEMBLE_SIZE = "bkcEnsembleSize";
+    // @Deprecated
+    public static final String BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD = "ensemble-size";
+    public static final int BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT = 3;
+    // Bookkeeper write quorum size
+    public static final String BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE = "bkcWriteQuorumSize";
+    // @Deprecated
+    public static final String BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD = "write-quorum-size";
+    public static final int BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT = 3;
+    // Bookkeeper ack quorum size
+    public static final String BKDL_BOOKKEEPER_ACK_QUORUM_SIZE = "bkcAckQuorumSize";
+    // @Deprecated
+    public static final String BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD = "ack-quorum-size";
+    public static final int BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT = 2;
+    public static final String BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT = "bkRowAwareEnsemblePlacement";
+    public static final String BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_OLD = "row-aware-ensemble-placement";
+    public static final boolean BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_DEFAULT = false;
+    public static final String BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS = "bkEnsemblePlacementDnsResolverClass";
+    public static final String BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS_DEFAULT =
+            DNSResolverForRacks.class.getName();
+    public static final String BKDL_BK_DNS_RESOLVER_OVERRIDES = "dnsResolverOverrides";
+    public static final String BKDL_BK_DNS_RESOLVER_OVERRIDES_DEFAULT = "";
+
+    // General Settings
+    // @Deprecated
+    public static final String BKDL_BOOKKEEPER_DIGEST_PW = "digestPw";
+    public static final String BKDL_BOOKKEEPER_DIGEST_PW_DEFAULT = "";
+    public static final String BKDL_BKCLIENT_NUM_IO_THREADS = "bkcNumIOThreads";
+    public static final String BKDL_TIMEOUT_TIMER_TICK_DURATION_MS = "timerTickDuration";
+    public static final long BKDL_TIMEOUT_TIMER_TICK_DURATION_MS_DEFAULT = 100;
+    public static final String BKDL_TIMEOUT_TIMER_NUM_TICKS = "timerNumTicks";
+    public static final int BKDL_TIMEOUT_TIMER_NUM_TICKS_DEFAULT = 1024;
+
+    //
+    // Deprecated BookKeeper Settings (in favor of "bkc." style bookkeeper settings)
+    //
+
+    public static final String BKDL_BKCLIENT_READ_TIMEOUT = "bkcReadTimeoutSeconds";
+    public static final int BKDL_BKCLIENT_READ_TIMEOUT_DEFAULT = 10;
+    public static final String BKDL_BKCLIENT_WRITE_TIMEOUT = "bkcWriteTimeoutSeconds";
+    public static final int BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT = 10;
+    public static final String BKDL_BKCLIENT_NUM_WORKER_THREADS = "bkcNumWorkerThreads";
+    public static final int BKDL_BKCLEINT_NUM_WORKER_THREADS_DEFAULT = 1;
+
+    //
+    // DL General Settings
+    //
+
+    // Executor Parameters
+    public static final String BKDL_NUM_WORKER_THREADS = "numWorkerThreads";
+    public static final String BKDL_NUM_READAHEAD_WORKER_THREADS = "numReadAheadWorkerThreads";
+    public static final String BKDL_NUM_LOCKSTATE_THREADS = "numLockStateThreads";
+    public static final String BKDL_NUM_RESOURCE_RELEASE_THREADS = "numResourceReleaseThreads";
+    public static final String BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS = "schedulerShutdownTimeoutMs";
+    public static final int BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS_DEFAULT = 5000;
+    public static final String BKDL_USE_DAEMON_THREAD = "useDaemonThread";
+    public static final boolean BKDL_USE_DAEMON_THREAD_DEFAULT = false;
+
+    // Metadata Parameters
+    public static final String BKDL_LEDGER_METADATA_LAYOUT_VERSION = "ledgerMetadataLayoutVersion";
+    public static final String BKDL_LEDGER_METADATA_LAYOUT_VERSION_OLD = "ledger-metadata-layout";
+    public static final int BKDL_LEDGER_METADATA_LAYOUT_VERSION_DEFAULT =
+            LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value;
+    public static final String BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK = "ledgerMetadataSkipMinVersionCheck";
+    public static final boolean BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK_DEFAULT = false;
+    public static final String BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER = "firstLogsegmentSequenceNumber";
+    public static final String BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_OLD = "first-logsegment-sequence-number";
+    public static final long BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_DEFAULT =
+            DistributedLogConstants.FIRST_LOGSEGMENT_SEQNO;
+    public static final String BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED = "logSegmentSequenceNumberValidationEnabled";
+    public static final boolean BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED_DEFAULT = true;
+    public static final String BKDL_ENABLE_RECORD_COUNTS = "enableRecordCounts";
+    public static final boolean BKDL_ENABLE_RECORD_COUNTS_DEFAULT = true;
+    public static final String BKDL_MAXID_SANITYCHECK = "maxIdSanityCheck";
+    public static final boolean BKDL_MAXID_SANITYCHECK_DEFAULT = true;
+    public static final String BKDL_ENCODE_REGION_ID_IN_VERSION = "encodeRegionIDInVersion";
+    public static final boolean BKDL_ENCODE_REGION_ID_IN_VERSION_DEFAULT = false;
+    // (@Deprecated)
+    public static final String BKDL_LOGSEGMENT_NAME_VERSION = "logSegmentNameVersion";
+    public static final int BKDL_LOGSEGMENT_NAME_VERSION_DEFAULT = DistributedLogConstants.LOGSEGMENT_NAME_VERSION;
+    // (@Derepcated) Name for the default (non-partitioned) stream
+    public static final String BKDL_UNPARTITIONED_STREAM_NAME = "unpartitionedStreamName";
+    public static final String BKDL_UNPARTITIONED_STREAM_NAME_DEFAULT = "<default>";
+
+    // Log Segment Cache Parameters
+    public static final String BKDL_LOGSEGMENT_CACHE_TTL_MS = "logSegmentCacheTTLMs";
+    public static final long BKDL_LOGSEGMENT_CACHE_TTL_MS_DEFAULT = 600000; // 10 mins
+    public static final String BKDL_LOGSEGMENT_CACHE_MAX_SIZE = "logSegmentCacheMaxSize";
+    public static final long BKDL_LOGSEGMENT_CACHE_MAX_SIZE_DEFAULT = 10000;
+    public static final String BKDL_LOGSEGMENT_CACHE_ENABLED = "logSegmentCacheEnabled";
+    public static final boolean BKDL_LOGSEGMENT_CACHE_ENABLED_DEFAULT = true;
+
+    //
+    // DL Writer Settings
+    //
+
+    // General Settings
+    public static final String BKDL_CREATE_STREAM_IF_NOT_EXISTS = "createStreamIfNotExists";
+    public static final boolean BKDL_CREATE_STREAM_IF_NOT_EXISTS_DEFAULT = true;
+    public static final String BKDL_LOG_FLUSH_TIMEOUT = "logFlushTimeoutSeconds";
+    public static final int BKDL_LOG_FLUSH_TIMEOUT_DEFAULT = 30;
+    /**
+     *  CompressionCodec.Type     String to use (See CompressionUtils)
+     *  ---------------------     ------------------------------------
+     *          NONE               none
+     *          LZ4                lz4
+     *          UNKNOWN            any other instance of String.class
+     */
+    public static final String BKDL_COMPRESSION_TYPE = "compressionType";
+    public static final String BKDL_COMPRESSION_TYPE_DEFAULT = "none";
+    public static final String BKDL_FAILFAST_ON_STREAM_NOT_READY = "failFastOnStreamNotReady";
+    public static final boolean BKDL_FAILFAST_ON_STREAM_NOT_READY_DEFAULT = false;
+    public static final String BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR = "disableRollingOnLogSegmentError";
+    public static final boolean BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR_DEFAULT = false;
+
+    // Durability Settings
+    public static final String BKDL_IS_DURABLE_WRITE_ENABLED = "isDurableWriteEnabled";
+    public static final boolean BKDL_IS_DURABLE_WRITE_ENABLED_DEFAULT = true;
+
+    // Transmit Settings
+    public static final String BKDL_OUTPUT_BUFFER_SIZE = "writerOutputBufferSize";
+    public static final String BKDL_OUTPUT_BUFFER_SIZE_OLD = "output-buffer-size";
+    public static final int BKDL_OUTPUT_BUFFER_SIZE_DEFAULT = 1024;
+    public static final String BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS = "periodicFlushFrequencyMilliSeconds";
+    public static final int BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT = 0;
+    public static final String BKDL_ENABLE_IMMEDIATE_FLUSH = "enableImmediateFlush";
+    public static final boolean BKDL_ENABLE_IMMEDIATE_FLUSH_DEFAULT = false;
+    public static final String BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS = "minimumDelayBetweenImmediateFlushMilliSeconds";
+    public static final int BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS_DEFAULT = 0;
+    public static final String BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS = "periodicKeepAliveMilliSeconds";
+    public static final int BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS_DEFAULT = 0;
+
+    // Retention/Truncation Settings
+    public static final String BKDL_RETENTION_PERIOD_IN_HOURS = "logSegmentRetentionHours";
+    public static final String BKDL_RETENTION_PERIOD_IN_HOURS_OLD = "retention-size";
+    public static final int BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT = 72;
+    public static final String BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION = "explicitTruncationByApp";
+    public static final boolean BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION_DEFAULT = false;
+
+    // Log Segment Rolling Settings
+    public static final String BKDL_ROLLING_INTERVAL_IN_MINUTES = "logSegmentRollingMinutes";
+    public static final String BKDL_ROLLING_INTERVAL_IN_MINUTES_OLD = "rolling-interval";
+    public static final int BKDL_ROLLING_INTERVAL_IN_MINUTES_DEFAULT = 120;
+    public static final String BKDL_MAX_LOGSEGMENT_BYTES = "maxLogSegmentBytes";
+    public static final int BKDL_MAX_LOGSEGMENT_BYTES_DEFAULT = 256 * 1024 * 1024; // default 256MB
+    public static final String BKDL_LOGSEGMENT_ROLLING_CONCURRENCY = "logSegmentRollingConcurrency";
+    public static final int BKDL_LOGSEGMENT_ROLLING_CONCURRENCY_DEFAULT = 1;
+
+    // Lock Settings
+    public static final String BKDL_WRITE_LOCK_ENABLED = "writeLockEnabled";
+    public static final boolean BKDL_WRITE_LOCK_ENABLED_DEFAULT = true;
+    public static final String BKDL_LOCK_TIMEOUT = "lockTimeoutSeconds";
+    public static final long BKDL_LOCK_TIMEOUT_DEFAULT = 30;
+    public static final String BKDL_LOCK_REACQUIRE_TIMEOUT = "lockReacquireTimeoutSeconds";
+    public static final long BKDL_LOCK_REACQUIRE_TIMEOUT_DEFAULT = DistributedLogConstants.LOCK_REACQUIRE_TIMEOUT_DEFAULT;
+    public static final String BKDL_LOCK_OP_TIMEOUT = "lockOpTimeoutSeconds";
+    public static final long BKDL_LOCK_OP_TIMEOUT_DEFAULT = DistributedLogConstants.LOCK_OP_TIMEOUT_DEFAULT;
+
+    // Ledger Allocator Settings
+    public static final String BKDL_ENABLE_LEDGER_ALLOCATOR_POOL = "enableLedgerAllocatorPool";
+    public static final boolean BKDL_ENABLE_LEDGER_ALLOCATOR_POOL_DEFAULT = false;
+    public static final String BKDL_LEDGER_ALLOCATOR_POOL_PATH = "ledgerAllocatorPoolPath";
+    public static final String BKDL_LEDGER_ALLOCATOR_POOL_PATH_DEFAULT = DistributedLogConstants.ALLOCATION_POOL_NODE;
+    public static final String BKDL_LEDGER_ALLOCATOR_POOL_NAME = "ledgerAllocatorPoolName";
+    public static final String BKDL_LEDGER_ALLOCATOR_POOL_NAME_DEFAULT = null;
+    public static final String BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE = "ledgerAllocatorPoolCoreSize";
+    public static final int BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE_DEFAULT = 20;
+
+    // Write Limit Settings
+    public static final String BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT = "perWriterOutstandingWriteLimit";
+    public static final int BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT_DEFAULT = -1;
+    public static final String BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT = "globalOutstandingWriteLimit";
+    public static final int BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT_DEFAULT = -1;
+    public static final String BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE = "outstandingWriteLimitDarkmode";
+    public static final boolean BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE_DEFAULT = true;
+
+    //
+    // DL Reader Settings
+    //
+
+    // General Settings
+    public static final String BKDL_READLAC_OPTION = "readLACLongPoll";
+    public static final int BKDL_READLAC_OPTION_DEFAULT = 3; //BKLogPartitionReadHandler.ReadLACOption.READENTRYPIGGYBACK_SEQUENTIAL.value
+    public static final String BKDL_READLACLONGPOLL_TIMEOUT = "readLACLongPollTimeout";
+    public static final int BKDL_READLACLONGPOLL_TIMEOUT_DEFAULT = 1000;
+    public static final String BKDL_DESERIALIZE_RECORDSET_ON_READS = "deserializeRecordSetOnReads";
+    public static final boolean BKDL_DESERIALIZE_RECORDSET_ON_READS_DEFAULT = true;
+
+    // Idle reader settings
+    public static final String BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS = "readerIdleWarnThresholdMillis";
+    public static final int BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS_DEFAULT = 120000;
+    public static final String BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS = "readerIdleErrorThresholdMillis";
+    public static final int BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT = Integer.MAX_VALUE;
+
+    // Reader constraint settings
+    public static final String BKDL_READER_IGNORE_TRUNCATION_STATUS = "ignoreTruncationStatus";
+    public static final boolean BKDL_READER_IGNORE_TRUNCATION_STATUS_DEFAULT = false;
+    public static final String BKDL_READER_ALERT_POSITION_ON_TRUNCATED = "alertPositionOnTruncated";
+    public static final boolean BKDL_READER_ALERT_POSITION_ON_TRUNCATED_DEFAULT = true;
+    public static final String BKDL_READER_POSITION_GAP_DETECTION_ENABLED = "positionGapDetectionEnabled";
+    public static final boolean BKDL_READER_POSITION_GAP_DETECTION_ENABLED_DEFAULT = false;
+
+    // Read ahead related parameters
+    public static final String BKDL_ENABLE_READAHEAD = "enableReadAhead";
+    public static final boolean BKDL_ENABLE_READAHEAD_DEFAULT = true;
+    public static final String BKDL_ENABLE_FORCEREAD = "enableForceRead";
+    public static final boolean BKDL_ENABLE_FORCEREAD_DEFAULT = true;
+    public static final String BKDL_READAHEAD_MAX_RECORDS = "readAheadMaxRecords";
+    public static final String BKDL_READAHEAD_MAX_RECORDS_OLD = "ReadAheadMaxEntries";
+    public static final int BKDL_READAHEAD_MAX_RECORDS_DEFAULT = 10;
+    public static final String BKDL_READAHEAD_BATCHSIZE = "readAheadBatchSize";
+    public static final String BKDL_READAHEAD_BATCHSIZE_OLD = "ReadAheadBatchSize";
+    public static final int BKDL_READAHEAD_BATCHSIZE_DEFAULT = 2;
+    public static final String BKDL_READAHEAD_WAITTIME = "readAheadWaitTime";
+    public static final String BKDL_READAHEAD_WAITTIME_OLD = "ReadAheadWaitTime";
+    public static final int BKDL_READAHEAD_WAITTIME_DEFAULT = 200;
+    public static final String BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM = "readAheadWaitTimeOnEndOfStream";
+    public static final String BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM_OLD = "ReadAheadWaitTimeOnEndOfStream";
+    public static final int BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM_DEFAULT = 10000;
+    public static final String BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS =
+            "readAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis";
+    public static final int BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS_DEFAULT = 10000;
+    public static final String BKDL_READAHEAD_SKIP_BROKEN_ENTRIES = "readAheadSkipBrokenEntries";
+    public static final boolean BKDL_READAHEAD_SKIP_BROKEN_ENTRIES_DEFAULT = false;
+    public static final String BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT = "numPrefetchEntriesPerLogSegment";
+    public static final int BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT = 4;
+    public static final String BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT = "maxPrefetchEntriesPerLogSegment";
+    public static final int BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT = 32;
+
+    // Scan Settings
+    public static final String BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN = "firstNumEntriesEachPerLastRecordScan";
+    public static final int BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT = 2;
+    public static final String BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN = "maxNumEntriesPerReadLastRecordScan";
+    public static final int BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT = 16;
+
+    // Log Existence Settings
+    public static final String BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS = "checkLogExistenceBackoffStartMillis";
+    public static final int BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS_DEFAULT = 200;
+    public static final String BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS = "checkLogExistenceBackoffMaxMillis";
+    public static final int BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS_DEFAULT = 1000;
+
+    //
+    // Tracing/Stats Settings
+    //
+
+    public static final String BKDL_TRACE_READAHEAD_DELIVERY_LATENCY = "traceReadAheadDeliveryLatency";
+    public static final boolean BKDL_TRACE_READAHEAD_DELIVERY_LATENCY_DEFAULT = false;
+    public static final String BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS = "metadataLatencyWarnThresholdMs";
+    public static final long BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT = DistributedLogConstants.LATENCY_WARN_THRESHOLD_IN_MILLIS;
+    public static final String BKDL_DATA_LATENCY_WARN_THRESHOLD_MS = "dataLatencyWarnThresholdMs";
+    public static final long BKDL_DATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT = 2 * DistributedLogConstants.LATENCY_WARN_THRESHOLD_IN_MILLIS;
+    public static final String BKDL_TRACE_READAHEAD_METADATA_CHANGES = "traceReadAheadMetadataChanges";
+    public static final boolean BKDL_TRACE_READAHEAD_MEATDATA_CHANGES_DEFAULT = false;
+    public final static String BKDL_ENABLE_TASK_EXECUTION_STATS = "enableTaskExecutionStats";
+    public final static boolean BKDL_ENABLE_TASK_EXECUTION_STATS_DEFAULT = false;
+    public final static String BKDL_TASK_EXECUTION_WARN_TIME_MICROS = "taskExecutionWarnTimeMicros";
+    public final static long BKDL_TASK_EXECUTION_WARN_TIME_MICROS_DEFAULT = 100000;
+    public static final String BKDL_ENABLE_PERSTREAM_STAT = "enablePerStreamStat";
+    public static final boolean BKDL_ENABLE_PERSTREAM_STAT_DEFAULT = false;
+
+    //
+    // Settings for Feature Providers
+    //
+
+    public static final String BKDL_FEATURE_PROVIDER_CLASS = "featureProviderClass";
+
+    //
+    // Settings for Configuration Based Feature Provider
+    //
+
+    public static final String BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH = "fileFeatureProviderBaseConfigPath";
+    public static final String BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH_DEFAULT = "decider.conf";
+    public static final String BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH = "fileFeatureProviderOverlayConfigPath";
+    public static final String BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH_DEFAULT = null;
+
+    //
+    // Settings for Namespaces
+    //
+
+    public static final String BKDL_FEDERATED_NAMESPACE_ENABLED = "federatedNamespaceEnabled";
+    public static final boolean BKDL_FEDERATED_NAMESPACE_ENABLED_DEFAULT = false;
+    public static final String BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE = "federatedMaxLogsPerSubnamespace";
+    public static final int BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE_DEFAULT = 15000;
+    public static final String BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS = "federatedCheckExistenceWhenCacheMiss";
+    public static final boolean BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS_DEFAULT = true;
+
+    // Settings for Configurations
+
+    public static final String BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC = "dynamicConfigReloadIntervalSec";
+    public static final int BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC_DEFAULT = 60;
+    public static final String BKDL_STREAM_CONFIG_ROUTER_CLASS = "streamConfigRouterClass";
+    public static final String BKDL_STREAM_CONFIG_ROUTER_CLASS_DEFAULT = "org.apache.distributedlog.service.config.IdentityConfigRouter";
+
+    // Settings for RateLimit (used by distributedlog-service)
+
+    public static final String BKDL_BPS_SOFT_WRITE_LIMIT = "bpsSoftWriteLimit";
+    public static final int BKDL_BPS_SOFT_WRITE_LIMIT_DEFAULT = -1;
+    public static final String BKDL_BPS_HARD_WRITE_LIMIT = "bpsHardWriteLimit";
+    public static final int BKDL_BPS_HARD_WRITE_LIMIT_DEFAULT = -1;
+    public static final String BKDL_RPS_SOFT_WRITE_LIMIT = "rpsSoftWriteLimit";
+    public static final int BKDL_RPS_SOFT_WRITE_LIMIT_DEFAULT = -1;
+    public static final String BKDL_RPS_HARD_WRITE_LIMIT = "rpsHardWriteLimit";
+    public static final int BKDL_RPS_HARD_WRITE_LIMIT_DEFAULT = -1;
+
+    // Rate and resource limits: per shard
+
+    public static final String BKDL_RPS_SOFT_SERVICE_LIMIT = "rpsSoftServiceLimit";
+    public static final int BKDL_RPS_SOFT_SERVICE_LIMIT_DEFAULT = -1;
+    public static final String BKDL_RPS_HARD_SERVICE_LIMIT = "rpsHardServiceLimit";
+    public static final int BKDL_RPS_HARD_SERVICE_LIMIT_DEFAULT = -1;
+    public static final String BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT = "rpsStreamAcquireServiceLimit";
+    public static final int BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT = -1;
+    public static final String BKDL_BPS_SOFT_SERVICE_LIMIT = "bpsSoftServiceLimit";
+    public static final int BKDL_BPS_SOFT_SERVICE_LIMIT_DEFAULT = -1;
+    public static final String BKDL_BPS_HARD_SERVICE_LIMIT = "bpsHardServiceLimit";
+    public static final int BKDL_BPS_HARD_SERVICE_LIMIT_DEFAULT = -1;
+    public static final String BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT = "bpsStreamAcquireServiceLimit";
+    public static final int BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT = -1;
+
+    // Settings for Partitioning
+
+    public static final String BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY = "maxAcquiredPartitionsPerProxy";
+    public static final int BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY_DEFAULT = -1;
+
+    public static final String BKDL_MAX_CACHED_PARTITIONS_PER_PROXY = "maxCachedPartitionsPerProxy";
+    public static final int BKDL_MAX_CACHED_PARTITIONS_PER_PROXY_DEFAULT = -1;
+
+    //
+    // Settings for Error Injection
+    //
+    public static final String BKDL_EI_INJECT_WRITE_DELAY = "eiInjectWriteDelay";
+    public static final boolean BKDL_EI_INJECT_WRITE_DELAY_DEFAULT = false;
+    public static final String BKDL_EI_INJECTED_WRITE_DELAY_PERCENT = "eiInjectedWriteDelayPercent";
+    public static final double BKDL_EI_INJECTED_WRITE_DELAY_PERCENT_DEFAULT = 0.0;
+    public static final String BKDL_EI_INJECTED_WRITE_DELAY_MS = "eiInjectedWriteDelayMs";
+    public static final int BKDL_EI_INJECTED_WRITE_DELAY_MS_DEFAULT = 0;
+    public static final String BKDL_EI_INJECT_READAHEAD_STALL = "eiInjectReadAheadStall";
+    public static final boolean BKDL_EI_INJECT_READAHEAD_STALL_DEFAULT = false;
+    public static final String BKDL_EI_INJECT_READAHEAD_DELAY = "eiInjectReadAheadDelay";
+    public static final boolean BKDL_EI_INJECT_READAHEAD_DELAY_DEFAULT = false;
+    public static final String BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS = "eiInjectMaxReadAheadDelayMs";
+    public static final int BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS_DEFAULT = 0;
+    public static final String BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT = "eiInjectReadAheadDelayPercent";
+    public static final int BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT_DEFAULT = 10;
+    public static final String BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES = "eiInjectReadAheadBrokenEntries";
+    public static final boolean BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES_DEFAULT = false;
+
+    // Whitelisted stream-level configuration settings.
+    private static final Set<String> streamSettings = Sets.newHashSet(
+        BKDL_READER_POSITION_GAP_DETECTION_ENABLED,
+        BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS,
+        BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS,
+        BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
+        BKDL_ENABLE_IMMEDIATE_FLUSH
+    );
+
+    /**
+     * Construct distributedlog configuration with default settings.
+     * It also loads the settings from system properties.
+     */
+    public DistributedLogConfiguration() {
+        super();
+        // add configuration for system properties
+        addConfiguration(new SystemConfiguration());
+    }
+
+    /**
+     * You can load configurations in precedence order. The first one takes
+     * precedence over any loaded later.
+     *
+     * @param confURL Configuration URL
+     */
+    public void loadConf(URL confURL) throws ConfigurationException {
+        Configuration loadedConf = new PropertiesConfiguration(confURL);
+        addConfiguration(loadedConf);
+    }
+
+    /**
+     * You can load configuration from other configuration
+     *
+     * @param baseConf Other Configuration
+     */
+    public void loadConf(DistributedLogConfiguration baseConf) {
+        addConfiguration(baseConf);
+    }
+
+    /**
+     * Load configuration from other configuration object
+     *
+     * @param otherConf Other configuration object
+     */
+    public void loadConf(Configuration otherConf) {
+        addConfiguration(otherConf);
+    }
+
+    /**
+     * Load whitelisted stream configuration from another configuration object
+     *
+     * @param streamConfiguration stream configuration overrides
+     */
+    public void loadStreamConf(Optional<DistributedLogConfiguration> streamConfiguration) {
+        if (!streamConfiguration.isPresent()) {
+            return;
+        }
+        ArrayList<Object> ignoredSettings = new ArrayList<Object>();
+        Iterator iterator = streamConfiguration.get().getKeys();
+        while (iterator.hasNext()) {
+            Object setting = iterator.next();
+            if (setting instanceof String && streamSettings.contains(setting)) {
+                String settingStr = (String) setting;
+                setProperty(settingStr, streamConfiguration.get().getProperty(settingStr));
+            } else {
+                ignoredSettings.add(setting);
+            }
+        }
+        if (LOG.isWarnEnabled() && !ignoredSettings.isEmpty()) {
+            LOG.warn("invalid stream configuration override(s): {}",
+                StringUtils.join(ignoredSettings, ";"));
+        }
+    }
+
+    //
+    // ZooKeeper Related Settings
+    //
+
+    /**
+     * Get all properties as a string.
+     */
+    public String getPropsAsString() {
+        Iterator iterator = getKeys();
+        StringBuilder builder = new StringBuilder();
+        boolean appendNewline = false;
+        while (iterator.hasNext()) {
+            Object key = iterator.next();
+            if (key instanceof String) {
+                if (appendNewline) {
+                    builder.append("\n");
+                }
+                Object value = getProperty((String)key);
+                builder.append(key).append("=").append(value);
+                appendNewline = true;
+            }
+        }
+        return builder.toString();
+    }
+
+    /**
+     * Get digest id used for ZK acl.
+     *
+     * @return zk acl id.
+     */
+    public String getZkAclId() {
+        return getString(BKDL_ZK_ACL_ID, BKDL_ZK_ACL_ID_DEFAULT);
+    }
+
+    /**
+     * Set digest id to use for ZK acl.
+     *
+     * @param zkAclId acl id.
+     * @return distributedlog configuration
+     * @see #getZkAclId()
+     */
+    public DistributedLogConfiguration setZkAclId(String zkAclId) {
+        setProperty(BKDL_ZK_ACL_ID, zkAclId);
+        return this;
+    }
+
+    /**
+     * Get ZK Session timeout in seconds.
+     * <p>
+     * This is the session timeout applied for zookeeper client used by distributedlog.
+     * Use {@link #getBKClientZKSessionTimeoutMilliSeconds()} for zookeeper client used
+     * by bookkeeper client.
+     *
+     * @return zookeeeper session timeout in seconds.
+     * @deprecated use {@link #getZKSessionTimeoutMilliseconds()}
+     */
+    public int getZKSessionTimeoutSeconds() {
+        return this.getInt(BKDL_ZK_SESSION_TIMEOUT_SECONDS, BKDL_ZK_SESSION_TIMEOUT_SECONDS_DEFAULT);
+    }
+
+    /**
+     * Get ZK Session timeout in milliseconds.
+     * <p>
+     * This is the session timeout applied for zookeeper client used by distributedlog.
+     * Use {@link #getBKClientZKSessionTimeoutMilliSeconds()} for zookeeper client used
+     * by bookkeeper client.
+     *
+     * @return zk session timeout in milliseconds.
+     */
+    public int getZKSessionTimeoutMilliseconds() {
+        return this.getInt(BKDL_ZK_SESSION_TIMEOUT_SECONDS, BKDL_ZK_SESSION_TIMEOUT_SECONDS_DEFAULT) * 1000;
+    }
+
+    /**
+     * Set ZK Session Timeout in seconds.
+     *
+     * @param zkSessionTimeoutSeconds session timeout in seconds.
+     * @return distributed log configuration
+     * @see #getZKSessionTimeoutMilliseconds()
+     */
+    public DistributedLogConfiguration setZKSessionTimeoutSeconds(int zkSessionTimeoutSeconds) {
+        setProperty(BKDL_ZK_SESSION_TIMEOUT_SECONDS, zkSessionTimeoutSeconds);
+        return this;
+    }
+
+    /**
+     * Get zookeeper access rate limit.
+     * <p>The rate limiter is basically a guava {@link com.google.common.util.concurrent.RateLimiter}.
+     * It is rate limiting the requests that sent by zookeeper client. If the value is non-positive,
+     * the rate limiting is disable. By default it is disable (value = 0).
+     *
+     * @return zookeeper access rate, by default it is 0.
+     */
+    public double getZKRequestRateLimit() {
+        return this.getDouble(BKDL_ZK_REQUEST_RATE_LIMIT, BKDL_ZK_REQUEST_RATE_LIMIT_DEFAULT);
+    }
+
+    /**
+     * Set zookeeper access rate limit (rps).
+     *
+     * @param requestRateLimit
+     *          zookeeper access rate limit
+     * @return distributedlog configuration
+     * @see #getZKRequestRateLimit()
+     */
+    public DistributedLogConfiguration setZKRequestRateLimit(double requestRateLimit) {
+        setProperty(BKDL_ZK_REQUEST_RATE_LIMIT, requestRateLimit);
+        return this;
+    }
+
+    /**
+     * Get num of retries per request for zookeeper client.
+     * <p>Retries only happen on retryable failures like session expired,
+     * session moved. for permanent failures, the request will fail immediately.
+     * The default value is 3.
+     *
+     * @return num of retries per request of zookeeper client.
+     */
+    public int getZKNumRetries() {
+        return this.getInt(BKDL_ZK_NUM_RETRIES, BKDL_ZK_NUM_RETRIES_DEFAULT);
+    }
+
+    /**
+     * Set num of retries per request for zookeeper client.
+     *
+     * @param zkNumRetries num of retries per request of zookeeper client.
+     * @return distributed log configuration
+     * @see #getZKNumRetries()
+     */
+    public DistributedLogConfiguration setZKNumRetries(int zkNumRetries) {
+        setProperty(BKDL_ZK_NUM_RETRIES, zkNumRetries);
+        return this;
+    }
+
+    /**
+     * Get the start backoff time of zookeeper operation retries, in milliseconds.
+     * <p>The retry time will increase in bound exponential way, and become flat
+     * after hit max backoff time ({@link #getZKRetryBackoffMaxMillis()}).
+     * The default start backoff time is 5000 milliseconds.
+     *
+     * @return start backoff time of zookeeper operation retries, in milliseconds.
+     * @see #getZKRetryBackoffMaxMillis()
+     */
+    public int getZKRetryBackoffStartMillis() {
+        return this.getInt(BKDL_ZK_RETRY_BACKOFF_START_MILLIS,
+                           BKDL_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT);
+    }
+
+    /**
+     * Set the start backoff time of zookeeper operation retries, in milliseconds.
+     *
+     * @param zkRetryBackoffStartMillis start backoff time of zookeeper operation retries,
+     *                                  in milliseconds.
+     * @return distributed log configuration
+     * @see #getZKRetryBackoffStartMillis()
+     */
+    public DistributedLogConfiguration setZKRetryBackoffStartMillis(int zkRetryBackoffStartMillis) {
+        setProperty(BKDL_ZK_RETRY_BACKOFF_START_MILLIS, zkRetryBackoffStartMillis);
+        return this;
+    }
+
+    /**
+     * Get the max backoff time of zookeeper operation retries, in milliseconds.
+     * <p>The retry time will increase in bound exponential way starting from
+     * {@link #getZKRetryBackoffStartMillis()}, and become flat after hit this max
+     * backoff time.
+     * The default max backoff time is 30000 milliseconds.
+     *
+     * @return max backoff time of zookeeper operation retries, in milliseconds.
+     * @see #getZKRetryBackoffStartMillis()
+     */
+    public int getZKRetryBackoffMaxMillis() {
+        return this.getInt(BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS,
+                           BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT);
+    }
+
+    /**
+     * Set the max backoff time of zookeeper operation retries, in milliseconds.
+     *
+     * @param zkRetryBackoffMaxMillis max backoff time of zookeeper operation retries,
+     *                                in milliseconds.
+     * @return distributed log configuration
+     * @see #getZKRetryBackoffMaxMillis()
+     */
+    public DistributedLogConfiguration setZKRetryBackoffMaxMillis(int zkRetryBackoffMaxMillis) {
+        setProperty(BKDL_ZK_RETRY_BACKOFF_MAX_MILLIS, zkRetryBackoffMaxMillis);
+        return this;
+    }
+
+    /**
+     * Get ZK client number of retry executor threads.
+     * By default it is 1.
+     *
+     * @return number of bookkeeper client worker threads.
+     */
+    public int getZKClientNumberRetryThreads() {
+        return this.getInt(BKDL_ZKCLIENT_NUM_RETRY_THREADS, BKDL_ZKCLIENT_NUM_RETRY_THREADS_DEFAULT);
+    }
+
+    /**
+     * Set ZK client number of retry executor threads.
+     *
+     * @param numThreads
+     *          number of retry executor threads.
+     * @return distributedlog configuration.
+     * @see #getZKClientNumberRetryThreads()
+     */
+    public DistributedLogConfiguration setZKClientNumberRetryThreads(int numThreads) {
+        setProperty(BKDL_ZKCLIENT_NUM_RETRY_THREADS, numThreads);
+        return this;
+    }
+
+    //
+    // BookKeeper ZooKeeper Client Settings
+    //
+
+    /**
+     * Get BK's zookeeper session timout in milliseconds.
+     * <p>
+     * This is the session timeout applied for zookeeper client used by bookkeeper client.
+     * Use {@link #getZKSessionTimeoutMilliseconds()} for zookeeper client used
+     * by distributedlog.
+     *
+     * @return Bk's zookeeper session timeout in milliseconds
+     */
+    public int getBKClientZKSessionTimeoutMilliSeconds() {
+        return this.getInt(BKDL_BKCLIENT_ZK_SESSION_TIMEOUT, BKDL_BKCLIENT_ZK_SESSION_TIMEOUT_DEFAULT) * 1000;
+    }
+
+    /**
+     * Set BK's zookeeper session timeout in seconds.
+     *
+     * @param sessionTimeout session timeout for the ZK Client used by BK Client, in seconds.
+     * @return distributed log configuration
+     * @see #getBKClientZKSessionTimeoutMilliSeconds()
+     */
+    public DistributedLogConfiguration setBKClientZKSessionTimeout(int sessionTimeout) {
+        setProperty(BKDL_BKCLIENT_ZK_SESSION_TIMEOUT, sessionTimeout);
+        return this;
+    }
+
+    /**
+     * Get zookeeper access rate limit for zookeeper client used in bookkeeper client.
+     * <p>The rate limiter is basically a guava {@link com.google.common.util.concurrent.RateLimiter}.
+     * It is rate limiting the requests that sent by zookeeper client. If the value is non-positive,
+     * the rate limiting is disable. By default it is disable (value = 0).
+     *
+     * @return zookeeper access rate limit for zookeeper client used in bookkeeper client.
+     * By default it is 0.
+     */
+    public double getBKClientZKRequestRateLimit() {
+        return this.getDouble(BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT,
+                BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT_DEFAULT);
+    }
+
+    /**
+     * Set zookeeper access rate limit for zookeeper client used in bookkeeper client.
+     *
+     * @param rateLimit
+     *          zookeeper access rate limit
+     * @return distributedlog configuration.
+     * @see #getBKClientZKRequestRateLimit()
+     */
+    public DistributedLogConfiguration setBKClientZKRequestRateLimit(double rateLimit) {
+        setProperty(BKDL_BKCLIENT_ZK_REQUEST_RATE_LIMIT, rateLimit);
+        return this;
+    }
+
+    /**
+     * Get num of retries for zookeeper client that used by bookkeeper client.
+     * <p>Retries only happen on retryable failures like session expired,
+     * session moved. for permanent failures, the request will fail immediately.
+     * The default value is 3. Setting it to zero or negative will retry infinitely.
+     *
+     * @return num of retries of zookeeper client used by bookkeeper client.
+     */
+    public int getBKClientZKNumRetries() {
+        int zkNumRetries = this.getInt(BKDL_BKCLIENT_ZK_NUM_RETRIES, BKDL_BKCLIENT_ZK_NUM_RETRIES_DEFAULT);
+        if (zkNumRetries <= 0) {
+            return Integer.MAX_VALUE;
+        }
+        return zkNumRetries;
+    }
+
+    /**
+     * Get the start backoff time of zookeeper operation retries, in milliseconds.
+     * <p>The retry time will increase in bound exponential way, and become flat
+     * after hit max backoff time ({@link #getBKClientZKRetryBackoffMaxMillis()}.
+     * The default start backoff time is 5000 milliseconds.
+     *
+     * @return start backoff time of zookeeper operation retries, in milliseconds.
+     * @see #getBKClientZKRetryBackoffMaxMillis()
+     */
+    public int getBKClientZKRetryBackoffStartMillis() {
+        return this.getInt(BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS,
+                           BKDL_BKCLIENT_ZK_RETRY_BACKOFF_START_MILLIS_DEFAULT);
+    }
+
+    /**
+     * Get the max backoff time of zookeeper operation retries, in milliseconds.
+     * <p>The retry time will increase in bound exponential way starting from
+     * {@link #getBKClientZKRetryBackoffStartMillis()}, and become flat after
+     * hit this max backoff time.
+     * The default max backoff time is 30000 milliseconds.
+     *
+     * @return max backoff time of zookeeper operation retries, in milliseconds.
+     * @see #getBKClientZKRetryBackoffStartMillis()
+     */
+    public int getBKClientZKRetryBackoffMaxMillis() {
+        return this.getInt(BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS,
+                BKDL_BKCLIENT_ZK_RETRY_BACKOFF_MAX_MILLIS_DEFAULT);
+    }
+
+    //
+    // BookKeeper Ensemble Placement Settings
+    //
+
+    /**
+     * Get ensemble size of each log segment (ledger) will use.
+     * By default it is 3.
+     * <p>
+     * A log segment's data is stored in an ensemble of bookies in
+     * a stripping way. Each entry will be added in a <code>write-quorum</code>
+     * size of bookies. The add operation will complete once it receives
+     * responses from a <code>ack-quorum</code> size of bookies. The stripping
+     * is done in a round-robin way in bookkeeper.
+     * <p>
+     * For example, we configure the ensemble-size to 5, write-quorum-size to 3,
+     * and ack-quorum-size to 2. The data will be stored in following stripping way.
+     * <pre>
+     * | entry id | bk1 | bk2 | bk3 | bk4 | bk5 |
+     * |     0    |  x  |  x  |  x  |     |     |
+     * |     1    |     |  x  |  x  |  x  |     |
+     * |     2    |     |     |  x  |  x  |  x  |
+     * |     3    |  x  |     |     |  x  |  x  |
+     * |     4    |  x  |  x  |     |     |  x  |
+     * |     5    |  x  |  x  |  x  |     |     |
+     * </pre>
+     * <p>
+     * We don't recommend stripping within a log segment to increase bandwidth.
+     * We'd recommend to strip by `partition` in higher level of distributedlog
+     * to increase performance. so typically the ensemble size will set to be
+     * the same value as write quorum size.
+     *
+     * @return ensemble size
+     * @see #getWriteQuorumSize()
+     * @see #getAckQuorumSize()
+     */
+    public int getEnsembleSize() {
+        return this.getInt(BKDL_BOOKKEEPER_ENSEMBLE_SIZE,
+                getInt(BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD,
+                        BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT));
+    }
+
+    /**
+     * Set ensemble size of each log segment (ledger) will use.
+     *
+     * @param ensembleSize ensemble size.
+     * @return distributed log configuration
+     * @see #getEnsembleSize()
+     */
+    public DistributedLogConfiguration setEnsembleSize(int ensembleSize) {
+        setProperty(BKDL_BOOKKEEPER_ENSEMBLE_SIZE, ensembleSize);
+        return this;
+    }
+
+    /**
+     * Get write quorum size of each log segment (ledger) will use.
+     * By default it is 3.
+     *
+     * @return write quorum size
+     * @see #getEnsembleSize()
+     */
+    public int getWriteQuorumSize() {
+        return this.getInt(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
+                getInt(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
+                        BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT));
+    }
+
+    /**
+     * Set write quorum size of each log segment (ledger) will use.
+     *
+     * @param quorumSize
+     *          quorum size.
+     * @return distributedlog configuration.
+     * @see #getWriteQuorumSize()
+     */
+    public DistributedLogConfiguration setWriteQuorumSize(int quorumSize) {
+        setProperty(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE, quorumSize);
+        return this;
+    }
+
+    /**
+     * Get ack quorum size of each log segment (ledger) will use.
+     * By default it is 2.
+     *
+     * @return ack quorum size
+     * @see #getEnsembleSize()
+     */
+    public int getAckQuorumSize() {
+        return this.getInt(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
+                getInt(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
+                        BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT));
+    }
+
+    /**
+     * Set ack quorum size of each log segment (ledger) will use.
+     *
+     * @param quorumSize
+     *          quorum size.
+     * @return distributedlog configuration.
+     * @see #getAckQuorumSize()
+     */
+    public DistributedLogConfiguration setAckQuorumSize(int quorumSize) {
+        setProperty(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE, quorumSize);
+        return this;
+    }
+
+    /**
+     * Get the quorum config for each log segment (ledger).
+     *
+     * @return quorum config that used by log segments
+     * @see #getEnsembleSize()
+     * @see #getWriteQuorumSize()
+     * @see #getAckQuorumSize()
+     */
+    public QuorumConfig getQuorumConfig() {
+        return new QuorumConfig(
+                getEnsembleSize(),
+                getWriteQuorumSize(),
+                getAckQuorumSize());
+    }
+
+    /**
+     * Get if row aware ensemble placement is enabled.
+     * <p>If enabled, {@link DNSResolverForRows} will be used for dns resolution
+     * rather than {@link DNSResolverForRacks}, if no other dns resolver set via
+     * {@link #setEnsemblePlacementDnsResolverClass(Class)}.
+     * By default it is disable.
+     *
+     * @return true if row aware ensemble placement is enabled, otherwise false.
+     * @see #getEnsemblePlacementDnsResolverClass()
+     */
+    public boolean getRowAwareEnsemblePlacementEnabled() {
+        return getBoolean(BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT,
+                getBoolean(BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_OLD,
+                        BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT_DEFAULT));
+    }
+
+    /**
+     * Set if we should enable row aware ensemble placement.
+     *
+     * @param enableRowAwareEnsemblePlacement
+     *          enableRowAwareEnsemblePlacement
+     * @return distributedlog configuration.
+     * @see #getRowAwareEnsemblePlacementEnabled()
+     */
+    public DistributedLogConfiguration setRowAwareEnsemblePlacementEnabled(boolean enableRowAwareEnsemblePlacement) {
+        setProperty(BKDL_ROW_AWARE_ENSEMBLE_PLACEMENT, enableRowAwareEnsemblePlacement);
+        return this;
+    }
+
+    /**
+     * Get the DNS resolver class for bookkeeper ensemble placement.
+     * <p>By default, {@link DNSResolverForRacks} will be used if
+     * {@link #getRowAwareEnsemblePlacementEnabled()} is disabled and
+     * {@link DNSResolverForRows} will be used if {@link #getRowAwareEnsemblePlacementEnabled()}
+     * is enabled.
+     *
+     * @return dns resolver class for bookkeeper ensemble placement.
+     * @throws ConfigurationException
+     * @see #getRowAwareEnsemblePlacementEnabled()
+     */
+    public Class<? extends DNSToSwitchMapping> getEnsemblePlacementDnsResolverClass()
+            throws ConfigurationException {
+        Class<? extends DNSToSwitchMapping> defaultResolverCls;
+        if (getRowAwareEnsemblePlacementEnabled()) {
+            defaultResolverCls = DNSResolverForRows.class;
+        } else {
+            defaultResolverCls = DNSResolverForRacks.class;
+        }
+        return ReflectionUtils.getClass(this, BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS,
+                defaultResolverCls, DNSToSwitchMapping.class, defaultLoader);
+    }
+
+    /**
+     * Set the DNS resolver class for bookkeeper ensemble placement.
+     *
+     * @param dnsResolverClass
+     *          dns resolver class for bookkeeper ensemble placement.
+     * @return distributedlog configuration
+     * @see #getEnsemblePlacementDnsResolverClass()
+     */
+    public DistributedLogConfiguration setEnsemblePlacementDnsResolverClass(
+            Class<? extends DNSToSwitchMapping> dnsResolverClass) {
+        setProperty(BKDL_ENSEMBLE_PLACEMENT_DNS_RESOLVER_CLASS, dnsResolverClass.getName());
+        return this;
+    }
+
+    /**
+     * Get mapping used to override the region mapping derived by the default resolver.
+     * <p>It is a string of pairs of host-region mappings (host:region) separated by semicolon.
+     * By default it is empty string.
+     *
+     * @return dns resolver overrides.
+     * @see #getEnsemblePlacementDnsResolverClass()
+     * @see DNSResolverForRacks
+     * @see DNSResolverForRows
+     */
+    public String getBkDNSResolverOverrides() {
+        return getString(BKDL_BK_DNS_RESOLVER_OVERRIDES, BKDL_BK_DNS_RESOLVER_OVERRIDES_DEFAULT);
+    }
+
+    /**
+     * Set mapping used to override the region mapping derived by the default resolver
+     * <p>It is a string of pairs of host-region mappings (host:region) separated by semicolon.
+     * By default it is empty string.
+     *
+     * @param overrides
+     *          dns resolver overrides
+     * @return dl configuration.
+     * @see #getBkDNSResolverOverrides()
+     */
+    public DistributedLogConfiguration setBkDNSResolverOverrides(String overrides) {
+        setProperty(BKDL_BK_DNS_RESOLVER_OVERRIDES, overrides);
+        return this;
+    }
+
+    //
+    // BookKeeper General Settings
+    //
+
+    /**
+     * Set password used by bookkeeper client for digestion.
+     * <p>
+     * NOTE: not recommend to change. will be derepcated in future.
+     *
+     * @param bkDigestPW BK password digest
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setBKDigestPW(String bkDigestPW) {
+        setProperty(BKDL_BOOKKEEPER_DIGEST_PW, bkDigestPW);
+        return this;
+    }
+
+    /**
+     * Get password used by bookkeeper client for digestion.
+     * <p>
+     * NOTE: not recommend to change. will be deprecated in future.
+     *
+     * @return password used by bookkeeper client for digestion
+     * @see #setBKDigestPW(String)
+     */
+    public String getBKDigestPW() {
+        return getString(BKDL_BOOKKEEPER_DIGEST_PW, BKDL_BOOKKEEPER_DIGEST_PW_DEFAULT);
+    }
+
+    /**
+     * Get BK client number of i/o threads used by Netty.
+     * The default value equals DL's number worker threads.
+     *
+     * @return number of bookkeeper netty i/o threads.
+     * @see #getNumWorkerThreads()
+     */
+    public int getBKClientNumberIOThreads() {
+        return this.getInt(BKDL_BKCLIENT_NUM_IO_THREADS, getNumWorkerThreads());
+    }
+
+    /**
+     * Set BK client number of i/o threads used by netty.
+     *
+     * @param numThreads
+     *          number io threads.
+     * @return distributedlog configuration.
+     * @see #getBKClientNumberIOThreads()
+     */
+    public DistributedLogConfiguration setBKClientNumberIOThreads(int numThreads) {
+        setProperty(BKDL_BKCLIENT_NUM_IO_THREADS, numThreads);
+        return this;
+    }
+
+    /**
+     * Get the tick duration in milliseconds that used for timeout timer in bookkeeper client.
+     * By default it is 100.
+     *
+     * @return tick duration in milliseconds
+     * @see org.jboss.netty.util.HashedWheelTimer
+     */
+    public long getTimeoutTimerTickDurationMs() {
+        return getLong(BKDL_TIMEOUT_TIMER_TICK_DURATION_MS, BKDL_TIMEOUT_TIMER_TICK_DURATION_MS_DEFAULT);
+    }
+
+    /**
+     * Set the tick duration in milliseconds that used for timeout timer in bookkeeper client.
+     *
+     * @param tickDuration
+     *          tick duration in milliseconds.
+     * @return distributed log configuration.
+     * @see #getTimeoutTimerTickDurationMs()
+     */
+    public DistributedLogConfiguration setTimeoutTimerTickDurationMs(long tickDuration) {
+        setProperty(BKDL_TIMEOUT_TIMER_TICK_DURATION_MS, tickDuration);
+        return this;
+    }
+
+    /**
+     * Get number of ticks that used for timeout timer in bookkeeper client.
+     * By default is 1024.
+     *
+     * @return number of ticks that used for timeout timer.
+     * @see org.jboss.netty.util.HashedWheelTimer
+     */
+    public int getTimeoutTimerNumTicks() {
+        return getInt(BKDL_TIMEOUT_TIMER_NUM_TICKS, BKDL_TIMEOUT_TIMER_NUM_TICKS_DEFAULT);
+    }
+
+    /**
+     * Set number of ticks that used for timeout timer in bookkeeper client.
+     *
+     * @param numTicks
+     *          number of ticks that used for timeout timer.
+     * @return distributed log configuration.
+     * @see #getTimeoutTimerNumTicks()
+     */
+    public DistributedLogConfiguration setTimeoutTimerNumTicks(int numTicks) {
+        setProperty(BKDL_TIMEOUT_TIMER_NUM_TICKS, numTicks);
+        return this;
+    }
+
+    //
+    // Deprecated BookKeeper Settings
+    //
+
+    /**
+     * Get BK client read timeout in seconds.
+     * <p>
+     * Please use {@link ClientConfiguration#getReadEntryTimeout()}
+     * instead of this setting.
+     *
+     * @return read timeout in seconds
+     * @deprecated
+     * @see ClientConfiguration#getReadEntryTimeout()
+     */
+    public int getBKClientReadTimeout() {
+        return this.getInt(BKDL_BKCLIENT_READ_TIMEOUT,
+                BKDL_BKCLIENT_READ_TIMEOUT_DEFAULT);
+    }
+
+    /**
+     * Set BK client read timeout in seconds.
+     *
+     * @param readTimeout read timeout in seconds.
+     * @return distributed log configuration
+     * @deprecated
+     * @see #getBKClientReadTimeout()
+     */
+    public DistributedLogConfiguration setBKClientReadTimeout(int readTimeout) {
+        setProperty(BKDL_BKCLIENT_READ_TIMEOUT, readTimeout);
+        return this;
+    }
+
+    /**
+     * Get BK client write timeout in seconds.
+     * <p>
+     * Please use {@link ClientConfiguration#getAddEntryTimeout()}
+     * instead of this setting.
+     *
+     * @return write timeout in seconds.
+     * @deprecated
+     * @see ClientConfiguration#getAddEntryTimeout()
+     */
+    public int getBKClientWriteTimeout() {
+        return this.getInt(BKDL_BKCLIENT_WRITE_TIMEOUT, BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT);
+    }
+
+    /**
+     * Set BK client write timeout in seconds
+     *
+     * @param writeTimeout write timeout in seconds.
+     * @return distributed log configuration
+     * @deprecated
+     * @see #getBKClientWriteTimeout()
+     */
+    public DistributedLogConfiguration setBKClientWriteTimeout(int writeTimeout) {
+        setProperty(BKDL_BKCLIENT_WRITE_TIMEOUT, writeTimeout);
+        return this;
+    }
+
+    /**
+     * Get BK client number of worker threads.
+     * <p>
+     * Please use {@link ClientConfiguration#getNumWorkerThreads()}
+     * instead of this setting.
+     *
+     * @return number of bookkeeper client worker threads.
+     * @deprecated
+     * @see ClientConfiguration#getNumWorkerThreads()
+     */
+    public int getBKClientNumberWorkerThreads() {
+        return this.getInt(BKDL_BKCLIENT_NUM_WORKER_THREADS, BKDL_BKCLEINT_NUM_WORKER_THREADS_DEFAULT);
+    }
+
+    /**
+     * Set BK client number of worker threads.
+     *
+     * @param numThreads
+     *          number worker threads.
+     * @return distributedlog configuration.
+     * @deprecated
+     * @see #getBKClientNumberWorkerThreads()
+     */
+    public DistributedLogConfiguration setBKClientNumberWorkerThreads(int numThreads) {
+        setProperty(BKDL_BKCLIENT_NUM_WORKER_THREADS, numThreads);
+        return this;
+    }
+
+    //
+    // DL Executor Settings
+    //
+
+    /**
+     * Get the number of worker threads used by distributedlog namespace.
+     * By default it is the number of available processors.
+     *
+     * @return number of worker threads used by distributedlog namespace.
+     */
+    public int getNumWorkerThreads() {
+        return getInt(BKDL_NUM_WORKER_THREADS, Runtime.getRuntime().availableProcessors());
+    }
+
+    /**
+     * Set the number of worker threads used by distributedlog namespace.
+     *
+     * @param numWorkerThreads
+     *          number of worker threads used by distributedlog namespace.
+     * @return configuration
+     * @see #getNumWorkerThreads()
+     */
+    public DistributedLogConfiguration setNumWorkerThreads(int numWorkerThreads) {
+        setProperty(BKDL_NUM_WORKER_THREADS, numWorkerThreads);
+        return this;
+    }
+
+    /**
+     * Get the number of dedicated readahead worker threads used by distributedlog namespace.
+     * <p>If this value is non-positive, it would share the normal executor (see {@link #getNumWorkerThreads()}
+     * for readahead. otherwise, it would use a dedicated executor for readhead. By default,
+     * it is 0.
+     *
+     * @return number of dedicated readahead worker threads.
+     * @see #getNumWorkerThreads()
+     */
+    @Deprecated
+    public int getNumReadAheadWorkerThreads() {
+        return getInt(BKDL_NUM_READAHEAD_WORKER_THREADS, 0);
+    }
+
+    /**
+     * Set the number of dedicated readahead worker threads used by distributedlog namespace.
+     *
+     * @param numWorkerThreads
+     *          number of dedicated readahead worker threads.
+     * @return configuration
+     * @see #getNumReadAheadWorkerThreads()
+     */
+    @Deprecated
+    public DistributedLogConfiguration setNumReadAheadWorkerThreads(int numWorkerThreads) {
+        setProperty(BKDL_NUM_READAHEAD_WORKER_THREADS, numWorkerThreads);
+        return this;
+    }
+
+    /**
+     * Get the number of lock state threads used by distributedlog namespace.
+     * By default it is 1.
+     *
+     * @return number of lock state threads used by distributedlog namespace.
+     */
+    public int getNumLockStateThreads() {
+        return getInt(BKDL_NUM_LOCKSTATE_THREADS, 1);
+    }
+
+    /**
+     * Set the number of lock state threads used by distributedlog manager factory.
+     *
+     * @param numLockStateThreads
+     *          number of lock state threads used by distributedlog manager factory.
+     * @return configuration
+     * @see #getNumLockStateThreads()
+     */
+    public DistributedLogConfiguration setNumLockStateThreads(int numLockStateThreads) {
+        setProperty(BKDL_NUM_LOCKSTATE_THREADS, numLockStateThreads);
+        return this;
+    }
+
+    /**
+     * Get the number of resource release threads used by distributedlog namespace.
+     * By default it is 0 - the thread will be created dynamically by a executor service.
+     * The executor service is an unbounded pool. Application can use `total_tasks - completed_tasks`
+     * on monitoring the number of threads that are used for releasing resources.
+     * <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() {
+        return getInt(BKDL_NUM_RESOURCE_RELEASE_THREADS, 0);
+    }
+
+    /**
+     * Set the number of resource release threads used by distributedlog manager factory.
+     *
+     * @param numResourceReleaseThreads
+     *          number of resource release threads used by distributedlog manager factory.
+     * @return configuration
+     * @see #getNumResourceReleaseThreads()
+     */
+    public DistributedLogConfiguration setNumResourceReleaseThreads(int numResourceReleaseThreads) {
+        setProperty(BKDL_NUM_RESOURCE_RELEASE_THREADS, numResourceReleaseThreads);
+        return this;
+    }
+
+    /**
+     * Get timeout for shutting down schedulers in dl manager, in milliseconds.
+     * By default, it is 5 seconds.
+     *
+     * @return timeout for shutting down schedulers in dl manager, in miliseconds.
+     */
+    public int getSchedulerShutdownTimeoutMs() {
+        return getInt(BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS, BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS_DEFAULT);
+    }
+
+    /**
+     * Set timeout for shutting down schedulers in dl manager, in milliseconds.
+     *
+     * @param timeoutMs
+     *         timeout for shutting down schedulers in dl manager, in milliseconds.
+     * @return dl configuration.
+     * @see #getSchedulerShutdownTimeoutMs()
+     */
+    public DistributedLogConfiguration setSchedulerShutdownTimeoutMs(int timeoutMs) {
+        setProperty(BKDL_SCHEDULER_SHUTDOWN_TIMEOUT_MS, timeoutMs);
+        return this;
+    }
+
+    /**
+     * Whether to use daemon thread for DL threads.
+     * By default it is false.
+     *
+     * @return true if use daemon threads, otherwise false.
+     */
+    public boolean getUseDaemonThread() {
+        return getBoolean(BKDL_USE_DAEMON_THREAD, BKDL_USE_DAEMON_THREAD_DEFAULT);
+    }
+
+    /**
+     * Set whether to use daemon thread for DL threads.
+     *
+     * @param daemon
+     *          whether to use daemon thread for DL threads.
+     * @return distributedlog configuration
+     * @see #getUseDaemonThread()
+     */
+    public DistributedLogConfiguration setUseDaemonThread(boolean daemon) {
+        setProperty(BKDL_USE_DAEMON_THREAD, daemon);
+        return this;
+    }
+
+    //
+    // Metadata Settings
+    //
+
+    /**
+     * Get DL ledger metadata output layout version.
+     *
+     * @return layout version
+     * @see org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion
+     */
+    public int getDLLedgerMetadataLayoutVersion() {
+        return this.getInt(BKDL_LEDGER_METADATA_LAYOUT_VERSION,
+                getInt(BKDL_LEDGER_METADATA_LAYOUT_VERSION_OLD,
+                        BKDL_LEDGER_METADATA_LAYOUT_VERSION_DEFAULT));
+    }
+
+    /**
+     * Set DL ledger metadata output layout version.
+     *
+     * @param layoutVersion layout version
+     * @return distributed log configuration
+     * @throws IllegalArgumentException if setting an unknown layout version.
+     * @see #getDLLedgerMetadataLayoutVersion()
+     */
+    public DistributedLogConfiguration setDLLedgerMetadataLayoutVersion(int layoutVersion)
+            throws IllegalArgumentException {
+        if ((layoutVersion <= 0) ||
+            (layoutVersion > LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION)) {
+            // Incorrect version specified
+            throw new IllegalArgumentException("Incorrect value for ledger metadata layout version");
+        }
+        setProperty(BKDL_LEDGER_METADATA_LAYOUT_VERSION, layoutVersion);
+        return this;
+    }
+
+    /**
+     * Get the setting for whether we should enforce the min ledger metadata version check.
+     * By default it is false.
+     *
+     * @return whether we should enforce the min ledger metadata version check
+     * @see org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion
+     */
+    public boolean getDLLedgerMetadataSkipMinVersionCheck() {
+        return this.getBoolean(BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK,
+                BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK_DEFAULT);
+    }
+
+    /**
+     * Set if we should skip the enforcement of min ledger metadata version.
+     * <p>NOTE: please be aware the side effects of skipping min ledger metadata
+     * version checking.
+     *
+     * @param skipMinVersionCheck whether we should enforce the min ledger metadata version check
+     * @return distributed log configuration
+     * @see #getDLLedgerMetadataSkipMinVersionCheck()
+     */
+    public DistributedLogConfiguration setDLLedgerMetadataSkipMinVersionCheck(boolean skipMinVersionCheck) throws IllegalArgumentException {
+        setProperty(BKDL_LEDGER_METADATA_SKIP_MIN_VERSION_CHECK, skipMinVersionCheck);
+        return this;
+    }
+
+    /**
+     * Get the value at which ledger sequence number should start for streams that are being
+     * upgraded and did not have ledger sequence number to start with or for newly created
+     * streams. By default, it is 1.
+     * <p>In most of the cases this value should not be changed. It is useful for backfilling
+     * in the case of migrating log segments whose metadata don't have log segment sequence number.
+     *
+     * @return first ledger sequence number
+     */
+    public long getFirstLogSegmentSequenceNumber() {
+        return this.getLong(BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER,
+                getLong(BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_OLD,
+                        BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER_DEFAULT));
+    }
+
+    /**
+     * Set the value at which ledger sequence number should start for streams that are being
+     * upgraded and did not have ledger sequence number to start with or for newly created
+     * streams
+     *
+     * @param firstLogSegmentSequenceNumber first ledger sequence number
+     * @return distributed log configuration
+     * @see #getFirstLogSegmentSequenceNumber()
+     */
+    public DistributedLogConfiguration setFirstLogSegmentSequenceNumber(long firstLogSegmentSequenceNumber)
+            throws IllegalArgumentException {
+        if (firstLogSegmentSequenceNumber <= 0) {
+            // Incorrect ledger sequence number specified
+            throw new IllegalArgumentException("Incorrect value for ledger sequence number");
+        }
+        setProperty(BKDL_FIRST_LOGSEGMENT_SEQUENCE_NUMBER, firstLogSegmentSequenceNumber);
+        return this;
+    }
+
+    /**
+     * Whether log segment sequence number validation is enabled?
+     *
+     * @return true if the log segment sequence number validation is enabled, otherwise false.
+     */
+    public boolean isLogSegmentSequenceNumberValidationEnabled() {
+        return this.getBoolean(BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED,
+                BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED_DEFAULT);
+    }
+
+    /**
+     * Whether log segment sequence number validation is enabled?
+     *
+     * @return true if the log segment sequence number validation is enabled, otherwise false.
+     */
+    public DistributedLogConfiguration setLogSegmentSequenceNumberValidationEnabled(boolean enabled) {
+        setProperty(BKDL_LOGSEGMENT_SEQUENCE_NUMBER_VALIDATION_ENABLED, enabled);
+        return this;
+    }
+
+    /**
+     * Whether we should publish record counts in the log records and metadata.
+     * <p>By default it is true. This is a legacy setting for log segment version 1. It
+     * should be considered removed.
+     *
+     * @return if record counts should be persisted
+     */
+    public boolean getEnableRecordCounts() {
+        return getBoolean(BKDL_ENABLE_RECORD_COUNTS, BKDL_ENABLE_RECORD_COUNTS_DEFAULT);
+    }
+
+    /**
+     * Set if we should publish record counts in the log records and metadata.
+     *
+     * @param enableRecordCounts enable record counts
+     * @return distributed log configuration
+     * @see #getEnableRecordCounts()
+     */
+    public DistributedLogConfiguration setEnableRecordCounts(boolean enableRecordCounts) {
+        setProperty(BKDL_ENABLE_RECORD_COUNTS, enableRecordCounts);
+        return this;
+    }
+
+    /**
+     * Whether sanity check txn id on starting log segments.
+     * <p>If it is enabled, DL writer would throw
+     * {@link org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException}
+     * when it received a smaller transaction id than current maximum transaction id.
+     *
+     * @return true if should check txn id with max txn id, otherwise false.
+     */
+    @Deprecated
+    public boolean getSanityCheckTxnID() {
+        return getBoolean(BKDL_MAXID_SANITYCHECK, BKDL_MAXID_SANITYCHECK_DEFAULT);
+    }
+
+    /**
+     * Enable/Disable sanity check txn id.
+     *
+     * @param enabled
+     *          enable/disable sanity check txn id.
+     * @return configuration.
+     * @see #getSanityCheckTxnID()
+     */
+    @Deprecated
+    public DistributedLogConfiguration setSanityCheckTxnID(boolean enabled) {
+        setProperty(BKDL_MAXID_SANITYCHECK, enabled);
+        return this;
+    }
+
+    /**
+     * Whether encode region id in log segment metadata.
+     * <p>In global DL use case, encoding region id in log segment medata would
+     * help understanding what region that a log segment is created. The region
+     * id field in log segment metadata would help for moniotring and troubleshooting.
+     *
+     * @return whether to encode region id in log segment metadata.
+     */
+    public boolean getEncodeRegionIDInLogSegmentMetadata() {
+        return getBoolean(BKDL_ENCODE_REGION_ID_IN_VERSION, BKDL_ENCODE_REGION_ID_IN_VERSION_DEFAULT);
+    }
+
+    /**
+     * Enable/Disable encoding region id in log segment metadata.
+     *
+     * @param enabled
+     *          flag to enable/disable encoding region id in log segment metadata.
+     * @return configuration instance.
+     * @see #getEncodeRegionIDInLogSegmentMetadata()
+     */
+    public DistributedLogConfiguration setEncodeRegionIDInLogSegmentMetadata(boolean enabled) {
+        setProperty(BKDL_ENCODE_REGION_ID_IN_VERSION, enabled);
+        return this;
+    }
+
+    /**
+     * Get log segment name version.
+     * <p>
+     * <ul>
+     * <li>version 0: inprogress_(start_txid) |
+     * logrecs_(start_txid)_(end_txid)</li>
+     * <li>version 1: inprogress_(logsegment_sequence_number) |
+     * logrecs_(logsegment_sequence_number)</li>
+     * </ul>
+     * By default it is 1.
+     *
+     * @return log segment name verison.
+     */
+    public int getLogSegmentNameVersion() {
+        return getInt(BKDL_LOGSEGMENT_NAME_VERSION, BKDL_LOGSEGMENT_NAME_VERSION_DEFAULT);
+    }
+
+    /**
+     * Set log segment name version.
+     *
+     * @param version
+     *          log segment name version.
+     * @return configuration object.
+     * @see #getLogSegmentNameVersion()
+     */
+    public DistributedLogConfiguration setLogSegmentNameVersion(int version) {
+        setProperty(BKDL_LOGSEGMENT_NAME_VERSION, version);
+        return this;
+    }
+
+    /**
+     * Get name of the unpartitioned stream.
+     * <p>It is a legacy setting. consider removing it in future.
+     *
+     * @return unpartitioned stream
+     */
+    public String getUnpartitionedStreamName() {
+        return getString(BKDL_UNPARTITIONED_STREAM_NAME, BKDL_UNPARTITIONED_STREAM_NAME_DEFAULT);
+    }
+
+    /**
+     * Set name of the unpartitioned stream
+     *
+     * @param streamName name of the unpartitioned stream
+     * @return distributedlog configuration
+     * @see #getUnpartitionedStreamName()
+     */
+    public DistributedLogConfiguration setUnpartitionedStreamName(String streamName) {
+        setProperty(BKDL_UNPARTITIONED_STREAM_NAME, streamName);
+        return this;
+    }
+
+    //
+    // LogSegment Cache Settings
+    //
+
+    /**
+     * Get the log segment cache entry TTL in milliseconds.
+     *
+     * @return log segment cache ttl in milliseconds.
+     */
+    public long getLogSegmentCacheTTLMs() {
+        return getLong(BKDL_LOGSEGMENT_CACHE_TTL_MS, BKDL_LOGSEGMENT_CACHE_MAX_SIZE_DEFAULT);
+    }
+
+    /**
+     * Set the log segment cache entry TTL in milliseconds.
+     *
+     * @param ttlMs TTL in milliseconds
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setLogSegmentCacheTTLMs(long ttlMs) {
+        setProperty(BKDL_LOGSEGMENT_CACHE_TTL_MS, ttlMs);
+        return this;
+    }
+
+    /**
+     * Get the maximum size of the log segment cache.
+     *
+     * @return maximum size of the log segment cache.
+     */
+    public long getLogSegmentCacheMaxSize() {
+        return getLong(BKDL_LOGSEGMENT_CACHE_MAX_SIZE, BKDL_LOGSEGMENT_CACHE_MAX_SIZE_DEFAULT);
+    }
+
+    /**
+     * Set the maximum size of the log segment cache.
+     *
+     * @param maxSize maximum size of the log segment cache.
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setLogSegmentCacheMaxSize(long maxSize) {
+        setProperty(BKDL_LOGSEGMENT_CACHE_MAX_SIZE, maxSize);
+        return this;
+    }
+
+    /**
+     * Is log segment cache enabled?
+     *
+     * @return true if log segment cache is enabled; otherwise false
+     */
+    public boolean isLogSegmentCacheEnabled() {
+        return getBoolean(BKDL_LOGSEGMENT_CACHE_ENABLED, BKDL_LOGSEGMENT_CACHE_ENABLED_DEFAULT);
+    }
+
+    /**
+     * Enable/disable log segment cache.
+     *
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setLogSegmentCacheEnabled(boolean enabled) {
+        setProperty(BKDL_LOGSEGMENT_CACHE_ENABLED, enabled);
+        return this;
+    }
+
+    //
+    // DL Writer General Settings
+    //
+
+    /**
+     * Whether to create stream if not exists. By default it is true.
+     *
+     * @return true if it is abled to create stream if not exists.
+     */
+    public boolean getCreateStreamIfNotExists() {
+        return getBoolean(BKDL_CREATE_STREAM_IF_NOT_EXISTS,
+                BKDL_CREATE_STREAM_IF_NOT_EXISTS_DEFAULT);
+    }
+
+    /**
+     * Enable/Disable creating stream if not exists.
+     *
+     * @param enabled
+     *          enable/disable sanity check txn id.
+     * @return distributed log configuration.
+     * @see #getCreateStreamIfNotExists()
+     */
+    public DistributedLogConfiguration setCreateStreamIfNotExists(boolean enabled) {
+        setProperty(BKDL_CREATE_STREAM_IF_NOT_EXISTS, enabled);
+        return this;
+    }
+
+    /**
+     * Get Log Flush timeout in seconds.
+     * <p>This is a setting used by DL writer on flushing data. It is typically used
+     * by synchronous writer and log segment writer. By default it is 30 seconds.
+     *
+     * @return log flush timeout in seconds.
+     */
+    // @Deprecated
+    public int getLogFlushTimeoutSeconds() {
+        return this.getInt(BKDL_LOG_FLUSH_TIMEOUT, BKDL_LOG_FLUSH_TIMEOUT_DEFAULT);
+    }
+
+    /**
+     * Set Log Flush Timeout in seconds.
+     *
+     * @param logFlushTimeoutSeconds log flush timeout.
+     * @return distributed log configuration
+     * @see #getLogFlushTimeoutSeconds()
+     */
+    public DistributedLogConfiguration setLogFlushTimeoutSeconds(int logFlushTimeoutSeconds) {
+        setProperty(BKDL_LOG_FLUSH_TIMEOUT, logFlushTimeoutSeconds);
+        return this;
+    }
+
+    /**
+     * The compression type to use while sending data to bookkeeper.
+     *
+     * @return compression type to use
+     * @see org.apache.distributedlog.io.CompressionCodec
+     */
+    public String getCompressionType() {
+        return getString(BKDL_COMPRESSION_TYPE, BKDL_COMPRESSION_TYPE_DEFAULT);
+    }
+
+    /**
+     * Set the compression type to use while sending data to bookkeeper.
+     *
+     * @param compressionType compression type
+     * @return distributedlog configuration
+     * @see #getCompressionType()
+     */
+    public DistributedLogConfiguration setCompressionType(String compressionType) {
+        Preconditions.checkArgument(null != compressionType && !compressionType.isEmpty());
+        setProperty(BKDL_COMPRESSION_TYPE, compressionType);
+        return this;
+    }
+
+    /**
+     * Whether to fail immediately if the stream is not ready rather than queueing the request.
+     * <p>If it is enabled, it would fail the write request immediately if the stream isn't ready.
+     * Consider turning it on for the use cases that could retry writing to other streams
+     * (aka non-strict ordering guarantee). It would result fast failure hence the client would
+     * retry immediately.
+     *
+     * @return true if should fail fast. otherwise, false.
+     */
+    public boolean getFailFastOnStreamNotReady() {
+        return getBoolean(BKDL_FAILFAST_ON_STREAM_NOT_READY,
+                BKDL_FAILFAST_ON_STREAM_NOT_READY_DEFAULT);
+    }
+
+    /**
+     * Set the failfast on stream not ready flag.
+     *
+     * @param failFastOnStreamNotReady
+     *        set failfast flag
+     * @return dl configuration.
+     * @see #getFailFastOnStreamNotReady()
+     */
+    public DistributedLogConfiguration setFailFastOnStreamNotReady(boolean failFastOnStreamNotReady) {
+        setProperty(BKDL_FAILFAST_ON_STREAM_NOT_READY, failFastOnStreamNotReady);
+        return this;
+    }
+
+    /**
+     * If this option is set, the log writer won't reset the segment writer if an error
+     * is encountered.
+     *
+     * @return true if we should disable automatic rolling
+     */
+    public boolean getDisableRollingOnLogSegmentError() {
+        return getBoolean(BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR,
+                BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR_DEFAULT);
+    }
+
+    /**
+     * Set the roll on segment error flag.
+     *
+     * @param disableRollingOnLogSegmentError
+     *        set roll on error flag
+     * @return dl configuration.
+     * @see #getDisableRollingOnLogSegmentError()
+     */
+    public DistributedLogConfiguration setDisableRollingOnLogSegmentError(boolean disableRollingOnLogSegmentError) {
+        setProperty(BKDL_DISABLE_ROLLING_ON_LOG_SEGMENT_ERROR, disableRollingOnLogSegmentError);
+        return this;
+    }
+
+    //
+    // DL Durability Settings
+    //
+
+    /**
+     * Check whether the durable write is enabled.
+     * <p>It is enabled by default.
+     *
+     * @return true if durable write is enabled. otherwise, false.
+     */
+    public boolean isDurableWriteEnabled() {
+        return this.getBoolean(BKDL_IS_DURABLE_WRITE_ENABLED, BKDL_IS_DURABLE_WRITE_ENABLED_DEFAULT);
+    }
+
+    /**
+     * Enable/Disable durable writes in writers.
+     *
+     * @param enabled
+     *          flag to enable/disable durable writes in writers.
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setDurableWriteEnabled(boolean enabled) {
+        setProperty(BKDL_IS_DURABLE_WRITE_ENABLED, enabled);
+        return this;
+    }
+
+    //
+    // DL Writer Transmit Settings
+    //
+
+    /**
+     * Get output buffer size for DL writers, in bytes.
+     * <p>Large buffer will result in higher compression ratio and
+     * it would use the bandwidth more efficiently and improve throughput.
+     * Set it to 0 would ask DL writers to transmit the data immediately,
+     * which it could achieve low latency.
+     * <p>The default value is 1KB.
+     *
+     * @return buffer size in byes.
+     */
+    public int getOutputBufferSize() {
+        return this.getInt(BKDL_OUTPUT_BUFFER_SIZE,
+                getInt(BKDL_OUTPUT_BUFFER_SIZE_OLD, BKDL_OUTPUT_BUFFER_SIZE_DEFAULT));
+    }
+
+    /**
+     * Set output buffer size for DL writers, in bytes.
+     *
+     * @param opBufferSize output buffer size.
+     * @return distributed log configuration
+     * @see #getOutputBufferSize()
+     */
+    public DistributedLogConfiguration setOutputBufferSize(int opBufferSize) {
+        setProperty(BKDL_OUTPUT_BUFFER_SIZE, opBufferSize);
+        return this;
+    }
+
+    /**
+     * Get Periodic Log Flush Frequency in milliseconds.
+     * <p>If the setting is set with a positive value, the data in output buffer
+     * will be flushed in this provided interval. The default value is 0.
+     *
+     * @return periodic flush frequency in milliseconds.
+     * @see #getOutputBufferSize()
+     */
+    public int getPeriodicFlushFrequencyMilliSeconds() {
+        return this.getInt(BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
+                BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT);
+    }
+
+    /**
+     * Set Periodic Log Flush Frequency in milliseconds.
+     *
+     * @param flushFrequencyMs periodic flush frequency in milliseconds.
+     * @return distributed log configuration
+     * @see #getPeriodicFlushFrequencyMilliSeconds()
+     */
+    public DistributedLogConfiguration setPeriodicFlushFrequencyMilliSeconds(int flushFrequencyMs) {
+        setProperty(BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS, flushFrequencyMs);
+        return this;
+    }
+
+    /**
+     * Is immediate flush enabled.
+     * <p>If it is enabled, it would flush control record immediately after adding
+     * data completed. The default value is false.
+     *
+     * @return whether immediate flush is enabled
+     */
+    public boolean getImmediateFlushEnabled() {
+        return getBoolean(BKDL_ENABLE_IMMEDIATE_FLUSH, BKDL_ENABLE_IMMEDIATE_FLUSH_DEFAULT);
+    }
+
+    /**
+     * Enable/Disable immediate flush
+     *
+     * @param enabled
+     *          flag to enable/disable immediate flush.
+     * @return configuration instance.
+     * @see #getImmediateFlushEnabled()
+     */
+    public DistributedLogConfiguration setImmediateFlushEnabled(boolean enabled) {
+        setProperty(BKDL_ENABLE_IMMEDIATE_FLUSH, enabled);
+        return this;
+    }
+
+    /**
+     * Get minimum delay between immediate flushes in milliseconds.
+     * <p>This setting only takes effects when {@link #getImmediateFlushEnabled()}
+     * is enabled. It torelants the bursty of traffic when immediate flush is enabled,
+     * which prevents sending too many control records to the bookkeeper.
+     *
+     * @return minimum delay between immediate flushes in milliseconds
+     * @see #getImmediateFlushEnabled()
+     */
+    public int getMinDelayBetweenImmediateFlushMs() {
+        return this.getInt(BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS, BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS_DEFAULT);
+    }
+
+    /**
+     * Set minimum delay between immediate flushes in milliseconds
+     *
+     * @param minDelayMs minimum delay between immediate flushes in milliseconds.
+     * @return distributed log configuration
+     * @see #getMinDelayBetweenImmediateFlushMs()
+     */
+    public DistributedLogConfiguration setMinDelayBetweenImmediateFlushMs(int minDelayMs) {
+        setProperty(BKDL_MINIMUM_DELAY_BETWEEN_IMMEDIATE_FLUSH_MILLISECONDS, minDelayMs);
+        return this;
+    }
+
+    /**
+     * Get Periodic Keep Alive Frequency in milliseconds.
+     * <p>If the setting is set with a positive value, it would periodically write a control record
+     * to keep the stream active. The default value is 0.
+     *
+     * @return periodic keep alive frequency in milliseconds.
+     */
+    public int getPeriodicKeepAliveMilliSeconds() {
+        return this.getInt(BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS, BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS_DEFAULT);
+    }
+
+    /**
+     * Set Periodic Keep Alive Frequency in milliseconds.
+     *
+     * @param keepAliveMs keep alive frequency in milliseconds.
+     * @return distributedlog configuration
+     * @see #getPeriodicKeepAliveMilliSeconds()
+     */
+    public DistributedLogConfiguration setPeriodicKeepAliveMilliSeconds(int keepAliveMs) {
+        setProperty(BKDL_PERIODIC_KEEP_ALIVE_MILLISECONDS, keepAliveMs);
+        return this;
+    }
+
+    //
+    // DL Retention/Truncation Settings
+    //
+
+    /**
+     * Get log segment retention period in hours.
+     * The default value is 3 days.
+     *
+     * @return log segment retention period in hours
+     */
+    public int getRetentionPeriodHours() {
+        return this.getInt(BKDL_RETENTION_PERIOD_IN_HOURS,
+                getInt(BKDL_RETENTION_PERIOD_IN_HOURS_OLD,
+                        BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT));
+    }
+
+    /**
+     * Set log segment retention period in hours.
+     *
+     * @param retentionHours retention period in hours.
+     * @return distributed log configuration
+     */
+    public DistributedLogConfiguration setRetentionPeriodHours(int retentionHours) {
+        setProperty(BKDL_RETENTION_PERIOD_IN_HOURS, retentionHours);
+        return this;
+    }
+
+    /**
+     * Is truncation managed explicitly by the application.
+     * <p>If this is set then time based retention is only a hint to perform
+     * deferred cleanup. However we never remove a segment that has not been
+     * already marked truncated.
+     * <p>It is disabled by default.
+     *
+     * @return whether truncation managed explicitly by the application
+     * @see org.apache.distributedlog.LogSegmentMetadata.TruncationStatus
+     */
+    public boolean getExplicitTruncationByApplication() {
+        return getBoolean(BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION,
+                BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION_DEFAULT);
+    }
+
+    /**
+     * Enable/Disable whether truncation is managed explicitly by the application.
+     *
+     * @param enabled
+     *          flag to enable/disable whether truncation is managed explicitly by the application.
+     * @return configuration instance.
+     */
+    public DistributedLogConfiguration setExplicitTruncationByApplication(boolean enabled) {
+        setProperty(BKDL_EXPLICIT_TRUNCATION_BY_APPLICATION, enabled);
+        return this;
+    }
+
+    //
+    // Log Segment Rolling Settings
+    //
+
+    /**
+     * Get log segment rolling interval in minutes.
+     * <p>If the setting is set to a positive value, DL writer will roll log segments
+     * based on time. Otherwise, it will roll log segments based on size.
+     * <p>The default value is 2 hours.
+     *
+     * @return log segment rolling interval in minutes
+     * @see #getMaxLogSegmentBytes()
+     */
+    public int getLogSegmentRollingIntervalMinutes() {
+        return this.getInt(BKDL_ROLLING_INTERVAL_IN_MINUTES,
+                getInt(BKDL_ROLLING_INTERVAL_IN_MINUTES_OLD,
+                        BKDL_ROLLING_INTERVAL_IN_MINUTES_DEFAULT));
+    }
+
+    /**
+     * Set log segment rolling interval in minutes.
+     *
+     * @param rollingMinutes rolling interval in minutes.
+     * @return distributed log configuration
+     * @see #getLogSegmentRollingIntervalMinutes()
+     */
+    public DistributedLogConfiguration setLogSegmentRollingIntervalMinutes(int rollingMinutes) {
+        setProperty(BKDL_ROLLING_INTERVAL_IN_MINUTES, rollingMinutes);
+        return this;
+    }
+
+    /**
+     * Get Max LogSegment Size in Bytes.
+     * <p>This setting only takes effects when time based rolling is disabled.
+     * DL writer will roll into a new log segment only after current one reaches
+     * this threshold.
+     * <p>The default value is 256MB.
+     *
+     * @return max logsegment size in bytes.
+     * @see #getLogSegmentRollingIntervalMinutes()
+     */
+    public long getMaxLogSegmentBytes() {
+        long maxBytes = this.getLong(BKDL_MAX_LOGSEGMENT_BYTES, BKDL_MAX_LOGSEGMENT_BYTES_DEFAULT);
+        if (maxBytes <= 0) {
+            maxBytes = BKDL_MAX_LOGSEGMENT_BYTES_DEFAULT;
+        }
+        return maxBytes;
+    }
+
+    /**
+     * Set Max LogSegment Size in Bytes.
+     *
+     * @param maxBytes
+     *          max logsegment size in bytes.
+     * @return configuration.
+     * @see #getMaxLogSegmentBytes()
+     */
+    public DistributedLogConfiguration setMaxLogSegmentBytes(long maxBytes) {
+        setProperty(BKDL_MAX_LOGSEGMENT_BYTES, maxBytes);
+        return this;
+    }
+
+    /**
+     * Get log segment rolling concurrency.
+     * <p>It limits how many writers could roll log segments concurrently.
+     * The default value is 1.
+     *
+     * @return log segment rolling concurrency.
+     * @see #setLogSegmentRollingConcurrency(int)
+     */
+    public int getLogSegmentRollingConcurrency() {
+        return getInt(BKDL_LOGSEGMENT_ROLLING_CONCURRENCY, BKDL_LOGSEGMENT_ROLLING_CONCURRENCY_DEFAULT);
+    }
+
+    /**
+     * Set log segment rolling concurrency. <i>0</i> means disable rolling concurrency.
+     * <i>larger than 0</i> means how many log segment could be rolled at the same time.
+     * <i>less than 0</i> means unlimited concurrency on rolling log segments.
+     *
+     * @param concurrency
+     *          log segment rolling concurrency.
+     * @return distributed log configuration.
+     * @see #getLogSegmentRollingConcurrency()
+     */
+    public DistributedLogConfiguration setLogSegmentRollingConcurrency(int concurrency) {
+        setProperty(BKDL_LOGSEGMENT_ROLLING_CONCURRENCY, concurrency);
+        return this;
+    }
+
+    //
+    // Lock Settings
+    //
+
+    /**
+     * Is lock enabled when opening a writer to write a stream?
+     * <p> We don't generally require a lock to write a stream to guarantee correctness. The lock
+     * is more on tracking ownerships. The built-in fencing mechanism is used guarantee correctness
+     * during stream owner failover. It is okay to disable lock if your application knows which nodes
+     * have to write which streams.
+     *
+     * @return true if lock is enabled, otherwise false.
+     */
+    public boolean isWriteLockEnabled() {
+        return this.getBoolean(BKDL_WRITE_LOCK_ENABLED, BKDL_WRITE_LOCK_ENABLED_DEFAULT);
+    }
+
+    /**
+     * Enable lock for opening a writer to write a stream?
+     *
+     * @param enabled flag to enable or disable lock for opening a writer to write a stream.
+     * @return distributedlog configuration.
+     */
+    public DistributedLogConfiguration setWriteLockEnabled(boolean enabled) {
+        setProperty(BKDL_WRITE_LOCK_ENABLED, enabled);
+        return this;
+    }
+
+    /**
+     * Get lock timeout in milliseconds. The default value is 30.
+     *
+     * @return lock timeout in milliseconds
+     */
+    public long getLockTimeoutMilliSeconds() {
+        return this.getLong(BKDL_LOCK_TIMEOUT, BKDL_LOCK_TIMEOUT_DEFAULT) * 1000;
+    }
+
+    /**
+     * Set lock timeout in seconds.
+     *
+     * @param lockTimeout lock timeout in seconds.
+     * @return distributed log configuration
+     * @see #getLockTimeoutMilliSeconds()
+     */
+    public DistributedLogConfiguration setLockTimeout(long lockTimeout) {
+        setProperty(BKDL_LOCK_TIMEOUT, lockTimeout);
+        return this;
+    }
+
+    /**
+     * Get lock reacquire timeout in milliseconds. The default value is 120 seconds.
+     *
+     * @return lock reacquire timeout in milliseconds
+     */
+    public long getLockReacquireTimeoutMilliSeconds() {
+        return this.getLong(BKDL_LOCK_REACQUIRE_TIMEOUT, BKDL_LOCK_REACQUIRE_TIMEOUT_DEFAULT) * 1000;
+    }
+
+    /**
+     * Set lock reacquire timeout in seconds.
+     *
+     * @param lockReacquireTimeout lock reacquire timeout in seconds.
+     * @return distributed log configuration
+     * @see #getLockReacquireTimeoutMilliSeconds()
+     */
+    public DistributedLogConfiguration setLockReacquireTimeoutSeconds(long lockReacquireTimeout) {
+        setProperty(BKDL_LOCK_REACQUIRE_TIMEOUT, lockReacquireTimeout);
+        return this;
+    }
+
+    /**
+     * Get lock internal operation timeout in milliseconds.
+     * The default value is 120 seconds.
+     *
+     * @return lock internal operation timeout in milliseconds.
+     */
+    public long getLockOpTimeoutMilliSeconds() {
+        return this.getLong(BKDL_LOCK_OP_TIMEOUT, BKDL_LOCK_OP_TIMEOUT_DEFAULT) * 1000;
+    }
+
+    /**
+     * Set lock internal operation timeout in seconds.
+     *
+     * @param lockOpTimeout lock internal operation timeout in seconds.
+     * @return distributed log configuration
+     * @see #getLockOpTimeoutMilliSeconds()
+     */
+    public DistributedLogConfiguration setLockOpTimeoutSeconds(long lockOpTimeout) {
+        setProperty(BKDL_LOCK_OP_TIMEOUT, lockOpTimeout);
+        return this;
+    }
+
+    //
+    // Ledger Allocator Settings
+    //
+
+    /**
+     * Whether to enable ledger allocator pool or not.
+     * It is disabled by default.
+     *
+     * @return whether using ledger allocator pool or not.
+     */
+    public boolean getEnableLedgerAllocatorPool() {
+        return getBoolean(BKDL_ENABLE_LEDGER_ALLOCATOR_POOL, BKDL_ENABLE_LEDGER_ALLOCATOR_POOL_DEFAULT);
+    }
+
+    /**
+     * Enable/Disable ledger allocator pool.
+     *
+     * @param enabled
+     *          enable/disable ledger allocator pool.
+     * @return configuration.
+     * @see #getEnableLedgerAllocatorPool()
+     */
+    public DistributedLogConfiguration setEnableLedgerAllocatorPool(boolean enabled) {
+        setProperty(BKDL_ENABLE_LEDGER_ALLOCATOR_POOL, enabled);
+        return this;
+    }
+
+    /**
+     * Get the path of ledger allocator pool.
+     * The default value is ".allocation_pool".
+     *
+     * @return path of ledger allocator pool.
+     */
+    public String getLedgerAllocatorPoolPath() {
+        return getString(BKDL_LEDGER_ALLOCATOR_POOL_PATH, BKDL_LEDGER_ALLOCATOR_POOL_PATH_DEFAULT);
+    }
+
+    /**
+     * Set the root path of ledger allocator pool
+     *
+     * @param path
+     *          path of ledger allocator pool.
+     * @return configuration
+     * @see #getLedgerAllocatorPoolPath()
+     */
+    public DistributedLogConfiguration setLedgerAllocatorPoolPath(String path) {
+        setProperty(BKDL_LEDGER_ALLOCATOR_POOL_PATH, path);
+        return this;
+    }
+
+    /**
+     * Get the name of ledger allocator pool.
+     *
+     * @return name of ledger allocator pool.
+     */
+    public String getLedgerAllocatorPoolName() {
+        return getString(BKDL_LEDGER_ALLOCATOR_POOL_NAME, BKDL_LEDGER_ALLOCATOR_POOL_NAME_DEFAULT);
+    }
+
+    /**
+     * Set name of ledger allocator pool.
+     *
+     * @param name
+     *          name of ledger allocator pool.
+     * @return configuration.
+     */
+    public DistributedLogConfiguration setLedgerAllocatorPoolName(String name) {
+        setProperty(BKDL_LEDGER_ALLOCATOR_POOL_NAME, name);
+        return this;
+    }
+
+    /**
+     * Get the core size of ledger allocator pool.
+     * The default value is 20.
+     *
+     * @return core size of ledger allocator pool.
+     */
+    public int getLedgerAllocatorPoolCoreSize() {
+        return getInt(BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE, BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE_DEFAULT);
+    }
+
+    /**
+     * Set core size of ledger allocator pool.
+     *
+     * @param poolSize
+     *          core size of ledger allocator pool.
+     * @return distributedlog configuration.
+     * @see #getLedgerAllocatorPoolCoreSize()
+     */
+    public DistributedLogConfiguration setLedgerAllocatorPoolCoreSize(int poolSize) {
+        setProperty(BKDL_LEDGER_ALLOCATOR_POOL_CORE_SIZE, poolSize);
+        return this;
+    }
+
+    //
+    // Write Limit Settings
+    //
+
+    /**
+     * Get the per stream outstanding write limit for dl.
+     * <p>If the setting is set with a positive value, the per stream
+     * write limiting is enabled. By default it is disabled.
+     *
+     * @return the per stream outstanding write limit for dl
+     * @see #getGlobalOutstandingWriteLimit()
+     */
+    public int getPerWriterOutstandingWriteLimit() {
+        return getInt(BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT,
+                BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT_DEFAULT);
+    }
+
+    /**
+     * Set the per stream outstanding write limit for dl.
+     *
+     * @param limit
+     *          per stream outstanding write limit for dl
+     * @return dl configuration
+     * @see #getPerWriterOutstandingWriteLimit()
+     */
+    public DistributedLogConfiguration setPerWriterOutstandingWriteLimit(int limit) {
+        setProperty(BKDL_PER_WRITER_OUTSTANDING_WRITE_LIMIT, limit);
+        return this;
+    }
+
+    /**
+     * Get the global write limit for dl.
+     * <p>If the setting is set with a positive value, the global
+     * write limiting is enabled. By default it is disabled.
+     *
+     * @return the global write limit for dl
+     * @see #getPerWriterOutstandingWriteLimit()
+     */
+    public int getGlobalOutstandingWriteLimit() {
+        return getInt(BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT, BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT_DEFAULT);
+    }
+
+    /**
+     * Set the global write limit for dl.
+     *
+     * @param limit
+     *          global write limit for dl
+     * @return dl configuration
+     * @see #getGlobalOutstandingWriteLimit()
+     */
+    public DistributedLogConfiguration setGlobalOutstandingWriteLimit(int limit) {
+        setProperty(BKDL_GLOBAL_OUTSTANDING_WRITE_LIMIT, limit);
+        return this;
+    }
+
+    /**
+     * Whether to darkmode outstanding writes limit.
+     * <p>If it is running in darkmode, it would not reject requests when
+     * it is over limit, but just record them in the stats.
+     * <p>By default, it is in darkmode.
+     *
+     * @return flag to darmkode pending write limit.
+     */
+    public boolean getOutstandingWriteLimitDarkmode() {
+        return getBoolean(BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE,
+                BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE_DEFAULT);
+    }
+
+    /**
+     * Set the flag to darkmode outstanding writes limit.
+     *
+     * @param darkmoded
+     *          flag to darmkode pending write limit
+     * @return dl configuration.
+     * @see #getOutstandingWriteLimitDarkmode()
+     */
+    public DistributedLogConfiguration setOutstandingWriteLimitDarkmode(boolean darkmoded) {
+        setProperty(BKDL_OUTSTANDING_WRITE_LIMIT_DARKMODE, darkmoded);
+        return this;
+    }
+
+    //
+    // DL Reader General Settings
+    //
+
+    /**
+     * Get the long poll time out for read last add confirmed requests, in milliseconds.
+     * The default value is 1 second.
+     *
+     * @return long poll timeout in milliseconds
+     * @see #getReadLACLongPollTimeout()
+     */
+    public int getReadLACLongPollTimeout() {
+        return this.getInt(BKDL_READLACLONGPOLL_TIMEOUT, BKDL_READLACLONGPOLL_TIMEOUT_DEFAULT);
+    }
+
+    /**
+     * Set the long poll time out for read last add confirmed requests, in milliseconds.
+     *
+     * @param readAheadLongPollTimeout long poll timeout in milliseconds
+     * @return distributed log configuration
+     * @see #getReadLACLongPollTimeout()
+     */
+    public DistributedLogConfiguration setReadLACLongPollTimeout(int readAheadLongPollTimeout) {
+        setProperty(BKDL_READLACLONGPOLL_TIMEOUT, readAheadLongPollTimeout);
+        return this;
+    }
+
+    /**
+     * Get the flag whether to deserialize record set on reads.
+     *
+     * @return true if it should deserialize, otherwise false.
+     */
+    public boolean getDeserializeRecordSetOnReads() {
+        return getBoolean(BKDL_DESERIALIZE_RECORDSET_ON_READS, BKDL_DESERIALIZE_RECORDSET_ON_READS_DEFAULT);
+    }
+
+    /**
+     * Enable or disable deserialize recordset on reads.
+     *
+     * @param enabled
+     *          flag whether to deserialize recordset
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setDeserializeRecordSetOnReads(boolean enabled) {
+        setProperty(BKDL_DESERIALIZE_RECORDSET_ON_READS, enabled);
+        return this;
+    }
+
+    //
+    // Idle reader settings
+    //
+
+    /**
+     * Get the time in milliseconds as the threshold for when an idle reader should dump warnings
+     * <p>The default value is 2 minutes.
+     *
+     * @return reader idle warn threshold in millis.
+     * @see #getReaderIdleErrorThresholdMillis()
+     */
+    public int getReaderIdleWarnThresholdMillis() {
+        return getInt(BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS,
+                BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS_DEFAULT);
+    }
+
+    /**
+     * Set the time in milliseconds as the threshold for when an idle reader should dump warnings
+     *
+     * @param warnThreshold time after which we should dump the read ahead state
+     * @return distributed log configuration
+     * @see #getReaderIdleWarnThresholdMillis()
+     */
+    public DistributedLogConfiguration setReaderIdleWarnThresholdMillis(int warnThreshold) {
+        setProperty(BKDL_READER_IDLE_WARN_THRESHOLD_MILLIS, warnThreshold);
+        return this;
+    }
+
+    /**
+     * Get the time in milliseconds as the threshold for when an idle reader should throw errors
+     * <p>The default value is <i>Integer.MAX_VALUE</i>.
+     *
+     * @return reader idle error threshold in millis
+     * @see #getReaderIdleWarnThresholdMillis()
+     */
+    public int getReaderIdleErrorThresholdMillis() {
+        return getInt(BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS,
+                BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT);
+    }
+
+    /**
+     * Set the time in milliseconds as the threshold for when an idle reader should throw errors
+     *
+     * @param warnThreshold time after which we should throw idle reader errors
+     * @return distributed log configuration
+     * @see #getReaderIdleErrorThresholdMillis()
+     */
+    public DistributedLogConfiguration setReaderIdleErrorThresholdMillis(int warnThreshold) {
+        setProperty(BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS, warnThreshold);
+        return this;
+    }
+
+    //
+    // Reader Constraint Settings
+    //
+
+    /**
+     * Get if we should ignore truncation status when reading the records
+     *
+     * @return if we should ignore truncation status
+     */
+    public boolean getIgnoreTruncationStatus() {
+        return getBoolean(BKDL_READER_IGNORE_TRUNCATION_STATUS, BKDL_READER_IGNORE_TRUNCATION_STATUS_DEFAULT);
+    }
+
+    /**
+     * Set if we should ignore truncation status when reading the records
+     *
+     * @param ignoreTruncationStatus
+     *          if we should ignore truncation status
+     */
+    public DistributedLogConfiguration setIgnoreTruncationStatus(boolean ignoreTruncationStatus) {
+        setProperty(BKDL_READER_IGNORE_TRUNCATION_STATUS, ignoreTruncationStatus);
+        return this;
+    }
+
+    /**
+     * Get if we should alert when reader is positioned on a truncated segment
+     *
+     * @return if we should alert when reader is positioned on a truncated segment
+     */
+    public boolean getAlertWhenPositioningOnTruncated() {
+        return getBoolean(BKDL_READER_ALERT_POSITION_ON_TRUNCATED, BKDL_READER_ALERT_POSITION_ON_TRUNCATED_DEFAULT);
+    }
+
+    /**
+     * Set if we should alert when reader is positioned on a truncated segment
+     *
+     * @param alertWhenPositioningOnTruncated
+     *          if we should alert when reader is positioned on a truncated segment
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setAlertWhenPositioningOnTruncated(boolean alertWhenPositioningOnTruncated) {
+        setProperty(BKDL_READER_ALERT_POSITION_ON_TRUNCATED, alertWhenPositioningOnTruncated);
+        return this;
+    }
+
+    /**
+     * Get whether position gap detection for reader enabled.
+     * @return whether position gap detection for reader enabled.
+     */
+    public boolean getPositionGapDetectionEnabled() {
+        return getBoolean(BKDL_READER_POSITION_GAP_DETECTION_ENABLED, BKDL_READER_POSITION_GAP_DETECTION_ENABLED_DEFAULT);
+    }
+
+    /**
+     * Set if enable position gap detection for reader.
+     *
+     * @param enabled
+     *          flag to enable/disable position gap detection on reader.
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setPositionGapDetectionEnabled(boolean enabled) {
+        setProperty(BKDL_READER_POSITION_GAP_DETECTION_ENABLED, enabled);
+        return this;
+    }
+
+    //
+    // ReadAhead Settings
+    //
+
+    /**
+     * Set if we should enable read ahead.
+     * By default is it enabled.
+     *
+     * @param enableReadAhead
+     *          Enable read ahead
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setEnableReadAhead(boolean enableReadAhead) {
+        setProperty(BKDL_ENABLE_READAHEAD, enableReadAhead);
+        return this;
+    }
+
+    /**
+     * Get if we should enable read ahead
+     *
+     * @return if read ahead is enabled
+     */
+    public boolean getEnableReadAhead() {
+        return getBoolean(BKDL_ENABLE_READAHEAD, BKDL_ENABLE_READAHEAD_DEFAULT);
+    }
+
+    /**
+     * Set if we should enable force read
+     *
+     * @param enableForceRead
+     *          Enable force read
+     */
+    public DistributedLogConfiguration setEnableForceRead(boolean enableForceRead) {
+        setProperty(BKDL_ENABLE_FORCEREAD, enableForceRead);
+        return this;
+    }
+
+    /**
+     * Get if we should enable force read
+     *
+     * @return if should use separate ZK Clients
+     */
+    public boolean getEnableForceRead() {
+        return getBoolean(BKDL_ENABLE_FORCEREAD, BKDL_ENABLE_FORCEREAD_DEFAULT);
+    }
+
+    /**
+     * Get the max records cached by readahead cache.
+     * <p>The default value is 10. Increase this value to improve throughput,
+     * but be careful about the memory.
+     *
+     * @return max records cached by readahead cache.
+     */
+    public int getReadAheadMaxRecords() {
+        return this.getInt(BKDL_READAHEAD_MAX_RECORDS,
+                getInt(BKDL_READAHEAD_MAX_RECORDS_OLD,
+                        BKDL_READAHEAD_MAX_RECORDS_DEFAULT));
+    }
+
+    /**
+     * Set the maximum records allowed to be cached by read ahead worker.
+     *
+     * @param readAheadMaxEntries max records to cache.
+     * @return distributed log configuration
+     * @see #getReadAheadMaxRecords()
+     */
+    public DistributedLogConfiguration setReadAheadMaxRecords(int readAheadMaxEntries) {
+        setProperty(BKDL_READAHEAD_MAX_RECORDS, readAheadMaxEntries);
+        return this;
+    }
+
+    /**
+     * Get number of entries read as a batch by readahead worker.
+     * <p>The default value is 2. Increase the value to increase the concurrency
+     * of reading entries from bookkeeper.
+     *
+     * @return number of entries read as a batch.
+     */
+    public int getReadAheadBatchSize() {
+        return this.getInt(BKDL_READAHEAD_BATCHSIZE,
+                getInt(BKDL_READAHEAD_BATCHSIZE_OLD,
+                        BKDL_READAHEAD_BATCHSIZE_DEFAULT));
+    }
+
+    /**
+     * Set number of entries read as a batch by readahead worker.
+     *
+     * @param readAheadBatchSize
+     *          Read ahead batch size.
+     * @return distributed log configuration
+     * @see #getReadAheadBatchSize()
+     */
+    public DistributedLogConfiguration setReadAheadBatchSize(int readAheadBatchSize) {
+        setProperty(BKDL_READAHEAD_BATCHSIZE, readAheadBatchSize);
+        return this;
+    }
+
+    /**
+     * Get the wait time between successive attempts to poll for new log records, in milliseconds.
+     * The default value is 200 ms.
+     *
+     * @return read ahead wait time
+     */
+    public int getReadAheadWaitTime() {
+        return this.getInt(BKDL_READAHEAD_WAITTIME,
+                getInt(BKDL_READAHEAD_WAITTIME_OLD, BKDL_READAHEAD_WAITTIME_DEFAULT));
+    }
+
+    /**
+     * Set the wait time between successive attempts to poll for new log records, in milliseconds
+     *
+     * @param readAheadWaitTime read ahead wait time
+     * @return distributed log configuration
+     * @see #getReadAheadWaitTime()
+     */
+    public DistributedLogConfiguration setReadAheadWaitTime(int readAheadWaitTime) {
+        setProperty(BKDL_READAHEAD_WAITTIME, readAheadWaitTime);
+        return this;
+    }
+
+    /**
+     * Get the wait time if it reaches end of stream and
+     * <b>there isn't any inprogress logsegment in the stream</b>, in millis.
+     * <p>The default value is 10 seconds.
+     *
+     * @see #setReadAheadWaitTimeOnEndOfStream(int)
+     * @return the wait time if it reaches end of stream and there isn't
+     * any inprogress logsegment in the stream, in millis.
+     */
+    public int getReadAheadWaitTimeOnEndOfStream() {
+        return this.getInt(BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM,
+                getInt(BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM_OLD,
+                        BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM_DEFAULT));
+    }
+
+    /**
+     * Set the wait time that would be used for readahead to backoff polling
+     * logsegments from zookeeper when it reaches end of stream and there isn't
+     * any inprogress logsegment in the stream. The unit is millis.
+     *
+     * @param waitTime
+     *          wait time that readahead used to backoff when reaching end of stream.
+     * @return distributedlog configuration
+     * @see #getReadAheadWaitTimeOnEndOfStream()
+     */
+    public DistributedLogConfiguration setReadAheadWaitTimeOnEndOfStream(int waitTime) {
+        setProperty(BKDL_READAHEAD_WAITTIME_ON_ENDOFSTREAM, waitTime);
+        return this;
+    }
+
+    /**
+     * If readahead keeps receiving {@link org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException} on
+     * reading last add confirmed in given period, it would stop polling last add confirmed and re-initialize the ledger
+     * handle and retry. The threshold is specified in milliseconds.
+     * <p>The default value is 10 seconds.
+     *
+     * @return error threshold in milliseconds, that readahead will reinitialize ledger handle after keeping receiving
+     * no such ledger exceptions.
+     */
+    public int getReadAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis() {
+        return this.getInt(BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS,
+                           BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS_DEFAULT);
+    }
+
+    /**
+     * Set the error threshold that readahead will reinitialize ledger handle after keeping receiving no such ledger exceptions.
+     *
+     * @see #getReadAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis()
+     * @param thresholdMillis
+     *          error threshold in milliseconds, that readahead will reinitialize ledger handle after keeping receiving
+     *          no such ledger exceptions.
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setReadAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis(long thresholdMillis) {
+        setProperty(BKDL_READAHEAD_NOSUCHLEDGER_EXCEPTION_ON_READLAC_ERROR_THRESHOLD_MILLIS, thresholdMillis);
+        return this;
+    }
+
+    /**
+     * When corruption is encountered in an entry, skip it and move on. Must disable gap detection for
+     * this to work.
+     *
+     * @return should broken records be skipped
+     */
+    public boolean getReadAheadSkipBrokenEntries() {
+        return getBoolean(BKDL_READAHEAD_SKIP_BROKEN_ENTRIES, BKDL_READAHEAD_SKIP_BROKEN_ENTRIES_DEFAULT);
+    }
+
+    /**
+     * Set the percentage of operations to delay in read ahead.
+     *
+     * @param enabled
+     *          should brokenn records be skipped
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setReadAheadSkipBrokenEntries(boolean enabled) {
+        setProperty(BKDL_READAHEAD_SKIP_BROKEN_ENTRIES, enabled);
+        return this;
+    }
+
+    /**
+     * Get the number prefetch entries per log segment. Default value is 4.
+     *
+     * @return the number prefetch entries per log segment.
+     */
+    public int getNumPrefetchEntriesPerLogSegment() {
+        return getInt(BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT, BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT);
+    }
+
+    /**
+     * Set the number prefetch entries per log segment.
+     *
+     * @param numEntries the number prefetch entries per log segment.
+     * @return configuration
+     */
+    public DistributedLogConfiguration setNumPrefetchEntriesPerLogSegment(int numEntries) {
+        setProperty(BKDL_NUM_PREFETCH_ENTRIES_PER_LOGSEGMENT, numEntries);
+        return this;
+    }
+
+    /**
+     * Get the max prefetch entries per log segment. Default value is 4.
+     *
+     * @return the max prefetch entries per log segment.
+     */
+    public int getMaxPrefetchEntriesPerLogSegment() {
+        return getInt(BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT, BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT_DEFAULT);
+    }
+
+    /**
+     * Set the max prefetch entries per log segment.
+     *
+     * @param numEntries the max prefetch entries per log segment.
+     * @return configuration
+     */
+    public DistributedLogConfiguration setMaxPrefetchEntriesPerLogSegment(int numEntries) {
+        setProperty(BKDL_MAX_PREFETCH_ENTRIES_PER_LOGSEGMENT, numEntries);
+        return this;
+    }
+
+    //
+    // DL Reader Scan Settings
+    //
+
+    /**
+     * Number of entries to scan for first scan of reading last record.
+     *
+     * @return number of entries to scan for first scan of reading last record.
+     */
+    public int getFirstNumEntriesPerReadLastRecordScan() {
+        return getInt(BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN, BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT);
+    }
+
+    /**
+     * Set number of entries to scan for first scan of reading last record.
+     *
+     * @param numEntries
+     *          number of entries to scan
+     * @return distributedlog configuration.
+     */
+    public DistributedLogConfiguration setFirstNumEntriesPerReadLastRecordScan(int numEntries) {
+        setProperty(BKDL_FIRST_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN, numEntries);
+        return this;
+    }
+
+    /**
+     * Max number of entries for each scan to read last record.
+     *
+     * @return max number of entries for each scan to read last record.
+     */
+    public int getMaxNumEntriesPerReadLastRecordScan() {
+        return getInt(BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN, BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN_DEFAULT);
+    }
+
+    /**
+     * Set max number of entries for each scan to read last record.
+     *
+     * @param numEntries
+     *          number of entries to scan
+     * @return distributedlog configuration.
+     */
+    public DistributedLogConfiguration setMaxNumEntriesPerReadLastRecordScan(int numEntries) {
+        setProperty(BKDL_MAX_NUM_ENTRIES_PER_READ_LAST_RECORD_SCAN, numEntries);
+        return this;
+    }
+
+    //
+    // DL Reader Log Existence Checking Settings
+    //
+
+    /**
+     * Get the backoff start time to check log existence if the log doesn't exist.
+     *
+     * @return the backoff start time to check log existence if the log doesn't exist.
+     */
+    public long getCheckLogExistenceBackoffStartMillis() {
+        return getLong(BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS, BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS_DEFAULT);
+    }
+
+    /**
+     * Set the backoff start time to check log existence if the log doesn't exist.
+     *
+     * @param backoffMillis
+     *          backoff time in millis
+     * @return dl configuration
+     */
+    public DistributedLogConfiguration setCheckLogExistenceBackoffStartMillis(long backoffMillis) {
+        setProperty(BKDL_CHECK_LOG_EXISTENCE_BACKOFF_START_MS, backoffMillis);
+        return this;
+    }
+
+    /**
+     * Get the backoff max time to check log existence if the log doesn't exist.
+     *
+     * @return the backoff max time to check log existence if the log doesn't exist.
+     */
+    public long getCheckLogExistenceBackoffMaxMillis() {
+        return getLong(BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS, BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS_DEFAULT);
+    }
+
+    /**
+     * Set the backoff max time to check log existence if the log doesn't exist.
+     *
+     * @param backoffMillis
+     *          backoff time in millis
+     * @return dl configuration
+     */
+    public DistributedLogConfiguration setCheckLogExistenceBackoffMaxMillis(long backoffMillis) {
+        setProperty(BKDL_CHECK_LOG_EXISTENCE_BACKOFF_MAX_MS, backoffMillis);
+        return this;
+    }
+
+    //
+    // Tracing/Stats Settings
+    //
+
+    /**
+     * Whether to trace read ahead delivery latency or not?
+     *
+     * @return flag to trace read ahead delivery latency.
+     */
+    public boolean getTraceReadAheadDeliveryLatency() {
+        return getBoolean(BKDL_TRACE_READAHEAD_DELIVERY_LATENCY, BKDL_TRACE_READAHEAD_DELIVERY_LATENCY_DEFAULT);
+    }
+
+    /**
+     * Set the flag to trace readahead delivery latency.
+     *
+     * @param enabled
+     *          flag to trace readahead delivery latency.
+     * @return dl configuration.
+     */
+    public DistributedLogConfiguration setTraceReadAheadDeliveryLatency(boolean enabled) {
+        setProperty(BKDL_TRACE_READAHEAD_DELIVERY_LATENCY, enabled);
+        return this;
+    }
+
+    /**
+     * Get the warn threshold (in millis) of metadata access latency.
+     *
+     * @return warn threshold of metadata access latency, in millis.
+     */
+    public long getMetadataLatencyWarnThresholdMillis() {
+        return getLong(BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS, BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT);
+    }
+
+    /**
+     * Set the warn threshold of metadata access latency, in millis.
+     *
+     * @param warnThresholdMillis
+     *          warn threshold of metadata access latency, in millis
+     * @return dl configuration
+     */
+    public DistributedLogConfiguration setMetadataLatencyWarnThresholdMillis(long warnThresholdMillis) {
+        setProperty(BKDL_METADATA_LATENCY_WARN_THRESHOLD_MS, warnThresholdMillis);
+        return this;
+    }
+
+    /**
+     * Get the warn threshold (in millis) of data access latency.
+     *
+     * @return warn threshold of data access latency, in millis.
+     */
+    public long getDataLatencyWarnThresholdMillis() {
+        return getLong(BKDL_DATA_LATENCY_WARN_THRESHOLD_MS, BKDL_DATA_LATENCY_WARN_THRESHOLD_MS_DEFAULT);
+    }
+
+    /**
+     * Set the warn threshold of data access latency, in millis.
+     *
+     * @param warnThresholdMillis
+     *          warn threshold of data access latency, in millis
+     * @return dl configuration
+     */
+    public DistributedLogConfiguration setDataLatencyWarnThresholdMillis(long warnThresholdMillis) {
+        setProperty(BKDL_DATA_LATENCY_WARN_THRESHOLD_MS, warnThresholdMillis);
+        return this;
+    }
+
+    /**
+     * Whether to trace read ahead changes? If enabled, it will log readahead metadata changes with timestamp.
+     * It is helpful when you are troubleshooting latency related issues.
+     *
+     * @return flag to trace read ahead delivery latency.
+     */
+    public boolean getTraceReadAheadMetadataChanges() {
+        return getBoolean(BKDL_TRACE_READAHEAD_METADATA_CHANGES, BKDL_TRACE_READAHEAD_MEATDATA_CHANGES_DEFAULT);
+    }
+
+    /**
+     * Set the flag to trace readahead metadata changes.
+     *
+     * @see #getTraceReadAheadMetadataChanges()
+     *
+     * @param enabled
+     *          flag to trace readahead metadata changes.
+     * @return dl configuration.
+     */
+    public DistributedLogConfiguration setTraceReadAheadMetadataChanges(boolean enabled) {
+        setProperty(BKDL_TRACE_READAHEAD_METADATA_CHANGES, enabled);
+        return this;
+    }
+
+    /**
+     * Whether to trace long running tasks and record task execution stats in thread pools.
+     *
+     * @return flag to enable task execution stats
+     */
+    public boolean getEnableTaskExecutionStats() {
+        return getBoolean(BKDL_ENABLE_TASK_EXECUTION_STATS, BKDL_ENABLE_TASK_EXECUTION_STATS_DEFAULT);
+    }
+
+    /**
+     * Set to trace long running tasks and record task execution stats in thread pools.
+     *
+     * @see #getEnableTaskExecutionStats()
+     *
+     * @param enabled
+     *          flag to enable task execution stats.
+     * @return dl configuration.
+     */
+    public DistributedLogConfiguration setEnableTaskExecutionStats(boolean enabled) {
+        setProperty(BKDL_ENABLE_TASK_EXECUTION_STATS, enabled);
+        return this;
+    }
+
+    /**
+     * Report long running task after execution takes longer than the given interval.
+     *
+     * @return warn time for long running tasks
+     */
+    public long getTaskExecutionWarnTimeMicros() {
+        return getLong(BKDL_TASK_EXECUTION_WARN_TIME_MICROS, BKDL_TASK_EXECUTION_WARN_TIME_MICROS_DEFAULT);
+    }
+
+    /**
+     * Set warn time for reporting long running tasks.
+     *
+     * @see #getTaskExecutionWarnTimeMicros()
+     *
+     * @param warnTimeMicros
+     *          warn time for long running tasks.
+     * @return dl configuration.
+     */
+    public DistributedLogConfiguration setTaskExecutionWarnTimeMicros(long warnTimeMicros) {
+        setProperty(BKDL_TASK_EXECUTION_WARN_TIME_MICROS, warnTimeMicros);
+        return this;
+    }
+
+    /**
+     * Whether to enable per stream stat or not.
+     *
+     * @deprecated please use {@link DistributedLogNamespaceBuilder#perLogStatsLogger(StatsLogger)}
+     * @return flag to enable per stream stat.
+     */
+    public boolean getEnablePerStreamStat() {
+        return getBoolean(BKDL_ENABLE_PERSTREAM_STAT, BKDL_ENABLE_PERSTREAM_STAT_DEFAULT);
+    }
+
+    /**
+     * Set the flag to enable per stream stat or not.
+     *
+     * @deprecated please use {@link DistributedLogNamespaceBuilder#perLogStatsLogger(StatsLogger)}
+     * @param enabled
+     *          flag to enable/disable per stream stat.
+     * @return dl configuration.
+     */
+    public DistributedLogConfiguration setEnablePerStreamStat(boolean enabled) {
+        setProperty(BKDL_ENABLE_PERSTREAM_STAT, enabled);
+        return this;
+    }
+
+    //
+    // Settings for Feature Providers
+    //
+
+    /**
+     * Get feature provider class.
+     *
+     * @return feature provider class.
+     * @throws ConfigurationException
+     */
+    public Class<? extends FeatureProvider> getFeatureProviderClass()
+            throws ConfigurationException {
+        return ReflectionUtils.getClass(this, BKDL_FEATURE_PROVIDER_CLASS, DefaultFeatureProvider.class,
+                FeatureProvider.class, FeatureProvider.class.getClassLoader());
+    }
+
+    /**
+     * Set feature provider class.
+     *
+     * @param providerClass
+     *          feature provider class.
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setFeatureProviderClass(Class<? extends FeatureProvider> providerClass) {
+        setProperty(BKDL_FEATURE_PROVIDER_CLASS, providerClass.getName());
+        return this;
+    }
+
+    /**
+     * Get the base config path for file feature provider.
+     *
+     * @return base config path for file feature provider.
+     */
+    public String getFileFeatureProviderBaseConfigPath() {
+        return getString(BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH,
+                BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH_DEFAULT);
+    }
+
+    /**
+     * Set the base config path for file feature provider.
+     *
+     * @param configPath
+     *          base config path for file feature provider.
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setFileFeatureProviderBaseConfigPath(String configPath) {
+        setProperty(BKDL_FILE_FEATURE_PROVIDER_BASE_CONFIG_PATH, configPath);
+        return this;
+    }
+
+    /**
+     * Get the overlay config path for file feature provider.
+     *
+     * @return overlay config path for file feature provider.
+     */
+    public String getFileFeatureProviderOverlayConfigPath() {
+        return getString(BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH,
+                BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH_DEFAULT);
+    }
+
+    /**
+     * Set the overlay config path for file feature provider.
+     *
+     * @param configPath
+     *          overlay config path for file feature provider.
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setFileFeatureProviderOverlayConfigPath(String configPath) {
+        setProperty(BKDL_FILE_FEATURE_PROVIDER_OVERLAY_CONFIG_PATH,
+                configPath);
+        return this;
+    }
+
+    //
+    // Settings for Namespaces
+    //
+
+    /**
+     * Is federated namespace implementation enabled.
+     *
+     * @return true if federated namespace is enabled. otherwise, false.
+     */
+    public boolean isFederatedNamespaceEnabled() {
+        return getBoolean(BKDL_FEDERATED_NAMESPACE_ENABLED, BKDL_FEDERATED_NAMESPACE_ENABLED_DEFAULT);
+    }
+
+    /**
+     * Use federated namespace implementation if this flag is enabled.
+     *
+     * @param enabled flag to enable federated namespace implementation
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setFederatedNamespaceEnabled(boolean enabled) {
+        setProperty(BKDL_FEDERATED_NAMESPACE_ENABLED, enabled);
+        return this;
+    }
+
+    /**
+     * Get the max logs per sub namespace for federated namespace.
+     *
+     * @return max logs per sub namespace
+     */
+    public int getFederatedMaxLogsPerSubnamespace() {
+        return getInt(BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE, BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE_DEFAULT);
+    }
+
+    /**
+     * Set the max logs per sub namespace for federated namespace.
+     *
+     * @param maxLogs
+     *          max logs per sub namespace
+     * @return distributedlog configuration.
+     */
+    public DistributedLogConfiguration setFederatedMaxLogsPerSubnamespace(int maxLogs) {
+        setProperty(BKDL_FEDERATED_MAX_LOGS_PER_SUBNAMESPACE, maxLogs);
+        return this;
+    }
+
+    /**
+     * Whether check the existence of a log if querying local cache of a federated namespace missed.
+     * Enabling it will issue zookeeper queries to check all sub namespaces under a federated namespace.
+     *
+     * NOTE: by default it is on for all admin related tools. for write proxies, consider turning off for
+     * performance.
+     *
+     * @return true if it needs to check existence of a log when querying local cache misses. otherwise false.
+     */
+    public boolean getFederatedCheckExistenceWhenCacheMiss() {
+        return getBoolean(BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS,
+                BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS_DEFAULT);
+    }
+
+    /**
+     * Enable check existence of a log if quering local cache of a federated namespace missed.
+     *
+     * @param enabled
+     *          flag to enable/disable this feature.
+     * @return distributedlog configuration.
+     */
+    public DistributedLogConfiguration setFederatedCheckExistenceWhenCacheMiss(boolean enabled) {
+        setProperty(BKDL_FEDERATED_CHECK_EXISTENCE_WHEN_CACHE_MISS, enabled);
+        return this;
+    }
+
+    //
+    // Settings for Configurations
+    //
+
+    /**
+     * Get dynamic configuration reload interval in seconds.
+     *
+     * @return dynamic configuration reload interval
+     */
+    public int getDynamicConfigReloadIntervalSec() {
+        return getInt(BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC, BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC_DEFAULT);
+    }
+
+    /**
+     * Get dynamic configuration reload interval in seconds.
+     *
+     * @param intervalSec dynamic configuration reload interval in seconds
+     * @return distributedlog configuration.
+     */
+    public DistributedLogConfiguration setDynamicConfigReloadIntervalSec(int intervalSec) {
+        setProperty(BKDL_DYNAMIC_CONFIG_RELOAD_INTERVAL_SEC, intervalSec);
+        return this;
+    }
+
+    /**
+     * Get config router class which determines how stream name is mapped to configuration.
+     *
+     * @return config router class.
+     */
+    public String getStreamConfigRouterClass() {
+        return getString(BKDL_STREAM_CONFIG_ROUTER_CLASS, BKDL_STREAM_CONFIG_ROUTER_CLASS_DEFAULT);
+    }
+
+    /**
+     * Set config router class.
+     *
+     * @param routerClass
+     *          config router class.
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setStreamConfigRouterClass(String routerClass) {
+        setProperty(BKDL_STREAM_CONFIG_ROUTER_CLASS, routerClass);
+        return this;
+    }
+
+    //
+    // Settings for RateLimit
+    //
+
+    /**
+     * A lower threshold bytes per second limit on writes to the distributedlog proxy.
+     *
+     * @return Bytes per second write limit
+     */
+    public int getBpsSoftWriteLimit() {
+        return getInt(BKDL_BPS_SOFT_WRITE_LIMIT, BKDL_BPS_SOFT_WRITE_LIMIT_DEFAULT);
+    }
+
+    /**
+     * An upper threshold bytes per second limit on writes to the distributedlog proxy.
+     *
+     * @return Bytes per second write limit
+     */
+    public int getBpsHardWriteLimit() {
+        return getInt(BKDL_BPS_HARD_WRITE_LIMIT, BKDL_BPS_HARD_WRITE_LIMIT_DEFAULT);
+    }
+
+    /**
+     * A lower threshold requests per second limit on writes to the distributedlog proxy.
+     *
+     * @return Requests per second write limit
+     */
+    public int getRpsSoftWriteLimit() {
+        return getInt(BKDL_RPS_SOFT_WRITE_LIMIT, BKDL_RPS_SOFT_WRITE_LIMIT_DEFAULT);
+    }
+
+    /**
+     * An upper threshold requests per second limit on writes to the distributedlog proxy.
+     *
+     * @return Requests per second write limit
+     */
+    public int getRpsHardWriteLimit() {
+        return getInt(BKDL_RPS_HARD_WRITE_LIMIT, BKDL_RPS_HARD_WRITE_LIMIT_DEFAULT);
+    }
+
+    //
+    // Settings for partitioning
+    //
+
+    /**
+     * Get the maximum number of partitions of each stream allowed to be acquired per proxy.
+     * <p>This setting is able to configure per stream. This is the default setting if it is
+     * not configured per stream. Default value is -1, which means no limit on the number of
+     * partitions could be acquired each stream.
+     *
+     * @return maximum number of partitions of each stream allowed to be acquired per proxy.
+     */
+    public int getMaxAcquiredPartitionsPerProxy() {
+        return getInt(BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY, BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY_DEFAULT);
+    }
+
+    /**
+     * Set the maximum number of partitions of each stream allowed to be acquired per proxy.
+     *
+     * @param numPartitions
+     *          number of partitions of each stream allowed to be acquired
+     * @return distributedlog configuration
+     * @see #getMaxAcquiredPartitionsPerProxy()
+     */
+    public DistributedLogConfiguration setMaxAcquiredPartitionsPerProxy(int numPartitions) {
+        setProperty(BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY, numPartitions);
+        return this;
+    }
+
+    /**
+     * Get the maximum number of partitions of each stream allowed to cache per proxy.
+     * <p>This setting is able to configure per stream. This is the default setting if it is
+     * not configured per stream. Default value is -1, which means no limit on the number of
+     * partitions could be acquired each stream.
+     *
+     * @return maximum number of partitions of each stream allowed to be acquired per proxy.
+     */
+    public int getMaxCachedPartitionsPerProxy() {
+        return getInt(BKDL_MAX_CACHED_PARTITIONS_PER_PROXY, BKDL_MAX_CACHED_PARTITIONS_PER_PROXY_DEFAULT);
+    }
+
+    /**
+     * Set the maximum number of partitions of each stream allowed to cache per proxy.
+     *
+     * @param numPartitions
+     *          number of partitions of each stream allowed to cache
+     * @return distributedlog configuration
+     * @see #getMaxAcquiredPartitionsPerProxy()
+     */
+    public DistributedLogConfiguration setMaxCachedPartitionsPerProxy(int numPartitions) {
+        setProperty(BKDL_MAX_CACHED_PARTITIONS_PER_PROXY, numPartitions);
+        return this;
+    }
+
+    // Error Injection Settings
+
+    /**
+     * Should we enable write delay injection? If false we won't check other write delay settings.
+     *
+     * @return true if write delay injection is enabled.
+     */
+    public boolean getEIInjectWriteDelay() {
+        return getBoolean(BKDL_EI_INJECT_WRITE_DELAY, BKDL_EI_INJECT_WRITE_DELAY_DEFAULT);
+    }
+
+    /**
+     * Get percent of write requests which should be delayed by BKDL_EI_INJECTED_WRITE_DELAY_MS.
+     *
+     * @return percent of writes to delay.
+     */
+    public double getEIInjectedWriteDelayPercent() {
+        return getDouble(BKDL_EI_INJECTED_WRITE_DELAY_PERCENT, BKDL_EI_INJECTED_WRITE_DELAY_PERCENT_DEFAULT);
+    }
+
+    /**
+     * Set percent of write requests which should be delayed by BKDL_EI_INJECTED_WRITE_DELAY_MS. 0 disables
+     * write delay.
+     *
+     * @param percent
+     *          percent of writes to delay.
+     * @return dl configuration.
+     */
+    public DistributedLogConfiguration setEIInjectedWriteDelayPercent(double percent) {
+        setProperty(BKDL_EI_INJECTED_WRITE_DELAY_PERCENT, percent);
+        return this;
+    }
+
+    /**
+     * Get amount of time to delay writes for in writer failure injection.
+     *
+     * @return millis to delay writes for.
+     */
+    public int getEIInjectedWriteDelayMs() {
+        return getInt(BKDL_EI_INJECTED_WRITE_DELAY_MS, BKDL_EI_INJECTED_WRITE_DELAY_MS_DEFAULT);
+    }
+
+    /**
+     * Set amount of time to delay writes for in writer failure injection. 0 disables write delay.
+     *
+     * @param delayMs
+     *          ms to delay writes for.
+     * @return dl configuration.
+     */
+    public DistributedLogConfiguration setEIInjectedWriteDelayMs(int delayMs) {
+        setProperty(BKDL_EI_INJECTED_WRITE_DELAY_MS, delayMs);
+        return this;
+    }
+
+    /**
+     * Get the flag whether to inject stalls in read ahead.
+     *
+     * @return true if to inject stalls in read ahead, otherwise false.
+     */
+    public boolean getEIInjectReadAheadStall() {
+        return getBoolean(BKDL_EI_INJECT_READAHEAD_STALL, BKDL_EI_INJECT_READAHEAD_STALL_DEFAULT);
+    }
+
+    /**
+     * Set the flag whether to inject stalls in read ahead.
+     *
+     * @param enabled
+     *          flag to inject stalls in read ahead.
+     * @return distributedlog configuration.
+     */
+    public DistributedLogConfiguration setEIInjectReadAheadStall(boolean enabled) {
+        setProperty(BKDL_EI_INJECT_READAHEAD_STALL, enabled);
+        return this;
+    }
+
+    /**
+     * Get the flag whether to inject broken entries in readahead.
+     *
+     * @return true if to inject corruption in read ahead, otherwise false.
+     */
+    public boolean getEIInjectReadAheadBrokenEntries() {
+        return getBoolean(BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES, BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES_DEFAULT);
+    }
+
+    /**
+     * Set the flag whether to inject broken entries in read ahead.
+     *
+     * @param enabled
+     *          flag to inject corruption in read ahead.
+     * @return distributedlog configuration.
+     */
+    public DistributedLogConfiguration setEIInjectReadAheadBrokenEntries(boolean enabled) {
+        setProperty(BKDL_EI_INJECT_READAHEAD_BROKEN_ENTRIES, enabled);
+        return this;
+    }
+
+    /**
+     * Get the flag whether to inject delay in read ahead.
+     *
+     * @return true if to inject delays in read ahead, otherwise false.
+     */
+    public boolean getEIInjectReadAheadDelay() {
+        return getBoolean(BKDL_EI_INJECT_READAHEAD_DELAY, BKDL_EI_INJECT_READAHEAD_DELAY_DEFAULT);
+    }
+
+    /**
+     * Set the flag whether to inject delays in read ahead.
+     *
+     * @param enabled
+     *          flag to inject delays in read ahead.
+     * @return distributedlog configuration.
+     */
+    public DistributedLogConfiguration setEIInjectReadAheadDelay(boolean enabled) {
+        setProperty(BKDL_EI_INJECT_READAHEAD_DELAY, enabled);
+        return this;
+    }
+
+    /**
+     * Get the max injected delay in read ahead, in millis.
+     *
+     * @return max injected delay in read ahead, in millis.
+     */
+    public int getEIInjectMaxReadAheadDelayMs() {
+        return getInt(BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS, BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS_DEFAULT);
+    }
+
+    /**
+     * Set the max injected delay in read ahead, in millis.
+     *
+     * @param delayMs
+     *          max injected delay in read ahead, in millis.
+     * @return distributedlog configuration.
+     */
+    public DistributedLogConfiguration setEIInjectMaxReadAheadDelayMs(int delayMs) {
+        setProperty(BKDL_EI_INJECT_MAX_READAHEAD_DELAY_MS, delayMs);
+        return this;
+    }
+
+    /**
+     * Get the percentage of operations to delay in read ahead.
+     *
+     * @return the percentage of operations to delay in read ahead.
+     */
+    public int getEIInjectReadAheadDelayPercent() {
+        return getInt(BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT, BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT_DEFAULT);
+    }
+
+    /**
+     * Set the percentage of operations to delay in read ahead.
+     *
+     * @param percent
+     *          the percentage of operations to delay in read ahead.
+     * @return distributedlog configuration
+     */
+    public DistributedLogConfiguration setEIInjectReadAheadDelayPercent(int percent) {
+        setProperty(BKDL_EI_INJECT_READAHEAD_DELAY_PERCENT, percent);
+        return this;
+    }
+
+    /**
+     * Validate the configuration
+     */
+    public void validate() {
+        Preconditions.checkArgument(getBKClientReadTimeout() * 1000 >= getReadLACLongPollTimeout(),
+            "Invalid timeout configuration: bkcReadTimeoutSeconds ("+getBKClientReadTimeout()+
+                ") should be longer than readLACLongPollTimeout ("+getReadLACLongPollTimeout()+")");
+        long readerIdleWarnThresholdMs = getReaderIdleWarnThresholdMillis();
+        if (readerIdleWarnThresholdMs > 0) { // NOTE: some test cases set the idle warn threshold to 0
+            Preconditions.checkArgument(readerIdleWarnThresholdMs > 2 * getReadLACLongPollTimeout(),
+                    "Invalid configuration: ReaderIdleWarnThreshold should be 2x larget than readLACLongPollTimeout");
+        }
+    }
+
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java
new file mode 100644
index 0000000..f951991
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConstants.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;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.data.ACL;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+public class DistributedLogConstants {
+    public static final byte[] EMPTY_BYTES = new byte[0];
+    public static final String SCHEME_PREFIX = "distributedlog";
+    public static final String BACKEND_BK = "bk";
+    public static final long INVALID_TXID = -999;
+    public static final long EMPTY_LOGSEGMENT_TX_ID = -99;
+    public static final long MAX_TXID = Long.MAX_VALUE;
+    public static final long SMALL_LOGSEGMENT_THRESHOLD = 10;
+    public static final int LOGSEGMENT_NAME_VERSION = 1;
+    public static final int FUTURE_TIMEOUT_IMMEDIATE = 0;
+    public static final int FUTURE_TIMEOUT_INFINITE = -1;
+    public static final long LOCK_IMMEDIATE = FUTURE_TIMEOUT_IMMEDIATE;
+    public static final long LOCK_TIMEOUT_INFINITE = FUTURE_TIMEOUT_INFINITE;
+    public static final long LOCK_OP_TIMEOUT_DEFAULT = 120;
+    public static final long LOCK_REACQUIRE_TIMEOUT_DEFAULT = 120;
+    public static final String UNKNOWN_CLIENT_ID = "Unknown-ClientId";
+    public static final int LOCAL_REGION_ID = 0;
+    public static final long LOGSEGMENT_DEFAULT_STATUS = 0;
+    public static final long UNASSIGNED_LOGSEGMENT_SEQNO = 0;
+    public static final long UNASSIGNED_SEQUENCE_ID = -1L;
+    public static final long FIRST_LOGSEGMENT_SEQNO = 1;
+    public static final long UNRESOLVED_LEDGER_ID = -1;
+    public static final long LATENCY_WARN_THRESHOLD_IN_MILLIS = TimeUnit.SECONDS.toMillis(1);
+    public static final int DL_INTERRUPTED_EXCEPTION_RESULT_CODE = Integer.MIN_VALUE + 1;
+    public static final int ZK_CONNECTION_EXCEPTION_RESULT_CODE = Integer.MIN_VALUE + 2;
+
+    public static final String ALLOCATION_POOL_NODE = ".allocation_pool";
+    // log segment prefix
+    public static final String INPROGRESS_LOGSEGMENT_PREFIX = "inprogress";
+    public static final String COMPLETED_LOGSEGMENT_PREFIX = "logrecs";
+    public static final String DISALLOW_PLACEMENT_IN_REGION_FEATURE_NAME = "disallow_bookie_placement";
+    static final byte[] CONTROL_RECORD_CONTENT = "control".getBytes(UTF_8);
+    static final byte[] KEEPALIVE_RECORD_CONTENT = "keepalive".getBytes(UTF_8);
+
+    // An ACL that gives all permissions to node creators and read permissions only to everyone else.
+    public static final List<ACL> EVERYONE_READ_CREATOR_ALL =
+        ImmutableList.<ACL>builder()
+            .addAll(Ids.CREATOR_ALL_ACL)
+            .addAll(Ids.READ_ACL_UNSAFE)
+            .build();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java
new file mode 100644
index 0000000..7d33e9c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java
@@ -0,0 +1,308 @@
+/**
+ * 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 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;
+
+/**
+ * A DistributedLogManager is responsible for managing a single place of storing
+ * edit logs. It may correspond to multiple files, a backup node, etc.
+ * Even when the actual underlying storage is rolled, or failed and restored,
+ * 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 {
+
+    /**
+     * Get the name of the stream managed by this log manager
+     * @return streamName
+     */
+    public String getStreamName();
+
+    /**
+     * Get the namespace driver used by this manager.
+     *
+     * @return the namespace driver
+     */
+    public NamespaceDriver getNamespaceDriver();
+
+    /**
+     * Get log segments.
+     *
+     * @return log segments
+     * @throws IOException
+     */
+    public List<LogSegmentMetadata> getLogSegments() throws IOException;
+
+    /**
+     * Register <i>listener</i> on log segment updates of this stream.
+     *
+     * @param listener
+     *          listener to receive update log segment list.
+     */
+    public void registerListener(LogSegmentListener listener) throws IOException ;
+
+    /**
+     * Unregister <i>listener</i> on log segment updates from this stream.
+     *
+     * @param listener
+     *          listener to receive update log segment list.
+     */
+    public 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();
+
+    /**
+     * Begin writing to the log stream identified by the name
+     *
+     * @return the writer interface to generate log records
+     */
+    public LogWriter startLogSegmentNonPartitioned() throws IOException;
+
+    /**
+     * Begin writing to the log stream identified by the name
+     *
+     * @return the writer interface to generate log records
+     */
+    // @Deprecated
+    public AsyncLogWriter startAsyncLogSegmentNonPartitioned() throws IOException;
+
+    /**
+     * 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;
+
+    /**
+     * 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;
+
+    /**
+     * 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)
+        throws IOException;
+
+    public LogReader getInputStream(DLSN fromDLSN) throws IOException;
+
+    /**
+     * Open an async log reader to read records from a log starting from <code>fromTxnId</code>.
+     *
+     * @param fromTxnId
+     *          transaction id to start reading from
+     * @return async log reader
+     */
+    public Future<AsyncLogReader> openAsyncLogReader(long fromTxnId);
+
+    /**
+     * 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);
+
+    // @Deprecated
+    public AsyncLogReader getAsyncLogReader(long fromTxnId) throws IOException;
+
+    // @Deprecated
+    public AsyncLogReader getAsyncLogReader(DLSN fromDLSN) throws IOException;
+
+    public Future<AsyncLogReader> getAsyncLogReaderWithLock(DLSN fromDLSN);
+
+    /**
+     * Get a log reader with lock starting from <i>fromDLSN</i> and using <i>subscriberId</i>.
+     * If two readers tried to open using same subscriberId, one would succeed, while the other
+     * will be blocked until it gets the lock.
+     *
+     * @param fromDLSN
+     *          start dlsn
+     * @param subscriberId
+     *          subscriber id
+     * @return async log reader
+     */
+    public 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
+     * will be blocked until it gets the lock.
+     *
+     * @param subscriberId
+     *          subscriber id
+     * @return async log reader
+     */
+    public Future<AsyncLogReader> getAsyncLogReaderWithLock(String subscriberId);
+
+    /**
+     * Get the {@link DLSN} of first log record whose transaction id is not less than <code>transactionId</code>.
+     *
+     * @param transactionId
+     *          transaction id
+     * @return dlsn of first log record whose transaction id is not less than transactionId.
+     */
+    public Future<DLSN> getDLSNNotLessThanTxId(long transactionId);
+
+    /**
+     * 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()
+        throws IOException;
+
+    /**
+     * Get the earliest Transaction Id available in the log
+     *
+     * @return earliest transaction id
+     * @throws IOException
+     */
+    public long getFirstTxId() throws IOException;
+
+    /**
+     * Get Latest Transaction Id in the log
+     *
+     * @return latest transaction id
+     * @throws IOException
+     */
+    public long getLastTxId() throws IOException;
+
+    /**
+     * Get Latest DLSN in the log
+     *
+     * @return last dlsn
+     * @throws IOException
+     */
+    public DLSN getLastDLSN() throws IOException;
+
+    /**
+     * Get Latest log record with DLSN in the log - async
+     *
+     * @return latest log record with DLSN
+     */
+    public Future<LogRecordWithDLSN> getLastLogRecordAsync();
+
+    /**
+     * Get Latest Transaction Id in the log - async
+     *
+     * @return latest transaction id
+     */
+    public Future<Long> getLastTxIdAsync();
+
+    /**
+     * Get first DLSN in the log.
+     *
+     * @return first dlsn in the stream
+     */
+    public Future<DLSN> getFirstDLSNAsync();
+
+    /**
+     * Get Latest DLSN in the log - async
+     *
+     * @return latest transaction id
+     */
+    public 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
+     *
+     * @return number of log records
+     * @throws IOException
+     */
+    public 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
+     *
+     * @return future number of log records
+     * @throws IOException
+     */
+    public Future<Long> getLogRecordCountAsync(final DLSN beginDLSN);
+
+    /**
+     * Run recovery on the log.
+     *
+     * @throws IOException
+     */
+    public 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
+     *
+     * @return true if the marker was added to the stream, false otherwise
+     * @throws IOException
+     */
+    public boolean isEndOfStreamMarked() throws IOException;
+
+    /**
+     * Delete the log.
+     *
+     * @throws IOException if the deletion fails
+     */
+    public void delete() throws IOException;
+
+    /**
+     * The DistributedLogManager may archive/purge any logs for transactionId
+     * less than or equal to minImageTxId.
+     * This is to be used only when the client explicitly manages deletion. If
+     * the cleanup policy is based on sliding time window, then this method need
+     * not be called.
+     *
+     * @param minTxIdToKeep the earliest txid that must be retained
+     * @throws IOException if purging fails
+     */
+    public 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();
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java b/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java
new file mode 100644
index 0000000..617282c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java
@@ -0,0 +1,403 @@
+/**
+ * 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.base.Optional;
+import com.google.common.base.Preconditions;
+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;
+
+import javax.annotation.Nullable;
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * A set of {@link LogRecord}s.
+ */
+public class Entry {
+
+    /**
+     * Create a new log record set.
+     *
+     * @param logName
+     *          name of the log
+     * @param initialBufferSize
+     *          initial buffer size
+     * @param envelopeBeforeTransmit
+     *          if envelope the buffer before transmit
+     * @param codec
+     *          compression codec
+     * @param statsLogger
+     *          stats logger to receive stats
+     * @return writer to build a log record set.
+     */
+    public static Writer newEntry(
+            String logName,
+            int initialBufferSize,
+            boolean envelopeBeforeTransmit,
+            CompressionCodec.Type codec,
+            StatsLogger statsLogger) {
+        return new EnvelopedEntryWriter(
+                logName,
+                initialBufferSize,
+                envelopeBeforeTransmit,
+                codec,
+                statsLogger);
+    }
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    /**
+     * Build the record set object.
+     */
+    public static class Builder {
+
+        private long logSegmentSequenceNumber = -1;
+        private long entryId = -1;
+        private long startSequenceId = Long.MIN_VALUE;
+        private boolean envelopeEntry = true;
+        // input stream
+        private InputStream in = null;
+        // or bytes array
+        private byte[] data = null;
+        private int offset = -1;
+        private int length = -1;
+        private Optional<Long> txidToSkipTo = Optional.absent();
+        private Optional<DLSN> dlsnToSkipTo = Optional.absent();
+        private boolean deserializeRecordSet = true;
+
+        private Builder() {}
+
+        /**
+         * Reset the builder.
+         *
+         * @return builder
+         */
+        public Builder reset() {
+            logSegmentSequenceNumber = -1;
+            entryId = -1;
+            startSequenceId = Long.MIN_VALUE;
+            envelopeEntry = true;
+            // input stream
+            in = null;
+            // or bytes array
+            data = null;
+            offset = -1;
+            length = -1;
+            txidToSkipTo = Optional.absent();
+            dlsnToSkipTo = Optional.absent();
+            return this;
+        }
+
+        /**
+         * Set the segment info of the log segment that this record
+         * set belongs to.
+         *
+         * @param lssn
+         *          log segment sequence number
+         * @param startSequenceId
+         *          start sequence id of this log segment
+         * @return builder
+         */
+        public Builder setLogSegmentInfo(long lssn, long startSequenceId) {
+            this.logSegmentSequenceNumber = lssn;
+            this.startSequenceId = startSequenceId;
+            return this;
+        }
+
+        /**
+         * Set the entry id of this log record set.
+         *
+         * @param entryId
+         *          entry id assigned for this log record set.
+         * @return builder
+         */
+        public Builder setEntryId(long entryId) {
+            this.entryId = entryId;
+            return this;
+        }
+
+        /**
+         * Set whether this record set is enveloped or not.
+         *
+         * @param enabled
+         *          flag indicates whether this record set is enveloped or not.
+         * @return builder
+         */
+        public Builder setEnvelopeEntry(boolean enabled) {
+            this.envelopeEntry = enabled;
+            return this;
+        }
+
+        /**
+         * Set the serialized bytes data of this record set.
+         *
+         * @param data
+         *          serialized bytes data of this record set.
+         * @param offset
+         *          offset of the bytes data
+         * @param length
+         *          length of the bytes data
+         * @return builder
+         */
+        public Builder setData(byte[] data, int offset, int length) {
+            this.data = data;
+            this.offset = offset;
+            this.length = length;
+            return this;
+        }
+
+        /**
+         * Set the input stream of the serialized bytes data of this record set.
+         *
+         * @param in
+         *          input stream
+         * @return builder
+         */
+        public Builder setInputStream(InputStream in) {
+            this.in = in;
+            return this;
+        }
+
+        /**
+         * Set the record set starts from <code>dlsn</code>.
+         *
+         * @param dlsn
+         *          dlsn to skip to
+         * @return builder
+         */
+        public Builder skipTo(@Nullable DLSN dlsn) {
+            this.dlsnToSkipTo = Optional.fromNullable(dlsn);
+            return this;
+        }
+
+        /**
+         * Set the record set starts from <code>txid</code>.
+         *
+         * @param txid
+         *          txid to skip to
+         * @return builder
+         */
+        public Builder skipTo(long txid) {
+            this.txidToSkipTo = Optional.of(txid);
+            return this;
+        }
+
+        /**
+         * Enable/disable deserialize record set.
+         *
+         * @param enabled
+         *          flag to enable/disable dserialize record set.
+         * @return builder
+         */
+        public Builder deserializeRecordSet(boolean enabled) {
+            this.deserializeRecordSet = enabled;
+            return this;
+        }
+
+        public Entry build() {
+            Preconditions.checkNotNull(data, "Serialized data isn't provided");
+            Preconditions.checkArgument(offset >= 0 && length >= 0
+                    && (offset + length) <= data.length,
+                    "Invalid offset or length of serialized data");
+            return new Entry(
+                    logSegmentSequenceNumber,
+                    entryId,
+                    startSequenceId,
+                    envelopeEntry,
+                    deserializeRecordSet,
+                    data,
+                    offset,
+                    length,
+                    txidToSkipTo,
+                    dlsnToSkipTo);
+        }
+
+        public Entry.Reader buildReader() throws IOException {
+            Preconditions.checkArgument(data != null || in != null,
+                    "Serialized data or input stream isn't provided");
+            InputStream in;
+            if (null != this.in) {
+                in = this.in;
+            } else {
+                Preconditions.checkArgument(offset >= 0 && length >= 0
+                                && (offset + length) <= data.length,
+                        "Invalid offset or length of serialized data");
+                in = new ByteArrayInputStream(data, offset, length);
+            }
+            return new EnvelopedEntryReader(
+                    logSegmentSequenceNumber,
+                    entryId,
+                    startSequenceId,
+                    in,
+                    envelopeEntry,
+                    deserializeRecordSet,
+                    NullStatsLogger.INSTANCE);
+        }
+
+    }
+
+    private final long logSegmentSequenceNumber;
+    private final long entryId;
+    private final long startSequenceId;
+    private final boolean envelopedEntry;
+    private final boolean deserializeRecordSet;
+    private final byte[] data;
+    private final int offset;
+    private final int length;
+    private final Optional<Long> txidToSkipTo;
+    private final Optional<DLSN> dlsnToSkipTo;
+
+    private Entry(long logSegmentSequenceNumber,
+                  long entryId,
+                  long startSequenceId,
+                  boolean envelopedEntry,
+                  boolean deserializeRecordSet,
+                  byte[] data,
+                  int offset,
+                  int length,
+                  Optional<Long> txidToSkipTo,
+                  Optional<DLSN> dlsnToSkipTo) {
+        this.logSegmentSequenceNumber = logSegmentSequenceNumber;
+        this.entryId = entryId;
+        this.startSequenceId = startSequenceId;
+        this.envelopedEntry = envelopedEntry;
+        this.deserializeRecordSet = deserializeRecordSet;
+        this.data = data;
+        this.offset = offset;
+        this.length = length;
+        this.txidToSkipTo = txidToSkipTo;
+        this.dlsnToSkipTo = dlsnToSkipTo;
+    }
+
+    /**
+     * Get raw data of this record set.
+     *
+     * @return raw data representation of this record set.
+     */
+    public byte[] getRawData() {
+        return data;
+    }
+
+    /**
+     * Create reader to iterate over this record set.
+     *
+     * @return reader to iterate over this record set.
+     * @throws IOException if the record set is invalid record set.
+     */
+    public Reader reader() throws IOException {
+        InputStream in = new ByteArrayInputStream(data, offset, length);
+        Reader reader = new EnvelopedEntryReader(
+                logSegmentSequenceNumber,
+                entryId,
+                startSequenceId,
+                in,
+                envelopedEntry,
+                deserializeRecordSet,
+                NullStatsLogger.INSTANCE);
+        if (txidToSkipTo.isPresent()) {
+            reader.skipTo(txidToSkipTo.get());
+        }
+        if (dlsnToSkipTo.isPresent()) {
+            reader.skipTo(dlsnToSkipTo.get());
+        }
+        return reader;
+    }
+
+    /**
+     * Writer to append {@link LogRecord}s to {@link Entry}.
+     */
+    public interface Writer extends EntryBuffer {
+
+        /**
+         * Write a {@link LogRecord} to this record set.
+         *
+         * @param record
+         *          record to write
+         * @param transmitPromise
+         *          callback for transmit result. the promise is only
+         *          satisfied when this record set is transmitted.
+         * @throws LogRecordTooLongException if the record is too long
+         * @throws WriteException when encountered exception writing the record
+         */
+        void writeRecord(LogRecord record, Promise<DLSN> transmitPromise)
+                throws LogRecordTooLongException, WriteException;
+
+        /**
+         * Reset the writer to write records.
+         */
+        void reset();
+
+    }
+
+    /**
+     * Reader to read {@link LogRecord}s from this record set.
+     */
+    public interface Reader {
+
+        /**
+         * Get the log segment sequence number.
+         *
+         * @return the log segment sequence number.
+         */
+        long getLSSN();
+
+        /**
+         * Return the entry id.
+         *
+         * @return the entry id.
+         */
+        long getEntryId();
+
+        /**
+         * Read next log record from this record set.
+         *
+         * @return next log record from this record set.
+         */
+        LogRecordWithDLSN nextRecord() throws IOException;
+
+        /**
+         * Skip the reader to the record whose transaction id is <code>txId</code>.
+         *
+         * @param txId
+         *          transaction id to skip to.
+         * @return true if skip succeeds, otherwise false.
+         * @throws IOException
+         */
+        boolean skipTo(long txId) throws IOException;
+
+        /**
+         * Skip the reader to the record whose DLSN is <code>dlsn</code>.
+         *
+         * @param dlsn
+         *          DLSN to skip to.
+         * @return true if skip succeeds, otherwise false.
+         * @throws IOException
+         */
+        boolean skipTo(DLSN dlsn) throws IOException;
+
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EntryBuffer.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EntryBuffer.java
new file mode 100644
index 0000000..c695420
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EntryBuffer.java
@@ -0,0 +1,70 @@
+/**
+ * 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 org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException;
+import org.apache.distributedlog.io.Buffer;
+import org.apache.distributedlog.io.TransmitListener;
+
+import java.io.IOException;
+
+/**
+ * Write representation of a {@link Entry}.
+ * It is a buffer of log record set, used for transmission.
+ */
+public interface EntryBuffer extends TransmitListener {
+
+    /**
+     * Return if this record set contains user records.
+     *
+     * @return true if this record set contains user records, otherwise
+     * return false.
+     */
+    boolean hasUserRecords();
+
+    /**
+     * Return number of records in current record set.
+     *
+     * @return number of records in current record set.
+     */
+    int getNumRecords();
+
+    /**
+     * Return number of bytes in current record set.
+     *
+     * @return number of bytes in current record set.
+     */
+    int getNumBytes();
+
+    /**
+     * Return max tx id in current record set.
+     *
+     * @return max tx id.
+     */
+    long getMaxTxId();
+
+    /**
+     * Get the buffer to transmit.
+     *
+     * @return the buffer to transmit.
+     * @throws InvalidEnvelopedEntryException if the record set buffer is invalid
+     * @throws IOException when encountered IOException during serialization
+     */
+    Buffer getBuffer() throws InvalidEnvelopedEntryException, IOException;
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EntryPosition.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EntryPosition.java
new file mode 100644
index 0000000..218662c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EntryPosition.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;
+
+/**
+ * The position of an entry, identified by log segment sequence number and entry id.
+ */
+class EntryPosition {
+
+    private long lssn;
+    private long entryId;
+
+    EntryPosition(long lssn, long entryId) {
+        this.lssn = lssn;
+        this.entryId = entryId;
+    }
+
+    public synchronized long getLogSegmentSequenceNumber() {
+        return lssn;
+    }
+
+    public synchronized long getEntryId() {
+        return entryId;
+    }
+
+    public synchronized boolean advance(long lssn, long entryId) {
+        if (lssn == this.lssn) {
+            if (entryId <= this.entryId) {
+                return false;
+            }
+            this.entryId = entryId;
+            return true;
+        } else if (lssn > this.lssn) {
+            this.lssn = lssn;
+            this.entryId = entryId;
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("(").append(lssn).append(", ").append(entryId).append(")");
+        return sb.toString();
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java
new file mode 100644
index 0000000..eb1e9af
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java
@@ -0,0 +1,296 @@
+/**
+ * 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 java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+import org.apache.distributedlog.annotations.DistributedLogAnnotations.Compression;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.io.CompressionUtils;
+import org.apache.distributedlog.util.BitMaskUtils;
+
+/**
+ * An enveloped entry written to BookKeeper.
+ *
+ * Data type in brackets. Interpretation should be on the basis of data types and not individual
+ * bytes to honor Endianness.
+ *
+ * Entry Structure:
+ * ---------------
+ * Bytes 0                                  : Version (Byte)
+ * Bytes 1 - (DATA = 1+Header.length-1)     : Header (Integer)
+ * Bytes DATA - DATA+3                      : Payload Length (Integer)
+ * BYTES DATA+4 - DATA+4+payload.length-1   : Payload (Byte[])
+ *
+ * V1 Header Structure: // Offsets relative to the start of the header.
+ * -------------------
+ * Bytes 0 - 3                              : Flags (Integer)
+ * Bytes 4 - 7                              : Original payload size before compression (Integer)
+ *
+ *      Flags: // 32 Bits
+ *      -----
+ *      0 ... 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
+ *                                      |_|
+ *                                       |
+ *                               Compression Type
+ *
+ *      Compression Type: // 2 Bits (Least significant)
+ *      ----------------
+ *      00      : No Compression
+ *      01      : LZ4 Compression
+ *      10      : Unused
+ *      11      : Unused
+ */
+public class EnvelopedEntry {
+
+    public static final int VERSION_LENGTH = 1; // One byte long
+    public static final byte VERSION_ONE = 1;
+
+    public static final byte LOWEST_SUPPORTED_VERSION = VERSION_ONE;
+    public static final byte HIGHEST_SUPPORTED_VERSION = VERSION_ONE;
+    public static final byte CURRENT_VERSION = VERSION_ONE;
+
+    private final OpStatsLogger compressionStat;
+    private final OpStatsLogger decompressionStat;
+    private final Counter compressedEntryBytes;
+    private final Counter decompressedEntryBytes;
+    private final byte version;
+
+    private Header header = new Header();
+    private Payload payloadCompressed = new Payload();
+    private Payload payloadDecompressed = new Payload();
+
+    public EnvelopedEntry(byte version,
+                          StatsLogger statsLogger) throws InvalidEnvelopedEntryException {
+        Preconditions.checkNotNull(statsLogger);
+        if (version < LOWEST_SUPPORTED_VERSION || version > HIGHEST_SUPPORTED_VERSION) {
+            throw new InvalidEnvelopedEntryException("Invalid enveloped entry version " + version + ", expected to be in [ "
+                    + LOWEST_SUPPORTED_VERSION + " ~ " + HIGHEST_SUPPORTED_VERSION + " ]");
+        }
+        this.version = version;
+        this.compressionStat = statsLogger.getOpStatsLogger("compression_time");
+        this.decompressionStat = statsLogger.getOpStatsLogger("decompression_time");
+        this.compressedEntryBytes = statsLogger.getCounter("compressed_bytes");
+        this.decompressedEntryBytes = statsLogger.getCounter("decompressed_bytes");
+    }
+
+    /**
+     * @param statsLogger
+     *          Used for getting stats for (de)compression time
+     * @param compressionType
+     *          The compression type to use
+     * @param decompressed
+     *          The decompressed payload
+     *          NOTE: The size of the byte array passed as the decompressed payload can be larger
+     *                than the actual contents to be compressed.
+     */
+    public EnvelopedEntry(byte version,
+                          CompressionCodec.Type compressionType,
+                          byte[] decompressed,
+                          int length,
+                          StatsLogger statsLogger)
+            throws InvalidEnvelopedEntryException {
+        this(version, statsLogger);
+        Preconditions.checkNotNull(compressionType);
+        Preconditions.checkNotNull(decompressed);
+        Preconditions.checkArgument(length >= 0, "Invalid bytes length " + length);
+
+        this.header = new Header(compressionType, length);
+        this.payloadDecompressed = new Payload(length, decompressed);
+    }
+
+    private boolean isReady() {
+        return (header.ready && payloadDecompressed.ready);
+    }
+
+    @Compression
+    public void writeFully(DataOutputStream out) throws IOException {
+        Preconditions.checkNotNull(out);
+        if (!isReady()) {
+            throw new IOException("Entry not writable");
+        }
+        // Version
+        out.writeByte(version);
+        // Header
+        header.write(out);
+        // Compress
+        CompressionCodec codec = CompressionUtils.getCompressionCodec(header.compressionType);
+        byte[] compressed = codec.compress(
+                payloadDecompressed.payload,
+                0,
+                payloadDecompressed.length,
+                compressionStat);
+        this.payloadCompressed = new Payload(compressed.length, compressed);
+        this.compressedEntryBytes.add(payloadCompressed.length);
+        this.decompressedEntryBytes.add(payloadDecompressed.length);
+        payloadCompressed.write(out);
+    }
+
+    @Compression
+    public void readFully(DataInputStream in) throws IOException {
+        Preconditions.checkNotNull(in);
+        // Make sure we're reading the right versioned entry.
+        byte version = in.readByte();
+        if (version != this.version) {
+            throw new IOException(String.format("Version mismatch while reading. Received: %d," +
+                    " Required: %d", version, this.version));
+        }
+        header.read(in);
+        payloadCompressed.read(in);
+        // Decompress
+        CompressionCodec codec = CompressionUtils.getCompressionCodec(header.compressionType);
+        byte[] decompressed = codec.decompress(
+                payloadCompressed.payload,
+                0,
+                payloadCompressed.length,
+                header.decompressedSize,
+                decompressionStat);
+        this.payloadDecompressed = new Payload(decompressed.length, decompressed);
+        this.compressedEntryBytes.add(payloadCompressed.length);
+        this.decompressedEntryBytes.add(payloadDecompressed.length);
+    }
+
+    public byte[] getDecompressedPayload() throws IOException {
+        if (!isReady()) {
+            throw new IOException("Decompressed payload is not initialized");
+        }
+        return payloadDecompressed.payload;
+    }
+
+    public static class Header {
+        public static final int COMPRESSION_CODEC_MASK = 0x3;
+        public static final int COMPRESSION_CODEC_NONE = 0x0;
+        public static final int COMPRESSION_CODEC_LZ4 = 0x1;
+
+        private int flags = 0;
+        private int decompressedSize = 0;
+        private CompressionCodec.Type compressionType = CompressionCodec.Type.UNKNOWN;
+
+        // Whether this struct is ready for reading/writing.
+        private boolean ready = false;
+
+        // Used while reading.
+        public Header() {
+        }
+
+        public Header(CompressionCodec.Type compressionType,
+                      int decompressedSize) {
+            this.compressionType = compressionType;
+            this.decompressedSize = decompressedSize;
+            this.flags = 0;
+            switch (compressionType) {
+                case NONE:
+                    this.flags = (int) BitMaskUtils.set(flags, COMPRESSION_CODEC_MASK,
+                                                        COMPRESSION_CODEC_NONE);
+                    break;
+                case LZ4:
+                    this.flags = (int) BitMaskUtils.set(flags, COMPRESSION_CODEC_MASK,
+                                                        COMPRESSION_CODEC_LZ4);
+                    break;
+                default:
+                    throw new RuntimeException(String.format("Unknown Compression Type: %s",
+                                                             compressionType));
+            }
+            // This can now be written.
+            this.ready = true;
+        }
+
+        private void write(DataOutputStream out) throws IOException {
+            out.writeInt(flags);
+            out.writeInt(decompressedSize);
+        }
+
+        private void read(DataInputStream in) throws IOException {
+            this.flags = in.readInt();
+            int compressionType = (int) BitMaskUtils.get(flags, COMPRESSION_CODEC_MASK);
+            if (compressionType == COMPRESSION_CODEC_NONE) {
+                this.compressionType = CompressionCodec.Type.NONE;
+            } else if (compressionType == COMPRESSION_CODEC_LZ4) {
+                this.compressionType = CompressionCodec.Type.LZ4;
+            } else {
+                throw new IOException(String.format("Unsupported Compression Type: %s",
+                                                    compressionType));
+            }
+            this.decompressedSize = in.readInt();
+            // Values can now be read.
+            this.ready = true;
+        }
+    }
+
+    public static class Payload {
+        private int length = 0;
+        private byte[] payload = null;
+
+        // Whether this struct is ready for reading/writing.
+        private boolean ready = false;
+
+        // Used for reading
+        Payload() {
+        }
+
+        Payload(int length, byte[] payload) {
+            this.length = length;
+            this.payload = payload;
+            this.ready = true;
+        }
+
+        private void write(DataOutputStream out) throws IOException {
+            out.writeInt(length);
+            out.write(payload, 0, length);
+        }
+
+        private void read(DataInputStream in) throws IOException {
+            this.length = in.readInt();
+            this.payload = new byte[length];
+            in.readFully(payload);
+            this.ready = true;
+        }
+    }
+
+    /**
+     * Return an InputStream that reads from the provided InputStream, decompresses the data
+     * and returns a new InputStream wrapping the underlying payload.
+     *
+     * Note that src is modified by this call.
+     *
+     * @return
+     *      New Input stream with the underlying payload.
+     * @throws Exception
+     */
+    public static InputStream fromInputStream(InputStream src,
+                                              StatsLogger statsLogger) throws IOException {
+        src.mark(VERSION_LENGTH);
+        byte version = new DataInputStream(src).readByte();
+        src.reset();
+        EnvelopedEntry entry = new EnvelopedEntry(version, statsLogger);
+        entry.readFully(new DataInputStream(src));
+        return new ByteArrayInputStream(entry.getDecompressedPayload());
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryReader.java
new file mode 100644
index 0000000..1761de5
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryReader.java
@@ -0,0 +1,102 @@
+/**
+ * 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 org.apache.bookkeeper.stats.StatsLogger;
+
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+
+/**
+ * Record reader to read records from an enveloped entry buffer.
+ */
+class EnvelopedEntryReader implements Entry.Reader, RecordStream {
+
+    private final long logSegmentSeqNo;
+    private final long entryId;
+    private final LogRecord.Reader reader;
+
+    // slot id
+    private long slotId = 0;
+
+    EnvelopedEntryReader(long logSegmentSeqNo,
+                         long entryId,
+                         long startSequenceId,
+                         InputStream in,
+                         boolean envelopedEntry,
+                         boolean deserializeRecordSet,
+                         StatsLogger statsLogger)
+            throws IOException {
+        this.logSegmentSeqNo = logSegmentSeqNo;
+        this.entryId = entryId;
+        InputStream src = in;
+        if (envelopedEntry) {
+            src = EnvelopedEntry.fromInputStream(in, statsLogger);
+        }
+        this.reader = new LogRecord.Reader(
+                this,
+                new DataInputStream(src),
+                startSequenceId,
+                deserializeRecordSet);
+    }
+
+    @Override
+    public long getLSSN() {
+        return logSegmentSeqNo;
+    }
+
+    @Override
+    public long getEntryId() {
+        return entryId;
+    }
+
+    @Override
+    public LogRecordWithDLSN nextRecord() throws IOException {
+        return reader.readOp();
+    }
+
+    @Override
+    public boolean skipTo(long txId) throws IOException {
+        return reader.skipTo(txId, true);
+    }
+
+    @Override
+    public boolean skipTo(DLSN dlsn) throws IOException {
+        return reader.skipTo(dlsn);
+    }
+
+    //
+    // Record Stream
+    //
+
+    @Override
+    public void advance(int numRecords) {
+        slotId += numRecords;
+    }
+
+    @Override
+    public DLSN getCurrentPosition() {
+        return new DLSN(logSegmentSeqNo, entryId, slotId);
+    }
+
+    @Override
+    public String getName() {
+        return "EnvelopedReader";
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java
new file mode 100644
index 0000000..54858d7
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java
@@ -0,0 +1,192 @@
+/**
+ * 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 org.apache.distributedlog.Entry.Writer;
+import org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.WriteCancelledException;
+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;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.util.LinkedList;
+import java.util.List;
+
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+
+/**
+ * {@link org.apache.distributedlog.io.Buffer} based log record set writer.
+ */
+class EnvelopedEntryWriter implements Writer {
+
+    static final Logger logger = LoggerFactory.getLogger(EnvelopedEntryWriter.class);
+
+    private static class WriteRequest {
+
+        private final int numRecords;
+        private final Promise<DLSN> promise;
+
+        WriteRequest(int numRecords, Promise<DLSN> promise) {
+            this.numRecords = numRecords;
+            this.promise = promise;
+        }
+
+    }
+
+    private final String logName;
+    private final Buffer buffer;
+    private final LogRecord.Writer writer;
+    private final List<WriteRequest> writeRequests;
+    private final boolean envelopeBeforeTransmit;
+    private final CompressionCodec.Type codec;
+    private final StatsLogger statsLogger;
+    private int count = 0;
+    private boolean hasUserData = false;
+    private long maxTxId = Long.MIN_VALUE;
+
+    EnvelopedEntryWriter(String logName,
+                         int initialBufferSize,
+                         boolean envelopeBeforeTransmit,
+                         CompressionCodec.Type codec,
+                         StatsLogger statsLogger) {
+        this.logName = logName;
+        this.buffer = new Buffer(initialBufferSize * 6 / 5);
+        this.writer = new LogRecord.Writer(new DataOutputStream(buffer));
+        this.writeRequests = new LinkedList<WriteRequest>();
+        this.envelopeBeforeTransmit = envelopeBeforeTransmit;
+        this.codec = codec;
+        this.statsLogger = statsLogger;
+    }
+
+    @Override
+    public synchronized void reset() {
+        cancelPromises(new WriteCancelledException(logName, "Record Set is reset"));
+        count = 0;
+        this.buffer.reset();
+    }
+
+    @Override
+    public synchronized void writeRecord(LogRecord record,
+                                         Promise<DLSN> transmitPromise)
+            throws LogRecordTooLongException, WriteException {
+        int logRecordSize = record.getPersistentSize();
+        if (logRecordSize > MAX_LOGRECORD_SIZE) {
+            throw new LogRecordTooLongException(
+                    "Log Record of size " + logRecordSize + " written when only "
+                            + MAX_LOGRECORD_SIZE + " is allowed");
+        }
+
+        try {
+            this.writer.writeOp(record);
+            int numRecords = 1;
+            if (!record.isControl()) {
+                hasUserData = true;
+            }
+            if (record.isRecordSet()) {
+                numRecords = LogRecordSet.numRecords(record);
+            }
+            count += numRecords;
+            writeRequests.add(new WriteRequest(numRecords, transmitPromise));
+            maxTxId = Math.max(maxTxId, record.getTransactionId());
+        } catch (IOException e) {
+            logger.error("Failed to append record to record set of {} : ",
+                    logName, e);
+            throw new WriteException(logName, "Failed to append record to record set of "
+                    + logName);
+        }
+    }
+
+    private synchronized void satisfyPromises(long lssn, long entryId) {
+        long nextSlotId = 0;
+        for (WriteRequest request : writeRequests) {
+            request.promise.setValue(new DLSN(lssn, entryId, nextSlotId));
+            nextSlotId += request.numRecords;
+        }
+        writeRequests.clear();
+    }
+
+    private synchronized void cancelPromises(Throwable reason) {
+        for (WriteRequest request : writeRequests) {
+            request.promise.setException(reason);
+        }
+        writeRequests.clear();
+    }
+
+    @Override
+    public synchronized long getMaxTxId() {
+        return maxTxId;
+    }
+
+    @Override
+    public synchronized boolean hasUserRecords() {
+        return hasUserData;
+    }
+
+    @Override
+    public int getNumBytes() {
+        return buffer.size();
+    }
+
+    @Override
+    public synchronized int getNumRecords() {
+        return count;
+    }
+
+    @Override
+    public synchronized Buffer getBuffer() throws InvalidEnvelopedEntryException, IOException {
+        if (!envelopeBeforeTransmit) {
+            return buffer;
+        }
+        // We can't escape this allocation because things need to be read from one byte array
+        // and then written to another. This is the destination.
+        Buffer toSend = new Buffer(buffer.size());
+        byte[] decompressed = buffer.getData();
+        int length = buffer.size();
+        EnvelopedEntry entry = new EnvelopedEntry(EnvelopedEntry.CURRENT_VERSION,
+                                                  codec,
+                                                  decompressed,
+                                                  length,
+                                                  statsLogger);
+        // This will cause an allocation of a byte[] for compression. This can be avoided
+        // but we can do that later only if needed.
+        entry.writeFully(new DataOutputStream(toSend));
+        return toSend;
+    }
+
+    @Override
+    public synchronized DLSN finalizeTransmit(long lssn, long entryId) {
+        return new DLSN(lssn, entryId, count - 1);
+    }
+
+    @Override
+    public void completeTransmit(long lssn, long entryId) {
+        satisfyPromises(lssn, entryId);
+    }
+
+    @Override
+    public void abortTransmit(Throwable reason) {
+        cancelPromises(reason);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LedgerReadPosition.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LedgerReadPosition.java
new file mode 100644
index 0000000..f94495f
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LedgerReadPosition.java
@@ -0,0 +1,171 @@
+/**
+ * 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 java.io.Serializable;
+import java.util.Comparator;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class LedgerReadPosition {
+    static final Logger LOG = LoggerFactory.getLogger(LedgerReadPosition.class);
+
+    private static enum PartialOrderingComparisonResult {
+        NotComparable,
+        GreaterThan,
+        LessThan,
+        EqualTo
+    }
+
+    long ledgerId = DistributedLogConstants.UNRESOLVED_LEDGER_ID;
+    long logSegmentSequenceNo;
+    long entryId;
+
+    public LedgerReadPosition(long ledgerId, long logSegmentSequenceNo, long entryId) {
+        this.ledgerId = ledgerId;
+        this.logSegmentSequenceNo = logSegmentSequenceNo;
+        this.entryId = entryId;
+    }
+
+    public LedgerReadPosition(LedgerReadPosition that) {
+        this.ledgerId = that.ledgerId;
+        this.logSegmentSequenceNo = that.logSegmentSequenceNo;
+        this.entryId = that.entryId;
+    }
+
+
+    public LedgerReadPosition(final DLSN dlsn) {
+        this(dlsn.getLogSegmentSequenceNo(), dlsn.getEntryId());
+    }
+
+    public LedgerReadPosition(long logSegmentSequenceNo, long entryId) {
+        this.logSegmentSequenceNo = logSegmentSequenceNo;
+        this.entryId = entryId;
+    }
+
+    public long getLedgerId() {
+        if (DistributedLogConstants.UNRESOLVED_LEDGER_ID == ledgerId) {
+            LOG.trace("Ledger Id is not initialized");
+            throw new IllegalStateException("Ledger Id is not initialized");
+        }
+        return ledgerId;
+    }
+
+    public long getLogSegmentSequenceNumber() {
+        return logSegmentSequenceNo;
+    }
+
+    public long getEntryId() {
+        return entryId;
+    }
+
+    public void advance() {
+        entryId++;
+    }
+
+    public void positionOnNewLogSegment(long ledgerId, long logSegmentSequenceNo) {
+        this.ledgerId = ledgerId;
+        this.logSegmentSequenceNo = logSegmentSequenceNo;
+        this.entryId = 0L;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("(lid=%d, lseqNo=%d, eid=%d)", ledgerId, logSegmentSequenceNo, entryId);
+    }
+
+    public boolean definitelyLessThanOrEqualTo(LedgerReadPosition threshold) {
+        PartialOrderingComparisonResult result = comparePartiallyOrdered(threshold);
+        return ((result == PartialOrderingComparisonResult.LessThan) ||
+            (result == PartialOrderingComparisonResult.EqualTo));
+    }
+
+    public boolean definitelyLessThan(LedgerReadPosition threshold) {
+        PartialOrderingComparisonResult result = comparePartiallyOrdered(threshold);
+        return result == PartialOrderingComparisonResult.LessThan;
+    }
+
+    private PartialOrderingComparisonResult comparePartiallyOrdered(LedgerReadPosition threshold) {
+        // If no threshold is passed we cannot make a definitive comparison
+        if (null == threshold) {
+            return PartialOrderingComparisonResult.NotComparable;
+        }
+
+        if (this.logSegmentSequenceNo != threshold.logSegmentSequenceNo) {
+            if (this.logSegmentSequenceNo < threshold.logSegmentSequenceNo) {
+                return PartialOrderingComparisonResult.LessThan;
+            } else {
+                return PartialOrderingComparisonResult.GreaterThan;
+            }
+        } else if (this.ledgerId != threshold.ledgerId) {
+            // When logSegmentSequenceNo is equal we cannot definitely say that this
+            // position is less than the threshold unless ledgerIds are equal
+            // since LogSegmentSequenceNumber maybe inferred from transactionIds in older
+            // versions of the metadata.
+            return PartialOrderingComparisonResult.NotComparable;
+        } else if (this.getEntryId() < threshold.getEntryId()) {
+            return PartialOrderingComparisonResult.LessThan;
+        } else if (this.getEntryId() > threshold.getEntryId()) {
+            return PartialOrderingComparisonResult.GreaterThan;
+        } else {
+            return PartialOrderingComparisonResult.EqualTo;
+        }
+    }
+
+    /**
+     * Comparator for the key portion
+     */
+    public static final ReadAheadCacheKeyComparator COMPARATOR = new ReadAheadCacheKeyComparator();
+
+    // Only compares the key portion
+    @Override
+    public boolean equals(Object other) {
+        if (!(other instanceof LedgerReadPosition)) {
+            return false;
+        }
+        LedgerReadPosition key = (LedgerReadPosition) other;
+        return ledgerId == key.ledgerId &&
+            entryId == key.entryId;
+    }
+
+    @Override
+    public int hashCode() {
+        return (int) (ledgerId * 13 ^ entryId * 17);
+    }
+
+    /**
+     * Compare EntryKey.
+     */
+    protected static class ReadAheadCacheKeyComparator implements Comparator<LedgerReadPosition>, Serializable {
+
+        private static final long serialVersionUID = 0L;
+
+        @Override
+        public int compare(LedgerReadPosition left, LedgerReadPosition right) {
+            long ret = left.ledgerId - right.ledgerId;
+            if (ret == 0) {
+                ret = left.entryId - right.entryId;
+            }
+            return (ret < 0) ? -1 : ((ret > 0) ? 1 : 0);
+        }
+    }
+
+}
+
+
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java
new file mode 100644
index 0000000..5623525
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java
@@ -0,0 +1,364 @@
+/**
+ * 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.base.Optional;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.metadata.DLMetadata;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.proto.BookieServer;
+import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
+import org.apache.bookkeeper.util.IOUtils;
+import org.apache.bookkeeper.util.LocalBookKeeper;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.BindException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Utility class for setting up bookkeeper ensembles
+ * and bringing individual bookies up and down
+ */
+public class LocalDLMEmulator {
+    private static final Logger LOG = LoggerFactory.getLogger(LocalDLMEmulator.class);
+
+    public static final String DLOG_NAMESPACE = "/messaging/distributedlog";
+
+    private static final int DEFAULT_BOOKIE_INITIAL_PORT = 0; // Use ephemeral ports
+    private static final int DEFAULT_ZK_TIMEOUT_SEC = 10;
+    private static final int DEFAULT_ZK_PORT = 2181;
+    private static final String DEFAULT_ZK_HOST = "127.0.0.1";
+    private static final String DEFAULT_ZK_ENSEMBLE = DEFAULT_ZK_HOST + ":" + DEFAULT_ZK_PORT;
+    private static final int DEFAULT_NUM_BOOKIES = 3;
+    private static final ServerConfiguration DEFAULT_SERVER_CONFIGURATION = new ServerConfiguration();
+
+    private final String zkEnsemble;
+    private final URI uri;
+    private final List<File> tmpDirs = new ArrayList<File>();
+    private final int zkTimeoutSec;
+    private final Thread bkStartupThread;
+    private final String zkHost;
+    private final int zkPort;
+    private final int numBookies;
+
+    public static class Builder {
+        private int zkTimeoutSec = DEFAULT_ZK_TIMEOUT_SEC;
+        private int numBookies = DEFAULT_NUM_BOOKIES;
+        private String zkHost = DEFAULT_ZK_HOST;
+        private int zkPort = DEFAULT_ZK_PORT;
+        private int initialBookiePort = DEFAULT_BOOKIE_INITIAL_PORT;
+        private boolean shouldStartZK = true;
+        private Optional<ServerConfiguration> serverConf = Optional.absent();
+
+        public Builder numBookies(int numBookies) {
+            this.numBookies = numBookies;
+            return this;
+        }
+        public Builder zkHost(String zkHost) {
+            this.zkHost = zkHost;
+            return this;
+        }
+        public Builder zkPort(int zkPort) {
+            this.zkPort = zkPort;
+            return this;
+        }
+        public Builder zkTimeoutSec(int zkTimeoutSec) {
+            this.zkTimeoutSec = zkTimeoutSec;
+            return this;
+        }
+        public Builder initialBookiePort(int initialBookiePort) {
+            this.initialBookiePort = initialBookiePort;
+            return this;
+        }
+        public Builder shouldStartZK(boolean shouldStartZK) {
+            this.shouldStartZK = shouldStartZK;
+            return this;
+        }
+        public Builder serverConf(ServerConfiguration serverConf) {
+            this.serverConf = Optional.of(serverConf);
+            return this;
+        }
+
+        public LocalDLMEmulator build() throws Exception {
+            ServerConfiguration conf = null;
+            if (serverConf.isPresent()) {
+                conf = serverConf.get();
+            } else {
+                conf = (ServerConfiguration) DEFAULT_SERVER_CONFIGURATION.clone();
+                conf.setZkTimeout(zkTimeoutSec * 1000);
+            }
+            ServerConfiguration newConf = new ServerConfiguration();
+            newConf.loadConf(conf);
+            newConf.setAllowLoopback(true);
+
+            return new LocalDLMEmulator(numBookies, shouldStartZK, zkHost, zkPort,
+                initialBookiePort, zkTimeoutSec, newConf);
+        }
+    }
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    private LocalDLMEmulator(final int numBookies, final boolean shouldStartZK, final String zkHost, final int zkPort, final int initialBookiePort, final int zkTimeoutSec, final ServerConfiguration serverConf) throws Exception {
+        this.numBookies = numBookies;
+        this.zkHost = zkHost;
+        this.zkPort = zkPort;
+        this.zkEnsemble = zkHost + ":" + zkPort;
+        this.uri = URI.create("distributedlog://" + zkEnsemble + DLOG_NAMESPACE);
+        this.zkTimeoutSec = zkTimeoutSec;
+        this.bkStartupThread = new Thread() {
+            public void run() {
+                try {
+                    LOG.info("Starting {} bookies : allowLoopback = {}", numBookies, serverConf.getAllowLoopback());
+                    LocalBookKeeper.startLocalBookies(zkHost, zkPort, numBookies, shouldStartZK, initialBookiePort, serverConf);
+                    LOG.info("{} bookies are started.");
+                } catch (InterruptedException e) {
+                    // go away quietly
+                } catch (Exception e) {
+                    LOG.error("Error starting local bk", e);
+                }
+            }
+        };
+    }
+
+    public void start() throws Exception {
+        bkStartupThread.start();
+        if (!LocalBookKeeper.waitForServerUp(zkEnsemble, zkTimeoutSec*1000)) {
+            throw new Exception("Error starting zookeeper/bookkeeper");
+        }
+        int bookiesUp = checkBookiesUp(numBookies, zkTimeoutSec);
+        assert (numBookies == bookiesUp);
+        // Provision "/messaging/distributedlog" namespace
+        DLMetadata.create(new BKDLConfig(zkEnsemble, "/ledgers")).create(uri);
+    }
+
+    public void teardown() throws Exception {
+        if (bkStartupThread != null) {
+            bkStartupThread.interrupt();
+            bkStartupThread.join();
+        }
+        for (File dir : tmpDirs) {
+            FileUtils.deleteDirectory(dir);
+        }
+    }
+
+    public String getZkServers() {
+        return zkEnsemble;
+    }
+
+    public URI getUri() {
+        return uri;
+    }
+
+    public BookieServer newBookie() throws Exception {
+        ServerConfiguration bookieConf = new ServerConfiguration();
+        bookieConf.setZkTimeout(zkTimeoutSec * 1000);
+        bookieConf.setBookiePort(0);
+        bookieConf.setAllowLoopback(true);
+        File tmpdir = File.createTempFile("bookie" + UUID.randomUUID() + "_",
+            "test");
+        if (!tmpdir.delete()) {
+            LOG.debug("Fail to delete tmpdir " + tmpdir);
+        }
+        if (!tmpdir.mkdir()) {
+            throw new IOException("Fail to create tmpdir " + tmpdir);
+        }
+        tmpDirs.add(tmpdir);
+
+        bookieConf.setZkServers(zkEnsemble);
+        bookieConf.setJournalDirName(tmpdir.getPath());
+        bookieConf.setLedgerDirNames(new String[]{tmpdir.getPath()});
+
+        BookieServer b = new BookieServer(bookieConf);
+        b.start();
+        for (int i = 0; i < 10 && !b.isRunning(); i++) {
+            Thread.sleep(10000);
+        }
+        if (!b.isRunning()) {
+            throw new IOException("Bookie would not start");
+        }
+        return b;
+    }
+
+    /**
+     * Check that a number of bookies are available
+     *
+     * @param count number of bookies required
+     * @param timeout number of seconds to wait for bookies to start
+     * @throws java.io.IOException if bookies are not started by the time the timeout hits
+     */
+    public int checkBookiesUp(int count, int timeout) throws Exception {
+        ZooKeeper zkc = connectZooKeeper(zkHost, zkPort, zkTimeoutSec);
+        try {
+            int mostRecentSize = 0;
+            for (int i = 0; i < timeout; i++) {
+                try {
+                    List<String> children = zkc.getChildren("/ledgers/available",
+                        false);
+                    children.remove("readonly");
+                    mostRecentSize = children.size();
+                    if ((mostRecentSize > count) || LOG.isDebugEnabled()) {
+                        LOG.info("Found " + mostRecentSize + " bookies up, "
+                            + "waiting for " + count);
+                        if ((mostRecentSize > count) || LOG.isTraceEnabled()) {
+                            for (String child : children) {
+                                LOG.info(" server: " + child);
+                            }
+                        }
+                    }
+                    if (mostRecentSize == count) {
+                        break;
+                    }
+                } catch (KeeperException e) {
+                    // ignore
+                }
+                Thread.sleep(1000);
+            }
+            return mostRecentSize;
+        } finally {
+            zkc.close();
+        }
+    }
+
+    public static String getBkLedgerPath() {
+        return "/ledgers";
+    }
+
+    public static ZooKeeper connectZooKeeper(String zkHost, int zkPort)
+        throws IOException, KeeperException, InterruptedException {
+            return connectZooKeeper(zkHost, zkPort, DEFAULT_ZK_TIMEOUT_SEC);
+    }
+
+    public static ZooKeeper connectZooKeeper(String zkHost, int zkPort, int zkTimeoutSec)
+        throws IOException, KeeperException, InterruptedException {
+        final CountDownLatch latch = new CountDownLatch(1);
+        final String zkHostPort = zkHost + ":" + zkPort;
+
+        ZooKeeper zkc = new ZooKeeper(zkHostPort, zkTimeoutSec * 1000, new Watcher() {
+            public void process(WatchedEvent event) {
+                if (event.getState() == Event.KeeperState.SyncConnected) {
+                    latch.countDown();
+                }
+            }
+        });
+        if (!latch.await(zkTimeoutSec, TimeUnit.SECONDS)) {
+            throw new IOException("Zookeeper took too long to connect");
+        }
+        return zkc;
+    }
+
+    public static URI createDLMURI(String path) throws Exception {
+        return createDLMURI(DEFAULT_ZK_ENSEMBLE, path);
+    }
+
+    public static URI createDLMURI(String zkServers, String path) throws Exception {
+        return URI.create("distributedlog://" + zkServers + DLOG_NAMESPACE + path);
+    }
+
+    /**
+     * Try to start zookkeeper locally on any port.
+     */
+    public static Pair<ZooKeeperServerShim, Integer> runZookeeperOnAnyPort(File zkDir) throws Exception {
+        return runZookeeperOnAnyPort((int) (Math.random()*10000+7000), zkDir);
+    }
+
+    /**
+     * Try to start zookkeeper locally on any port beginning with some base port.
+     * Dump some socket info when bind fails.
+     */
+    public static Pair<ZooKeeperServerShim, Integer> runZookeeperOnAnyPort(int basePort, File zkDir) throws Exception {
+
+        final int MAX_RETRIES = 20;
+        final int MIN_PORT = 1025;
+        final int MAX_PORT = 65535;
+        ZooKeeperServerShim zks = null;
+        int zkPort = basePort;
+        boolean success = false;
+        int retries = 0;
+
+        while (!success) {
+            try {
+                LOG.info("zk trying to bind to port " + zkPort);
+                zks = LocalBookKeeper.runZookeeper(1000, zkPort, zkDir);
+                success = true;
+            } catch (BindException be) {
+                retries++;
+                if (retries > MAX_RETRIES) {
+                    throw be;
+                }
+                zkPort++;
+                if (zkPort > MAX_PORT) {
+                    zkPort = MIN_PORT;
+                }
+            }
+        }
+
+        return Pair.of(zks, zkPort);
+    }
+
+    public static void main(String[] args) throws Exception {
+        try {
+            if (args.length < 1) {
+                System.out.println("Usage: LocalDLEmulator <zk_port>");
+                System.exit(-1);
+            }
+
+            final int zkPort = Integer.parseInt(args[0]);
+            final File zkDir = IOUtils.createTempDir("distrlog", "zookeeper");
+            final LocalDLMEmulator localDlm = LocalDLMEmulator.newBuilder()
+                .zkPort(zkPort)
+                .build();
+
+            Runtime.getRuntime().addShutdownHook(new Thread() {
+                @Override
+                public void run() {
+                    try {
+                        localDlm.teardown();
+                        FileUtils.deleteDirectory(zkDir);
+                        System.out.println("ByeBye!");
+                    } catch (Exception e) {
+                        // do nothing
+                    }
+                }
+            });
+            localDlm.start();
+
+            System.out.println(String.format(
+                "DistributedLog Sandbox is running now. You could access distributedlog://%s:%s",
+                DEFAULT_ZK_HOST,
+                zkPort));
+        } catch (Exception ex) {
+            System.out.println("Exception occurred running emulator " + ex);
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java
new file mode 100644
index 0000000..75a32ef
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java
@@ -0,0 +1,195 @@
+/**
+ * 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 org.apache.distributedlog.io.AsyncCloseable;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+/**
+ * <i>LogReader</i> is a `synchronous` reader reading records from a DL log.
+ *
+ * <h3>Lifecycle of a Reader</h3>
+ *
+ * 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)}
+ * to read records out the log from provided position.
+ * <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,
+ * 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:
+ * <pre>
+ * DistributedLogManager dlm = ...;
+ * long nextTxId = ...;
+ * LogReader reader = dlm.getInputStream(nextTxId);
+ *
+ * while (true) { // keep reading & processing records
+ *     LogRecord record;
+ *     try {
+ *         record = reader.readNext(false);
+ *         nextTxId = record.getTransactionId();
+ *         // process the record
+ *         ...
+ *     } catch (IOException ioe) {
+ *         // handle the exception
+ *         ...
+ *         reader = dlm.getInputStream(nextTxId + 1);
+ *     }
+ * }
+ *
+ * </pre>
+ *
+ * <h3>Read Records</h3>
+ *
+ * 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.
+ *
+ * <h4>Blocking vs NonBlocking</h4>
+ *
+ * <i>Blocking</i> (nonBlocking = false) means the reads will wait for records
+ * 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
+ * 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
+ * will wait for a small period of time (defined in
+ * {@link DistributedLogConfiguration#getReadAheadWaitTime()} and return whatever
+ * 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
+ * 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.
+ *
+ * <h4>Read Single Record</h4>
+ *
+ * {@link #readNext(boolean)} is reading individual records from a DL log.
+ *
+ * <pre>
+ * LogReader reader = ...
+ *
+ * // keep reading records in blocking way until no records available in the log
+ * LogRecord record = reader.readNext(false);
+ * while (null != record) {
+ *     // process the record
+ *     ...
+ *     // read next record
+ *     records = reader.readNext(false);
+ * }
+ *
+ * ...
+ *
+ * // reader is caught up with writer, doing non-blocking reads to tail the log
+ * while (true) {
+ *     record = reader.readNext(true)
+ *     // process the new records
+ *     ...
+ * }
+ * </pre>
+ *
+ * <h4>Read Batch of Records</h4>
+ *
+ * {@link #readBulk(boolean, int)} is a convenient way to read a batch of records
+ * from a DL log.
+ *
+ * <pre>
+ * LogReader reader = ...
+ * int N = 10;
+ *
+ * // keep reading N records in blocking way until no records available in the log
+ * List<LogRecord> records = reader.readBulk(false, N);
+ * while (!records.isEmpty()) {
+ *     // process the list of records
+ *     ...
+ *     if (records.size() < N) { // no more records available in the log
+ *         break;
+ *     }
+ *     // read next N records
+ *     records = reader.readBulk(false, N);
+ * }
+ *
+ * ...
+ *
+ * // reader is caught up with writer, doing non-blocking reads to tail the log
+ * while (true) {
+ *     records = reader.readBulk(true, N)
+ *     // process the new records
+ *     ...
+ * }
+ *
+ * </pre>
+ *
+ * @see AsyncLogReader
+ *
+ * NOTE:
+ * 1. Extending {@link AsyncCloseable}: BKSyncLogReader is implemented based on BKAsyncLogReader, exposing
+ *    the {@link AsyncCloseable} interface so the reader could be closed asynchronously
+ */
+public interface LogReader extends Closeable, AsyncCloseable {
+
+    /**
+     * Read the next log record from the stream.
+     * <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
+     * 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.
+     *
+     * @param nonBlocking should the read make blocking calls to the backend or rely on the
+     * readAhead cache
+     * @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;
+
+    /**
+     * 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
+     * @param numLogRecords maximum number of log records returned by this call.
+     * @return an operation from the stream or empty list if at end of stream
+     * @throws IOException if there is an error reading from the stream
+     * @see #readNext(boolean)
+     */
+    public List<LogRecordWithDLSN> readBulk(boolean nonBlocking, int numLogRecords) throws IOException;
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
new file mode 100644
index 0000000..c5050ec
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
@@ -0,0 +1,1125 @@
+/**
+ * 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 java.io.File;
+import java.io.IOException;
+import java.util.Comparator;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+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.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;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Utility class for storing the metadata associated
+ * with a single edit log segment, stored in a single ledger
+ */
+public class LogSegmentMetadata {
+    static final Logger LOG = LoggerFactory.getLogger(LogSegmentMetadata.class);
+
+    public static enum LogSegmentMetadataVersion {
+        VERSION_INVALID(0),
+        VERSION_V1_ORIGINAL(1),
+        VERSION_V2_LEDGER_SEQNO(2),
+        VERSION_V3_MIN_ACTIVE_DLSN(3),
+        VERSION_V4_ENVELOPED_ENTRIES(4),
+        VERSION_V5_SEQUENCE_ID(5);
+
+        public final int value;
+
+        private LogSegmentMetadataVersion(int value) {
+            this.value = value;
+        }
+
+        public static LogSegmentMetadataVersion of(int version) {
+            switch (version) {
+                case 5:
+                    return VERSION_V5_SEQUENCE_ID;
+                case 4:
+                    return VERSION_V4_ENVELOPED_ENTRIES;
+                case 3:
+                    return VERSION_V3_MIN_ACTIVE_DLSN;
+                case 2:
+                    return VERSION_V2_LEDGER_SEQNO;
+                case 1:
+                    return VERSION_V1_ORIGINAL;
+                case 0:
+                    return VERSION_INVALID;
+                default:
+                    throw new IllegalArgumentException("unknown version " + version);
+            }
+        }
+    }
+
+    public static enum TruncationStatus {
+        ACTIVE (0), PARTIALLY_TRUNCATED(1), TRUNCATED (2);
+        private final int value;
+
+        private TruncationStatus(int value) {
+            this.value = value;
+        }
+    }
+
+    public static class LogSegmentMetadataBuilder {
+        protected String zkPath;
+        protected long logSegmentId;
+        protected LogSegmentMetadataVersion version;
+        protected long firstTxId;
+        protected int regionId;
+        protected long status;
+        protected long lastTxId;
+        protected long completionTime;
+        protected int recordCount;
+        protected long logSegmentSequenceNo;
+        protected long lastEntryId;
+        protected long lastSlotId;
+        protected long minActiveEntryId;
+        protected long minActiveSlotId;
+        protected long startSequenceId;
+        protected boolean inprogress;
+
+        // This is a derived attribute.
+        // Since we overwrite the original version with the target version, information that is
+        // derived from the original version (e.g. does it support enveloping of entries)
+        // is lost while parsing.
+        // NOTE: This value is not stored in the Metadata store.
+        protected boolean envelopeEntries = false;
+
+        LogSegmentMetadataBuilder(String zkPath,
+                                  LogSegmentMetadataVersion version,
+                                  long logSegmentId,
+                                  long firstTxId) {
+            initialize();
+            this.zkPath = zkPath;
+            this.version = version;
+            this.logSegmentId = logSegmentId;
+            this.firstTxId = firstTxId;
+        }
+
+        LogSegmentMetadataBuilder(String zkPath,
+                                  int version,
+                                  long logSegmentId,
+                                  long firstTxId) {
+            this(zkPath, LogSegmentMetadataVersion.values()[version], logSegmentId, firstTxId);
+        }
+
+        private void initialize() {
+            regionId = DistributedLogConstants.LOCAL_REGION_ID;
+            status = DistributedLogConstants.LOGSEGMENT_DEFAULT_STATUS;
+            lastTxId = DistributedLogConstants.INVALID_TXID;
+            completionTime = 0;
+            recordCount = 0;
+            lastEntryId = -1;
+            lastSlotId = -1;
+            minActiveEntryId = 0;
+            minActiveSlotId = 0;
+            startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
+            inprogress = true;
+        }
+
+        LogSegmentMetadataBuilder setRegionId(int regionId) {
+            this.regionId = regionId;
+            return this;
+        }
+
+        LogSegmentMetadataBuilder setStatus(long status) {
+            this.status = status;
+            return this;
+        }
+
+        public LogSegmentMetadataBuilder setLastTxId(long lastTxId) {
+            this.lastTxId = lastTxId;
+            return this;
+        }
+
+        public LogSegmentMetadataBuilder setCompletionTime(long completionTime) {
+            this.completionTime = completionTime;
+            return this;
+        }
+
+        public LogSegmentMetadataBuilder setRecordCount(int recordCount) {
+            this.recordCount = recordCount;
+            return this;
+        }
+
+        public LogSegmentMetadataBuilder setRecordCount(LogRecord record) {
+            this.recordCount = record.getLastPositionWithinLogSegment();
+            return this;
+        }
+
+        public LogSegmentMetadataBuilder setInprogress(boolean inprogress) {
+            this.inprogress = inprogress;
+            return this;
+        }
+
+        LogSegmentMetadataBuilder setLogSegmentSequenceNo(long logSegmentSequenceNo) {
+            this.logSegmentSequenceNo = logSegmentSequenceNo;
+            return this;
+        }
+
+        public LogSegmentMetadataBuilder setLastEntryId(long lastEntryId) {
+            this.lastEntryId = lastEntryId;
+            return this;
+        }
+
+        LogSegmentMetadataBuilder setLastSlotId(long lastSlotId) {
+            this.lastSlotId = lastSlotId;
+            return this;
+        }
+
+        LogSegmentMetadataBuilder setEnvelopeEntries(boolean envelopeEntries) {
+            this.envelopeEntries = envelopeEntries;
+            return this;
+        }
+
+        LogSegmentMetadataBuilder setMinActiveEntryId(long minActiveEntryId) {
+            this.minActiveEntryId = minActiveEntryId;
+            return this;
+        }
+
+        LogSegmentMetadataBuilder setMinActiveSlotId(long minActiveSlotId) {
+            this.minActiveSlotId = minActiveSlotId;
+            return this;
+        }
+
+        LogSegmentMetadataBuilder setStartSequenceId(long startSequenceId) {
+            this.startSequenceId = startSequenceId;
+            return this;
+        }
+
+        public LogSegmentMetadata build() {
+            return new LogSegmentMetadata(
+                zkPath,
+                version,
+                    logSegmentId,
+                firstTxId,
+                lastTxId,
+                completionTime,
+                inprogress,
+                recordCount,
+                    logSegmentSequenceNo,
+                lastEntryId,
+                lastSlotId,
+                regionId,
+                status,
+                minActiveEntryId,
+                minActiveSlotId,
+                startSequenceId,
+                envelopeEntries
+            );
+        }
+
+    }
+
+    /**
+     * Mutator to mutate the metadata of a log segment. This mutator is going to create
+     * a new instance of the log segment metadata without changing the existing one.
+     */
+    public static class Mutator extends LogSegmentMetadataBuilder {
+
+        Mutator(LogSegmentMetadata original) {
+            super(original.getZkPath(), original.getVersion(), original.getLogSegmentId(), original.getFirstTxId());
+            this.inprogress = original.isInProgress();
+            this.logSegmentSequenceNo = original.getLogSegmentSequenceNumber();
+            this.lastEntryId = original.getLastEntryId();
+            this.lastSlotId = original.getLastSlotId();
+            this.lastTxId = original.getLastTxId();
+            this.completionTime = original.getCompletionTime();
+            this.recordCount = original.getRecordCount();
+            this.regionId = original.getRegionId();
+            this.status = original.getStatus();
+            this.minActiveEntryId = original.getMinActiveDLSN().getEntryId();
+            this.minActiveSlotId = original.getMinActiveDLSN().getSlotId();
+            this.startSequenceId = original.getStartSequenceId();
+            this.envelopeEntries = original.getEnvelopeEntries();
+        }
+
+        @VisibleForTesting
+        public Mutator setVersion(LogSegmentMetadataVersion version) {
+            this.version = version;
+            return this;
+        }
+
+        public Mutator setLogSegmentSequenceNumber(long seqNo) {
+            this.logSegmentSequenceNo = seqNo;
+            return this;
+        }
+
+        public Mutator setZkPath(String zkPath) {
+            this.zkPath = zkPath;
+            return this;
+        }
+
+        public Mutator setLastDLSN(DLSN dlsn) {
+            this.logSegmentSequenceNo = dlsn.getLogSegmentSequenceNo();
+            this.lastEntryId = dlsn.getEntryId();
+            this.lastSlotId = dlsn.getSlotId();
+            return this;
+        }
+
+        public Mutator setMinActiveDLSN(DLSN dlsn) {
+            if (this.logSegmentSequenceNo != dlsn.getLogSegmentSequenceNo()) {
+                throw new IllegalArgumentException("Updating minDLSN in an incorrect log segment");
+            }
+            this.minActiveEntryId = dlsn.getEntryId();
+            this.minActiveSlotId = dlsn.getSlotId();
+            return this;
+        }
+
+        public Mutator setTruncationStatus(TruncationStatus truncationStatus) {
+            status &= ~METADATA_TRUNCATION_STATUS_MASK;
+            status |= (truncationStatus.value & METADATA_TRUNCATION_STATUS_MASK);
+            return this;
+        }
+
+        public Mutator setStartSequenceId(long startSequenceId) {
+            this.startSequenceId = startSequenceId;
+            return this;
+        }
+    }
+
+    private final String zkPath;
+    private final long logSegmentId;
+    private final LogSegmentMetadataVersion version;
+    private final long firstTxId;
+    private final int regionId;
+    private final long status;
+    private final long lastTxId;
+    private final long completionTime;
+    private final int recordCount;
+    private final DLSN lastDLSN;
+    private final DLSN minActiveDLSN;
+    private final long startSequenceId;
+    private final boolean inprogress;
+    // This is a derived attribute.
+    // Since we overwrite the original version with the target version, information that is
+    // derived from the original version (e.g. does it support enveloping of entries)
+    // is lost while parsing.
+    // NOTE: This value is not stored in the Metadata store.
+    private final boolean envelopeEntries;
+
+    public static final Comparator<LogSegmentMetadata> COMPARATOR
+        = new Comparator<LogSegmentMetadata>() {
+
+        public int compare(LogSegmentMetadata o1,
+                           LogSegmentMetadata o2) {
+            if ((o1.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO) ||
+                (o2.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)) {
+                if (o1.firstTxId < o2.firstTxId) {
+                    return -1;
+                } else if (o1.firstTxId == o2.firstTxId) {
+                    return 0;
+                } else {
+                    return 1;
+                }
+            } else {
+                if (o1.getLogSegmentSequenceNumber() < o2.getLogSegmentSequenceNumber()) {
+                    return -1;
+                } else if (o1.getLogSegmentSequenceNumber() == o2.getLogSegmentSequenceNumber()) {
+                    // make sure we won't move over inprogress log segment if it still presents in the list
+                    if (o1.isInProgress() && !o2.isInProgress()) {
+                        return -1;
+                    } else if (!o1.isInProgress() && o2.isInProgress()) {
+                        return 1;
+                    } else {
+                        return 0;
+                    }
+                } else {
+                    return 1;
+                }
+            }
+
+
+        }
+    };
+
+    public static final Comparator<LogSegmentMetadata> DESC_COMPARATOR
+        = new Comparator<LogSegmentMetadata>() {
+        public int compare(LogSegmentMetadata o1,
+                           LogSegmentMetadata o2) {
+            if ((o1.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO) ||
+                (o2.getLogSegmentSequenceNumber() == DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO)) {
+                if (o1.firstTxId > o2.firstTxId) {
+                    return -1;
+                } else if (o1.firstTxId == o2.firstTxId) {
+                    return 0;
+                } else {
+                    return 1;
+                }
+            } else {
+                if (o1.getLogSegmentSequenceNumber() > o2.getLogSegmentSequenceNumber()) {
+                    return -1;
+                } else if (o1.getLogSegmentSequenceNumber() == o2.getLogSegmentSequenceNumber()) {
+                    // make sure we won't move over inprogress log segment if it still presents in the list
+                    if (o1.isInProgress() && !o2.isInProgress()) {
+                        return 1;
+                    } else if (!o1.isInProgress() && o2.isInProgress()) {
+                        return -1;
+                    } else {
+                        return 0;
+                    }
+                } else {
+                    return 1;
+                }
+            }
+        }
+    };
+
+    public static final int LEDGER_METADATA_CURRENT_LAYOUT_VERSION =
+                LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value;
+
+    public static final int LEDGER_METADATA_OLDEST_SUPPORTED_VERSION =
+        LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value;
+
+    static final int LOGRECORD_COUNT_SHIFT = 32;
+    static final long LOGRECORD_COUNT_MASK = 0xffffffff00000000L;
+    static final int REGION_SHIFT = 28;
+    static final long MAX_REGION_ID = 0xfL;
+    static final long REGION_MASK = 0x00000000f0000000L;
+    static final int STATUS_BITS_SHIFT = 8;
+    static final long STATUS_BITS_MASK = 0x000000000000ff00L;
+    static final long UNUSED_BITS_MASK = 0x000000000fff0000L;
+    static final long METADATA_VERSION_MASK = 0x00000000000000ffL;
+
+    //Metadata status bits
+    static final long METADATA_TRUNCATION_STATUS_MASK = 0x3L;
+    static final long METADATA_STATUS_BIT_MAX = 0xffL;
+
+    private LogSegmentMetadata(String zkPath,
+                               LogSegmentMetadataVersion version,
+                               long logSegmentId,
+                               long firstTxId,
+                               long lastTxId,
+                               long completionTime,
+                               boolean inprogress,
+                               int recordCount,
+                               long logSegmentSequenceNumber,
+                               long lastEntryId,
+                               long lastSlotId,
+                               int regionId,
+                               long status,
+                               long minActiveEntryId,
+                               long minActiveSlotId,
+                               long startSequenceId,
+                               boolean envelopeEntries) {
+        this.zkPath = zkPath;
+        this.logSegmentId = logSegmentId;
+        this.version = version;
+        this.firstTxId = firstTxId;
+        this.lastTxId = lastTxId;
+        this.inprogress = inprogress;
+        this.completionTime = completionTime;
+        this.recordCount = recordCount;
+        this.lastDLSN = new DLSN(logSegmentSequenceNumber, lastEntryId, lastSlotId);
+        this.minActiveDLSN = new DLSN(logSegmentSequenceNumber, minActiveEntryId, minActiveSlotId);
+        this.startSequenceId = startSequenceId;
+        this.regionId = regionId;
+        this.status = status;
+        this.envelopeEntries = envelopeEntries;
+    }
+
+    public String getZkPath() {
+        return zkPath;
+    }
+
+    public String getZNodeName() {
+        return new File(zkPath).getName();
+    }
+
+    public long getFirstTxId() {
+        return firstTxId;
+    }
+
+    public long getLastTxId() {
+        return lastTxId;
+    }
+
+    public long getCompletionTime() {
+        return completionTime;
+    }
+
+    public long getLogSegmentId() {
+        return logSegmentId;
+    }
+
+    public long getLogSegmentSequenceNumber() {
+        return lastDLSN.getLogSegmentSequenceNo();
+    }
+
+    public int getVersion() {
+        return version.value;
+    }
+
+    public boolean getEnvelopeEntries() {
+        return envelopeEntries;
+    }
+
+    public long getLastEntryId() {
+        return lastDLSN.getEntryId();
+    }
+
+    long getStatus() {
+        return status;
+    }
+
+    public long getStartSequenceId() {
+        // generate negative sequence id for log segments that created <= v4
+        return supportsSequenceId() && startSequenceId != DistributedLogConstants.UNASSIGNED_SEQUENCE_ID ?
+                startSequenceId : Long.MIN_VALUE + (getLogSegmentSequenceNumber() << 32L);
+    }
+
+    public boolean isTruncated() {
+        return ((status & METADATA_TRUNCATION_STATUS_MASK)
+                == TruncationStatus.TRUNCATED.value);
+    }
+
+    public boolean isPartiallyTruncated() {
+        return ((status & METADATA_TRUNCATION_STATUS_MASK)
+                == TruncationStatus.PARTIALLY_TRUNCATED.value);
+    }
+
+    public boolean isNonTruncated() {
+        return ((status & METADATA_TRUNCATION_STATUS_MASK)
+                == TruncationStatus.ACTIVE.value);
+    }
+
+    public long getLastSlotId() {
+        return lastDLSN.getSlotId();
+    }
+
+    public DLSN getLastDLSN() {
+        return lastDLSN;
+    }
+
+    public DLSN getMinActiveDLSN() {
+        return minActiveDLSN;
+    }
+
+    public DLSN getFirstDLSN() {
+        return new DLSN(getLogSegmentSequenceNumber(), 0, 0);
+    }
+
+    public int getRecordCount() {
+        return recordCount;
+    }
+
+    public int getRegionId() {
+        return regionId;
+    }
+
+    public boolean isInProgress() {
+        return this.inprogress;
+    }
+
+    @VisibleForTesting
+    public boolean isDLSNinThisSegment(DLSN dlsn) {
+        return dlsn.getLogSegmentSequenceNo() == getLogSegmentSequenceNumber();
+    }
+
+    @VisibleForTesting
+    public boolean isRecordPositionWithinSegmentScope(LogRecord record) {
+        return record.getLastPositionWithinLogSegment() <= getRecordCount();
+    }
+
+    @VisibleForTesting
+    public boolean isRecordLastPositioninThisSegment(LogRecord record) {
+        return record.getLastPositionWithinLogSegment() == getRecordCount();
+    }
+
+    /**
+     * complete current log segment. A new log segment metadata instance will be returned.
+     *
+     * @param zkPath
+     *          zk path for the completed log segment.
+     * @param newLastTxId
+     *          last tx id
+     * @param recordCount
+     *          record count
+     * @param lastEntryId
+     *          last entry id
+     * @param lastSlotId
+     *          last slot id
+     * @return completed log segment.
+     */
+    LogSegmentMetadata completeLogSegment(String zkPath,
+                                                long newLastTxId,
+                                                int recordCount,
+                                                long lastEntryId,
+                                                long lastSlotId,
+                                                long startSequenceId) {
+        assert this.lastTxId == DistributedLogConstants.INVALID_TXID;
+
+        return new Mutator(this)
+                .setZkPath(zkPath)
+                .setLastDLSN(new DLSN(this.lastDLSN.getLogSegmentSequenceNo(), lastEntryId, lastSlotId))
+                .setLastTxId(newLastTxId)
+                .setInprogress(false)
+                .setCompletionTime(Utils.nowInMillis())
+                .setRecordCount(recordCount)
+                .setStartSequenceId(startSequenceId)
+                .build();
+    }
+
+    public static Future<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>();
+        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));
+                        } else {
+                            FutureUtils.setException(result,
+                                    new ZKException("Failed to read log segment metadata from " + path,
+                                            KeeperException.Code.get(rc)));
+                        }
+                        return;
+                    }
+                    try {
+                        LogSegmentMetadata metadata = parseData(path, data, skipMinVersionCheck);
+                        FutureUtils.setValue(result, metadata);
+                    } catch (IOException ie) {
+                        LOG.error("Error on parsing log segment metadata from {} : ", path, ie);
+                        result.setException(ie);
+                    }
+                }
+            }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            result.setException(FutureUtils.zkException(e, path));
+        } catch (InterruptedException e) {
+            result.setException(FutureUtils.zkException(e, path));
+        }
+        return result;
+    }
+
+    static LogSegmentMetadata parseDataV1(String path, byte[] data, String[] parts)
+        throws IOException {
+        long versionStatusCount = Long.parseLong(parts[0]);
+
+        long version = versionStatusCount & METADATA_VERSION_MASK;
+        assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
+        assert (1 == version);
+
+        LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.VERSION_V1_ORIGINAL;
+
+        int regionId = (int)(versionStatusCount & REGION_MASK) >> REGION_SHIFT;
+        assert (regionId >= 0 && regionId <= 0xf);
+
+        long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
+        assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
+
+        if (parts.length == 3) {
+            long logSegmentId = Long.parseLong(parts[1]);
+            long txId = Long.parseLong(parts[2]);
+            return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
+                    .setRegionId(regionId)
+                    .setStatus(status)
+                    .build();
+        } else if (parts.length == 5) {
+            long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
+            assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
+
+            long logSegmentId = Long.parseLong(parts[1]);
+            long firstTxId = Long.parseLong(parts[2]);
+            long lastTxId = Long.parseLong(parts[3]);
+            long completionTime = Long.parseLong(parts[4]);
+            return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
+                .setInprogress(false)
+                .setLastTxId(lastTxId)
+                .setCompletionTime(completionTime)
+                .setRecordCount((int) recordCount)
+                .setRegionId(regionId)
+                .setStatus(status)
+                .build();
+        } else {
+            throw new IOException("Invalid log segment metadata : "
+                + new String(data, UTF_8));
+        }
+    }
+
+    static LogSegmentMetadata parseDataV2(String path, byte[] data, String[] parts)
+        throws IOException {
+        long versionStatusCount = Long.parseLong(parts[0]);
+
+        long version = versionStatusCount & METADATA_VERSION_MASK;
+        assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
+        assert (2 == version);
+
+        LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO;
+
+        int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
+        assert (regionId >= 0 && regionId <= 0xf);
+
+        long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
+        assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
+
+        if (parts.length == 4) {
+            long logSegmentId = Long.parseLong(parts[1]);
+            long txId = Long.parseLong(parts[2]);
+            long logSegmentSequenceNumber = Long.parseLong(parts[3]);
+            return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
+                .setLogSegmentSequenceNo(logSegmentSequenceNumber)
+                .setRegionId(regionId)
+                .setStatus(status)
+                .build();
+        } else if (parts.length == 8) {
+            long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
+            assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
+
+            long logSegmentId = Long.parseLong(parts[1]);
+            long firstTxId = Long.parseLong(parts[2]);
+            long lastTxId = Long.parseLong(parts[3]);
+            long completionTime = Long.parseLong(parts[4]);
+            long logSegmentSequenceNumber = Long.parseLong(parts[5]);
+            long lastEntryId = Long.parseLong(parts[6]);
+            long lastSlotId = Long.parseLong(parts[7]);
+            return new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
+                .setInprogress(false)
+                .setLastTxId(lastTxId)
+                .setCompletionTime(completionTime)
+                .setRecordCount((int) recordCount)
+                .setLogSegmentSequenceNo(logSegmentSequenceNumber)
+                .setLastEntryId(lastEntryId)
+                .setLastSlotId(lastSlotId)
+                .setRegionId(regionId)
+                .setStatus(status)
+                .build();
+        } else {
+            throw new IOException("Invalid logsegment metadata : "
+                + new String(data, UTF_8));
+        }
+
+    }
+
+    static LogSegmentMetadata parseDataVersionsWithMinActiveDLSN(String path, byte[] data, String[] parts)
+        throws IOException {
+        long versionStatusCount = Long.parseLong(parts[0]);
+
+        long version = versionStatusCount & METADATA_VERSION_MASK;
+        assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
+        assert (LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version &&
+                LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version);
+
+        LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.of((int) version);
+
+        int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
+        assert (regionId >= 0 && regionId <= 0xf);
+
+        long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
+        assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
+
+        if (parts.length == 6) {
+            long logSegmentId = Long.parseLong(parts[1]);
+            long txId = Long.parseLong(parts[2]);
+            long logSegmentSequenceNumber = Long.parseLong(parts[3]);
+            long minActiveEntryId = Long.parseLong(parts[4]);
+            long minActiveSlotId = Long.parseLong(parts[5]);
+
+            LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
+                .setLogSegmentSequenceNo(logSegmentSequenceNumber)
+                .setMinActiveEntryId(minActiveEntryId)
+                .setMinActiveSlotId(minActiveSlotId)
+                .setRegionId(regionId)
+                .setStatus(status);
+            if (supportsEnvelopedEntries((int) version)) {
+                builder = builder.setEnvelopeEntries(true);
+            }
+            return builder.build();
+        } else if (parts.length == 10) {
+            long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
+            assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
+
+            long logSegmentId = Long.parseLong(parts[1]);
+            long firstTxId = Long.parseLong(parts[2]);
+            long lastTxId = Long.parseLong(parts[3]);
+            long completionTime = Long.parseLong(parts[4]);
+            long logSegmentSequenceNumber = Long.parseLong(parts[5]);
+            long lastEntryId = Long.parseLong(parts[6]);
+            long lastSlotId = Long.parseLong(parts[7]);
+            long minActiveEntryId = Long.parseLong(parts[8]);
+            long minActiveSlotId = Long.parseLong(parts[9]);
+            LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
+                .setInprogress(false)
+                .setLastTxId(lastTxId)
+                .setCompletionTime(completionTime)
+                .setRecordCount((int) recordCount)
+                .setLogSegmentSequenceNo(logSegmentSequenceNumber)
+                .setLastEntryId(lastEntryId)
+                .setLastSlotId(lastSlotId)
+                .setMinActiveEntryId(minActiveEntryId)
+                .setMinActiveSlotId(minActiveSlotId)
+                .setRegionId(regionId)
+                .setStatus(status);
+            if (supportsEnvelopedEntries((int) version)) {
+                builder = builder.setEnvelopeEntries(true);
+            }
+            return builder.build();
+        } else {
+            throw new IOException("Invalid logsegment metadata : "
+                + new String(data, UTF_8));
+        }
+
+    }
+
+    static LogSegmentMetadata parseDataVersionsWithSequenceId(String path, byte[] data, String[] parts)
+        throws IOException {
+        long versionStatusCount = Long.parseLong(parts[0]);
+
+        long version = versionStatusCount & METADATA_VERSION_MASK;
+        assert (version >= Integer.MIN_VALUE && version <= Integer.MAX_VALUE);
+        assert (LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value <= version &&
+                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION >= version);
+
+        LogSegmentMetadataVersion llmv = LogSegmentMetadataVersion.of((int) version);
+
+        int regionId = (int)((versionStatusCount & REGION_MASK) >> REGION_SHIFT);
+        assert (regionId >= 0 && regionId <= 0xf);
+
+        long status = (versionStatusCount & STATUS_BITS_MASK) >> STATUS_BITS_SHIFT;
+        assert (status >= 0 && status <= METADATA_STATUS_BIT_MAX);
+
+        if (parts.length == 7) {
+            long logSegmentId = Long.parseLong(parts[1]);
+            long txId = Long.parseLong(parts[2]);
+            long logSegmentSequenceNumber = Long.parseLong(parts[3]);
+            long minActiveEntryId = Long.parseLong(parts[4]);
+            long minActiveSlotId = Long.parseLong(parts[5]);
+            long startSequenceId = Long.parseLong(parts[6]);
+
+            LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, txId)
+                    .setLogSegmentSequenceNo(logSegmentSequenceNumber)
+                    .setMinActiveEntryId(minActiveEntryId)
+                    .setMinActiveSlotId(minActiveSlotId)
+                    .setRegionId(regionId)
+                    .setStatus(status)
+                    .setStartSequenceId(startSequenceId)
+                    .setEnvelopeEntries(true);
+            return builder.build();
+        } else if (parts.length == 11) {
+            long recordCount = (versionStatusCount & LOGRECORD_COUNT_MASK) >> LOGRECORD_COUNT_SHIFT;
+            assert (recordCount >= Integer.MIN_VALUE && recordCount <= Integer.MAX_VALUE);
+
+            long logSegmentId = Long.parseLong(parts[1]);
+            long firstTxId = Long.parseLong(parts[2]);
+            long lastTxId = Long.parseLong(parts[3]);
+            long completionTime = Long.parseLong(parts[4]);
+            long logSegmentSequenceNumber = Long.parseLong(parts[5]);
+            long lastEntryId = Long.parseLong(parts[6]);
+            long lastSlotId = Long.parseLong(parts[7]);
+            long minActiveEntryId = Long.parseLong(parts[8]);
+            long minActiveSlotId = Long.parseLong(parts[9]);
+            long startSequenceId = Long.parseLong(parts[10]);
+            LogSegmentMetadataBuilder builder = new LogSegmentMetadataBuilder(path, llmv, logSegmentId, firstTxId)
+                    .setInprogress(false)
+                    .setLastTxId(lastTxId)
+                    .setCompletionTime(completionTime)
+                    .setRecordCount((int) recordCount)
+                    .setLogSegmentSequenceNo(logSegmentSequenceNumber)
+                    .setLastEntryId(lastEntryId)
+                    .setLastSlotId(lastSlotId)
+                    .setMinActiveEntryId(minActiveEntryId)
+                    .setMinActiveSlotId(minActiveSlotId)
+                    .setRegionId(regionId)
+                    .setStatus(status)
+                    .setStartSequenceId(startSequenceId)
+                    .setEnvelopeEntries(true);
+            return builder.build();
+        } else {
+            throw new IOException("Invalid log segment metadata : "
+                    + new String(data, UTF_8));
+        }
+    }
+
+    public static LogSegmentMetadata parseData(String path, byte[] data)
+            throws IOException {
+        return parseData(path, data, false);
+    }
+
+    static LogSegmentMetadata parseData(String path, byte[] data, boolean skipMinVersionCheck) throws IOException {
+        String[] parts = new String(data, UTF_8).split(";");
+        long version;
+        try {
+            version = Long.parseLong(parts[0]) & METADATA_VERSION_MASK;
+        } catch (Exception exc) {
+            throw new IOException("Invalid ledger entry, "
+                + new String(data, UTF_8));
+        }
+
+        if (!skipMinVersionCheck && version < LogSegmentMetadata.LEDGER_METADATA_OLDEST_SUPPORTED_VERSION) {
+            throw new UnsupportedMetadataVersionException("Ledger metadata version '" + version + "' is no longer supported: "
+                + new String(data, UTF_8));
+        }
+
+        if (version > LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION) {
+            throw new UnsupportedMetadataVersionException("Metadata version '" + version + "' is higher than the highest supported version : "
+                + new String(data, UTF_8));
+        }
+
+        if (LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value == version) {
+            return parseDataV1(path, data, parts);
+        } else if (LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value == version) {
+            return parseDataV2(path, data, parts);
+        } else if (LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version &&
+                   LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version) {
+            return parseDataVersionsWithMinActiveDLSN(path, data, parts);
+        } else {
+            assert(version >= LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value);
+            return parseDataVersionsWithSequenceId(path, data, parts);
+        }
+    }
+
+    public String getFinalisedData() {
+        return getFinalisedData(this.version);
+    }
+
+    public String getFinalisedData(LogSegmentMetadataVersion version) {
+        String finalisedData;
+        final long logSegmentSeqNo = getLogSegmentSequenceNumber();
+        final long lastEntryId = getLastEntryId();
+        final long lastSlotId = getLastSlotId();
+        final long minActiveEntryId = minActiveDLSN.getEntryId();
+        final long minActiveSlotId = minActiveDLSN.getSlotId();
+
+        if (LogSegmentMetadataVersion.VERSION_V1_ORIGINAL == version) {
+            if (inprogress) {
+                finalisedData = String.format("%d;%d;%d",
+                    version.value, logSegmentId, firstTxId);
+            } else {
+                long versionAndCount = ((long) version.value) | ((long)recordCount << LOGRECORD_COUNT_SHIFT);
+                finalisedData = String.format("%d;%d;%d;%d;%d",
+                    versionAndCount, logSegmentId, firstTxId, lastTxId, completionTime);
+            }
+        } else {
+            long versionStatusCount = ((long) version.value);
+            versionStatusCount |= ((status & METADATA_STATUS_BIT_MAX) << STATUS_BITS_SHIFT);
+            versionStatusCount |= (((long) regionId & MAX_REGION_ID) << REGION_SHIFT);
+            if (!inprogress) {
+                versionStatusCount |= ((long)recordCount << LOGRECORD_COUNT_SHIFT);
+            }
+            if (LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO == version) {
+                if (inprogress) {
+                    finalisedData = String.format("%d;%d;%d;%d",
+                        versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo);
+                } else {
+                    finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d",
+                        versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
+                        logSegmentSeqNo, lastEntryId, lastSlotId);
+                }
+            } else if (LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value >= version.value &&
+                        LogSegmentMetadataVersion.VERSION_V3_MIN_ACTIVE_DLSN.value <= version.value) {
+                if (inprogress) {
+                    finalisedData = String.format("%d;%d;%d;%d;%d;%d",
+                        versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo, minActiveEntryId, minActiveSlotId);
+                } else {
+                    finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d;%d;%d",
+                        versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
+                        logSegmentSeqNo, lastEntryId, lastSlotId, minActiveEntryId, minActiveSlotId);
+                }
+            } else if (LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value <= version.value &&
+                        LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION >= version.value) {
+                if (inprogress) {
+                    finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d",
+                        versionStatusCount, logSegmentId, firstTxId, logSegmentSeqNo, minActiveEntryId, minActiveSlotId, startSequenceId);
+                } else {
+                    finalisedData = String.format("%d;%d;%d;%d;%d;%d;%d;%d;%d;%d;%d",
+                        versionStatusCount, logSegmentId, firstTxId, lastTxId, completionTime,
+                        logSegmentSeqNo, lastEntryId, lastSlotId, minActiveEntryId, minActiveSlotId, startSequenceId);
+                }
+            } else {
+                throw new IllegalStateException("Unsupported log segment ledger metadata version '" + version + "'");
+            }
+        }
+        return finalisedData;
+    }
+
+    String getSegmentName() {
+        String[] parts = this.zkPath.split("/");
+        if (parts.length <= 0) {
+            throw new IllegalStateException("ZK Path is not valid");
+        }
+        return parts[parts.length - 1];
+    }
+
+    public void write(ZooKeeperClient zkc)
+        throws IOException, KeeperException.NodeExistsException {
+        String finalisedData = getFinalisedData(version);
+        try {
+            zkc.get().create(zkPath, finalisedData.getBytes(UTF_8),
+                zkc.getDefaultACL(), CreateMode.PERSISTENT);
+        } catch (KeeperException.NodeExistsException nee) {
+            throw nee;
+        } catch (InterruptedException ie) {
+            throw new DLInterruptedException("Interrupted on creating ledger znode " + zkPath, ie);
+        } catch (Exception e) {
+            LOG.error("Error creating ledger znode {}", zkPath, e);
+            throw new IOException("Error creating ledger znode " + zkPath);
+        }
+    }
+
+    boolean checkEquivalence(ZooKeeperClient zkc, String path) {
+        try {
+            LogSegmentMetadata other = FutureUtils.result(read(zkc, path));
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Verifying {} against {}", this, other);
+            }
+
+            boolean retVal;
+
+            // All fields may not be comparable so only compare the ones
+            // that can be compared
+            // completionTime is set when a node is finalized, so that
+            // cannot be compared
+            // if the node is inprogress, don't compare the lastTxId either
+            if (this.getLogSegmentSequenceNumber() != other.getLogSegmentSequenceNumber() ||
+                this.logSegmentId != other.logSegmentId ||
+                this.firstTxId != other.firstTxId) {
+                retVal = false;
+            } else if (this.inprogress) {
+                retVal = other.inprogress;
+            } else {
+                retVal = (!other.inprogress && (this.lastTxId == other.lastTxId));
+            }
+
+            if (!retVal) {
+                LOG.warn("Equivalence check failed between {} and {}", this, other);
+            }
+
+            return retVal;
+        } catch (Exception e) {
+            LOG.error("Could not check equivalence between:" + this + " and data in " + path, e);
+            return false;
+        }
+    }
+
+    public boolean equals(Object o) {
+        if (!(o instanceof LogSegmentMetadata)) {
+            return false;
+        }
+        LogSegmentMetadata ol = (LogSegmentMetadata) o;
+        return getLogSegmentSequenceNumber() == ol.getLogSegmentSequenceNumber()
+            && logSegmentId == ol.logSegmentId
+            && firstTxId == ol.firstTxId
+            && lastTxId == ol.lastTxId
+            && version == ol.version
+            && completionTime == ol.completionTime
+            && Objects.equal(lastDLSN, ol.lastDLSN)
+            && Objects.equal(minActiveDLSN, ol.minActiveDLSN)
+            && startSequenceId == ol.startSequenceId
+            && status == ol.status;
+    }
+
+    public int hashCode() {
+        int hash = 1;
+        hash = hash * 31 + (int) logSegmentId;
+        hash = hash * 31 + (int) firstTxId;
+        hash = hash * 31 + (int) lastTxId;
+        hash = hash * 31 + version.value;
+        hash = hash * 31 + (int) completionTime;
+        hash = hash * 31 + (int) getLogSegmentSequenceNumber();
+        return hash;
+    }
+
+    public String toString() {
+        return "[LogSegmentId:" + logSegmentId +
+            ", firstTxId:" + firstTxId +
+            ", lastTxId:" + lastTxId +
+            ", version:" + version +
+            ", completionTime:" + completionTime +
+            ", recordCount:" + recordCount +
+            ", regionId:" + regionId +
+            ", status:" + status +
+            ", logSegmentSequenceNumber:" + getLogSegmentSequenceNumber() +
+            ", lastEntryId:" + getLastEntryId() +
+            ", lastSlotId:" + getLastSlotId() +
+            ", inprogress:" + inprogress +
+            ", minActiveDLSN:" + minActiveDLSN +
+            ", startSequenceId:" + startSequenceId +
+            "]";
+    }
+
+    public Mutator mutator() {
+        return new Mutator(this);
+    }
+
+
+    //
+    // Version Checking Utilities
+    //
+
+    public boolean supportsLogSegmentSequenceNo() {
+        return supportsLogSegmentSequenceNo(version.value);
+    }
+
+    /**
+     * Whether the provided version supports log segment sequence number.
+     *
+     * @param version
+     *          log segment metadata version
+     * @return true if this log segment supports log segment sequence number.
+     */
+    public static boolean supportsLogSegmentSequenceNo(int version) {
+        return version >= LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value;
+    }
+
+    /**
+     * Whether the provided version supports enveloping entries before writing to bookkeeper.
+     *
+     * @param version
+     *          log segment metadata version
+     * @return true if this log segment supports enveloping entries
+     */
+    public static boolean supportsEnvelopedEntries(int version) {
+        return version >= LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value;
+    }
+
+    public boolean supportsSequenceId() {
+        return supportsSequenceId(version.value);
+    }
+
+    /**
+     * Whether the provided version supports sequence id.
+     *
+     * @param version
+     *          log segment metadata version
+     * @return true if the log segment support sequence id.
+     */
+    public static boolean supportsSequenceId(int version) {
+        return version >= LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value;
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java
new file mode 100644
index 0000000..8a4a30b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.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 org.apache.distributedlog.io.Abortable;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+
+/*
+* 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.
+     *
+     * @param record single log record
+     * @throws IOException
+     */
+    public void write(LogRecord record) throws IOException;
+
+
+    /**
+     * Write a list of log records to the stream.
+     *
+     * @param records list of log records
+     * @throws IOException
+     */
+    @Deprecated
+    public int writeBulk(List<LogRecord> records) throws IOException;
+
+    /**
+     * All data that has been written to the stream so far will be sent to
+     * persistent storage.
+     * The transmission is asynchronous and new data can be still written to the
+     * stream while flushing is performed.
+     *
+     * TODO: rename this to flush()
+     */
+    public long setReadyToFlush() throws IOException;
+
+    /**
+     * Flush and sync all data that is ready to be flush
+     * {@link #setReadyToFlush()} into underlying persistent store.
+     * @throws IOException
+     *
+     * TODO: rename this to commit()
+     */
+    public long flushAndSync() throws IOException;
+
+    /**
+     * Flushes all the data up to this point,
+     * adds the end of stream marker and marks the stream
+     * as read-only in the metadata. No appends to the
+     * stream will be allowed after this point
+     *
+     * @throws IOException
+     */
+    public void markEndOfStream() throws IOException;
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/MaxLogSegmentSequenceNo.java b/distributedlog-core/src/main/java/org/apache/distributedlog/MaxLogSegmentSequenceNo.java
new file mode 100644
index 0000000..a76f547
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/MaxLogSegmentSequenceNo.java
@@ -0,0 +1,72 @@
+/**
+ * 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 org.apache.distributedlog.util.DLUtils;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+
+/**
+ * Utility class for storing and reading max ledger sequence number
+ */
+class MaxLogSegmentSequenceNo {
+
+    Version version;
+    long maxSeqNo;
+
+    MaxLogSegmentSequenceNo(Versioned<byte[]> logSegmentsData) {
+        if (null != logSegmentsData
+                && null != logSegmentsData.getValue()
+                && null != logSegmentsData.getVersion()) {
+            version = logSegmentsData.getVersion();
+            try {
+                maxSeqNo = DLUtils.deserializeLogSegmentSequenceNumber(logSegmentsData.getValue());
+            } catch (NumberFormatException nfe) {
+                maxSeqNo = DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
+            }
+        } else {
+            maxSeqNo = DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO;
+            if (null != logSegmentsData && null != logSegmentsData.getVersion()) {
+                version = logSegmentsData.getVersion();
+            } else {
+                throw new IllegalStateException("Invalid MaxLogSegmentSequenceNo found - " + logSegmentsData);
+            }
+        }
+    }
+
+    synchronized Version getVersion() {
+        return version;
+    }
+
+    synchronized long getSequenceNumber() {
+        return maxSeqNo;
+    }
+
+    synchronized MaxLogSegmentSequenceNo update(Version version, long logSegmentSeqNo) {
+        if (version.compare(this.version) == Version.Occurred.AFTER) {
+            this.version = version;
+            this.maxSeqNo = logSegmentSeqNo;
+        }
+        return this;
+    }
+
+    public synchronized Versioned<Long> getVersionedData(long seqNo) {
+        return new Versioned<Long>(seqNo, version);
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/MaxTxId.java b/distributedlog-core/src/main/java/org/apache/distributedlog/MaxTxId.java
new file mode 100644
index 0000000..8f077e2
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/MaxTxId.java
@@ -0,0 +1,72 @@
+/**
+ * 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 org.apache.distributedlog.util.DLUtils;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Utility class for storing and reading
+ * the max seen txid in zookeeper
+ */
+class MaxTxId {
+    static final Logger LOG = LoggerFactory.getLogger(MaxTxId.class);
+
+    private Version version;
+    private long currentMax;
+
+    MaxTxId(Versioned<byte[]> maxTxIdData) {
+        if (null != maxTxIdData
+                && null != maxTxIdData.getValue()
+                && null != maxTxIdData.getVersion()) {
+            this.version = maxTxIdData.getVersion();
+            try {
+                this.currentMax = DLUtils.deserializeTransactionId(maxTxIdData.getValue());
+            } catch (NumberFormatException e) {
+                LOG.warn("Invalid txn id stored in {}", e);
+                this.currentMax = DistributedLogConstants.INVALID_TXID;
+            }
+        } else {
+            this.currentMax = DistributedLogConstants.INVALID_TXID;
+            if (null != maxTxIdData && null != maxTxIdData.getVersion()) {
+                this.version = maxTxIdData.getVersion();
+            } else {
+                throw new IllegalStateException("Invalid MaxTxId found - " + maxTxIdData);
+            }
+        }
+    }
+
+    synchronized void update(Version version, long txId) {
+        if (version.compare(this.version) == Version.Occurred.AFTER) {
+            this.version = version;
+            this.currentMax = txId;
+        }
+    }
+
+    synchronized long get() {
+        return currentMax;
+    }
+
+    public synchronized Versioned<Long> getVersionedData(long txId) {
+        return new Versioned<Long>(Math.max(txId, get()), version);
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/MetadataAccessor.java b/distributedlog-core/src/main/java/org/apache/distributedlog/MetadataAccessor.java
new file mode 100644
index 0000000..3d1d601
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/MetadataAccessor.java
@@ -0,0 +1,43 @@
+/**
+ * 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 org.apache.distributedlog.io.AsyncCloseable;
+
+import java.io.Closeable;
+import java.io.IOException;
+
+public interface MetadataAccessor extends Closeable, AsyncCloseable {
+    /**
+     * Get the name of the stream managed by this log manager
+     * @return streamName
+     */
+    public String getStreamName();
+
+    public void createOrUpdateMetadata(byte[] metadata) throws IOException;
+
+    public void deleteMetadata() throws IOException;
+
+    public byte[] getMetadata() throws IOException;
+
+    /**
+     * Close the distributed log metadata, freeing any resources it may hold.
+     */
+    public void close() throws IOException;
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
new file mode 100644
index 0000000..0b8c55a
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
@@ -0,0 +1,992 @@
+/**
+ * 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 com.google.common.base.Stopwatch;
+import com.google.common.base.Ticker;
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.exceptions.AlreadyTruncatedTransactionException;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.EndOfLogSegmentException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.io.AsyncCloseable;
+import org.apache.distributedlog.logsegment.LogSegmentEntryReader;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.logsegment.LogSegmentFilter;
+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;
+import java.util.List;
+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;
+
+/**
+ * New ReadAhead Reader that uses {@link org.apache.distributedlog.logsegment.LogSegmentEntryReader}.
+ *
+ * NOTE: all the state changes happen in the same thread. All *unsafe* methods should be submitted to the order
+ * scheduler using stream name as the key.
+ */
+public class ReadAheadEntryReader implements
+        AsyncCloseable,
+        LogSegmentListener,
+        LogSegmentEntryReader.StateChangeListener,
+        FutureEventListener<List<Entry.Reader>> {
+
+    private static final Logger logger = LoggerFactory.getLogger(ReadAheadEntryReader.class);
+
+    //
+    // 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;
+        }
+    };
+
+    //
+    // Internal Classes
+    //
+
+    class SegmentReader implements FutureEventListener<LogSegmentEntryReader> {
+
+        private LogSegmentMetadata metadata;
+        private final long startEntryId;
+        private Future<LogSegmentEntryReader> openFuture = null;
+        private LogSegmentEntryReader reader = null;
+        private boolean isStarted = false;
+        private boolean isClosed = false;
+
+        SegmentReader(LogSegmentMetadata metadata,
+                      long startEntryId) {
+            this.metadata = metadata;
+            this.startEntryId = startEntryId;
+        }
+
+        synchronized LogSegmentEntryReader getEntryReader() {
+            return reader;
+        }
+
+        synchronized boolean isBeyondLastAddConfirmed() {
+            return null != reader && reader.isBeyondLastAddConfirmed();
+        }
+
+        synchronized LogSegmentMetadata getSegment() {
+            return metadata;
+        }
+
+        synchronized boolean isReaderOpen() {
+            return null != openFuture;
+        }
+
+        synchronized void openReader() {
+            if (null != openFuture) {
+                return;
+            }
+            openFuture = entryStore.openReader(metadata, startEntryId).addEventListener(this);
+        }
+
+        synchronized boolean isReaderStarted() {
+            return isStarted;
+        }
+
+        synchronized void startRead() {
+            if (isStarted) {
+                return;
+            }
+            isStarted = true;
+            if (null != reader) {
+                reader.start();
+            } else {
+                openFuture.onSuccess(START_READER_FUNC);
+            }
+        }
+
+        synchronized Future<List<Entry.Reader>> readNext() {
+            if (null != reader) {
+                checkCatchingUpStatus(reader);
+                return reader.readNext(numReadAheadEntries);
+            } else {
+                return openFuture.flatMap(readFunc);
+            }
+        }
+
+        synchronized void updateLogSegmentMetadata(final LogSegmentMetadata segment) {
+            if (null != reader) {
+                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;
+                    }
+                });
+            }
+        }
+
+        @Override
+        synchronized public void onSuccess(LogSegmentEntryReader reader) {
+            this.reader = reader;
+            if (reader.getSegment().isInProgress()) {
+                reader.registerListener(ReadAheadEntryReader.this);
+            }
+        }
+
+        @Override
+        public void onFailure(Throwable cause) {
+            // no-op, the failure will be propagated on first read.
+        }
+
+        synchronized boolean isClosed() {
+            return isClosed;
+        }
+
+        synchronized Future<Void> close() {
+            if (null == openFuture) {
+                return Future.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() {
+                    synchronized (SegmentReader.this) {
+                        isClosed = true;
+                    }
+                    return null;
+                }
+            });
+        }
+    }
+
+    private class ReadEntriesFunc extends AbstractFunction1<LogSegmentEntryReader, Future<List<Entry.Reader>>> {
+
+        private final int numEntries;
+
+        ReadEntriesFunc(int numEntries) {
+            this.numEntries = numEntries;
+        }
+
+        @Override
+        public Future<List<Entry.Reader>> apply(LogSegmentEntryReader reader) {
+            checkCatchingUpStatus(reader);
+            return reader.readNext(numEntries);
+        }
+    }
+
+    private abstract class CloseableRunnable implements Runnable {
+
+        @Override
+        public void run() {
+            synchronized (ReadAheadEntryReader.this) {
+                if (null != closePromise) {
+                    return;
+                }
+            }
+            try {
+                safeRun();
+            } catch (Throwable cause) {
+                logger.error("Caught unexpected exception : ", cause);
+            }
+        }
+
+        abstract void safeRun();
+
+    }
+
+    //
+    // 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;
+        }
+    };
+
+    //
+    // Resources
+    //
+    private final DistributedLogConfiguration conf;
+    private final BKLogReadHandler readHandler;
+    private final LogSegmentEntryStore entryStore;
+    private final OrderedScheduler scheduler;
+
+    //
+    // Parameters
+    //
+    private final String streamName;
+    private final DLSN fromDLSN;
+    private final int maxCachedEntries;
+    private final int numReadAheadEntries;
+    private final int idleWarnThresholdMillis;
+
+    //
+    // Cache
+    //
+    private final LinkedBlockingQueue<Entry.Reader> entryQueue;
+
+    //
+    // State of the reader
+    //
+
+    private final AtomicBoolean started = new AtomicBoolean(false);
+    private boolean isInitialized = false;
+    private boolean readAheadPaused = false;
+    private Promise<Void> closePromise = null;
+    // segment readers
+    private long currentSegmentSequenceNumber;
+    private SegmentReader currentSegmentReader;
+    private SegmentReader nextSegmentReader;
+    private DLSN lastDLSN;
+    private final EntryPosition nextEntryPosition;
+    private volatile boolean isCatchingUp = true;
+    private final LinkedList<SegmentReader> segmentReaders;
+    private final LinkedList<SegmentReader> segmentReadersToClose;
+    // last exception that this reader encounters
+    private final AtomicReference<IOException> lastException = new AtomicReference<IOException>(null);
+    // last entry added time
+    private final Stopwatch lastEntryAddedTime;
+    // state change notification
+    private final CopyOnWriteArraySet<AsyncNotification> stateChangeNotifications =
+            new CopyOnWriteArraySet<AsyncNotification>();
+    // idle reader check task
+    private final ScheduledFuture<?> idleReaderCheckTask;
+
+    //
+    // Stats
+    //
+    private final AlertStatsLogger alertStatsLogger;
+
+    public ReadAheadEntryReader(String streamName,
+                                DLSN fromDLSN,
+                                DistributedLogConfiguration conf,
+                                BKLogReadHandler readHandler,
+                                LogSegmentEntryStore entryStore,
+                                OrderedScheduler scheduler,
+                                Ticker ticker,
+                                AlertStatsLogger alertStatsLogger) {
+        this.streamName = streamName;
+        this.fromDLSN = lastDLSN = fromDLSN;
+        this.nextEntryPosition = new EntryPosition(
+                fromDLSN.getLogSegmentSequenceNo(),
+                fromDLSN.getEntryId());
+        this.conf = conf;
+        this.maxCachedEntries = conf.getReadAheadMaxRecords();
+        this.numReadAheadEntries = conf.getReadAheadBatchSize();
+        this.idleWarnThresholdMillis = conf.getReaderIdleWarnThresholdMillis();
+        this.readHandler = readHandler;
+        this.entryStore = entryStore;
+        this.scheduler = scheduler;
+        this.readFunc = new ReadEntriesFunc(numReadAheadEntries);
+        this.alertStatsLogger = alertStatsLogger;
+
+        // create the segment reader list
+        this.segmentReaders = new LinkedList<SegmentReader>();
+        this.segmentReadersToClose = new LinkedList<SegmentReader>();
+        // create the readahead entry queue
+        this.entryQueue = new LinkedBlockingQueue<Entry.Reader>();
+
+        // start the idle reader detection
+        lastEntryAddedTime = Stopwatch.createStarted(ticker);
+        // start the idle reader check task
+        idleReaderCheckTask = scheduleIdleReaderTaskIfNecessary();
+    }
+
+    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();
+                }
+            }, idleWarnThresholdMillis, idleWarnThresholdMillis, TimeUnit.MILLISECONDS);
+        }
+        return null;
+    }
+
+    private void unsafeCheckIfReadAheadIsIdle() {
+        boolean forceReadLogSegments =
+                (null == currentSegmentReader) || currentSegmentReader.isBeyondLastAddConfirmed();
+        if (forceReadLogSegments) {
+            readHandler.readLogSegmentsFromStore(
+                    LogSegmentMetadata.COMPARATOR,
+                    LogSegmentFilter.DEFAULT_FILTER,
+                    null
+            ).addEventListener(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
+                }
+
+                @Override
+                public void onSuccess(Versioned<List<LogSegmentMetadata>> segments) {
+                    onSegmentsUpdated(segments.getValue());
+                }
+            });
+        }
+    }
+
+    private void cancelIdleReaderTask() {
+        if (null != idleReaderCheckTask) {
+            idleReaderCheckTask.cancel(true);
+        }
+    }
+
+    @VisibleForTesting
+    EntryPosition getNextEntryPosition() {
+        return nextEntryPosition;
+    }
+
+    @VisibleForTesting
+    SegmentReader getCurrentSegmentReader() {
+        return currentSegmentReader;
+    }
+
+    @VisibleForTesting
+    long getCurrentSegmentSequenceNumber() {
+        return currentSegmentSequenceNumber;
+    }
+
+    @VisibleForTesting
+    SegmentReader getNextSegmentReader() {
+        return nextSegmentReader;
+    }
+
+    @VisibleForTesting
+    LinkedList<SegmentReader> getSegmentReaders() {
+        return segmentReaders;
+    }
+
+    @VisibleForTesting
+    boolean isInitialized() {
+        return isInitialized;
+    }
+
+    private void orderedSubmit(Runnable runnable) {
+        synchronized (this) {
+            if (null != closePromise) {
+                return;
+            }
+        }
+        try {
+            scheduler.submit(streamName, runnable);
+        } catch (RejectedExecutionException ree) {
+            logger.debug("Failed to submit and execute an operation for readhead entry reader of {}",
+                    streamName, ree);
+        }
+    }
+
+    public void start(final List<LogSegmentMetadata> segmentList) {
+        logger.info("Starting the readahead entry reader for {} : segments = {}",
+                readHandler.getFullyQualifiedName(), segmentList);
+        started.set(true);
+        processLogSegments(segmentList);
+    }
+
+    private void removeClosedSegmentReaders() {
+        orderedSubmit(new CloseableRunnable() {
+            @Override
+            void safeRun() {
+                unsafeRemoveClosedSegmentReaders();
+            }
+        });
+    }
+
+    private void unsafeRemoveClosedSegmentReaders() {
+        SegmentReader reader = segmentReadersToClose.peekFirst();
+        while (null != reader) {
+            if (reader.isClosed()) {
+                segmentReadersToClose.pollFirst();
+                reader = segmentReadersToClose.peekFirst();
+            } else {
+                break;
+            }
+        }
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        final Promise<Void> closeFuture;
+        synchronized (this) {
+            if (null != closePromise) {
+                return closePromise;
+            }
+            closePromise = closeFuture = new Promise<Void>();
+        }
+
+        // cancel the idle reader task
+        cancelIdleReaderTask();
+
+        // use runnable here instead of CloseableRunnable,
+        // because we need this to be executed
+        try {
+            scheduler.submit(streamName, new Runnable() {
+                @Override
+                public void run() {
+                    unsafeAsyncClose(closeFuture);
+                }
+            });
+        } catch (RejectedExecutionException ree) {
+            logger.warn("Scheduler has been shutdown before closing the readahead entry reader for stream {}",
+                    streamName, ree);
+            unsafeAsyncClose(closeFuture);
+        }
+
+        return closeFuture;
+    }
+
+    private void unsafeAsyncClose(Promise<Void> closePromise) {
+        List<Future<Void>> closeFutures = Lists.newArrayListWithExpectedSize(
+                segmentReaders.size() + segmentReadersToClose.size() + 1);
+        if (null != currentSegmentReader) {
+            segmentReadersToClose.add(currentSegmentReader);
+        }
+        if (null != nextSegmentReader) {
+            segmentReadersToClose.add(nextSegmentReader);
+        }
+        for (SegmentReader reader : segmentReaders) {
+            segmentReadersToClose.add(reader);
+        }
+        segmentReaders.clear();
+        for (SegmentReader reader : segmentReadersToClose) {
+            closeFutures.add(reader.close());
+        }
+        Futures.collect(closeFutures).proxyTo(closePromise);
+    }
+
+    //
+    // Reader State Changes
+    //
+
+    ReadAheadEntryReader addStateChangeNotification(AsyncNotification notification) {
+        this.stateChangeNotifications.add(notification);
+        return this;
+    }
+
+    ReadAheadEntryReader removeStateChangeNotification(AsyncNotification notification) {
+        this.stateChangeNotifications.remove(notification);
+        return this;
+    }
+
+    private void notifyStateChangeOnSuccess() {
+        for (AsyncNotification notification : stateChangeNotifications) {
+            notification.notifyOnOperationComplete();
+        }
+    }
+
+    private void notifyStateChangeOnFailure(Throwable cause) {
+        for (AsyncNotification notification : stateChangeNotifications) {
+            notification.notifyOnError(cause);
+        }
+    }
+
+    void setLastException(IOException cause) {
+        if (!lastException.compareAndSet(null, cause)) {
+            logger.debug("last exception has already been set to ", lastException.get());
+        }
+        // the exception is set and notify the state change
+        notifyStateChangeOnFailure(cause);
+    }
+
+    void checkLastException() throws IOException {
+        if (null != lastException.get()) {
+            throw lastException.get();
+        }
+    }
+
+    void checkCatchingUpStatus(LogSegmentEntryReader reader) {
+        if (reader.getSegment().isInProgress()
+                && isCatchingUp
+                && reader.hasCaughtUpOnInprogress()) {
+            logger.info("ReadAhead for {} is caught up at entry {} @ log segment {}.",
+                    new Object[] { readHandler.getFullyQualifiedName(),
+                            reader.getLastAddConfirmed(), reader.getSegment() });
+            isCatchingUp = false;
+        }
+    }
+
+    void markCaughtup() {
+        if (isCatchingUp) {
+            isCatchingUp = false;
+            logger.info("ReadAhead for {} is caught up", readHandler.getFullyQualifiedName());
+        }
+    }
+
+    public boolean isReadAheadCaughtUp() {
+        return !isCatchingUp;
+    }
+
+    @Override
+    public void onCaughtupOnInprogress() {
+        markCaughtup();
+    }
+
+    //
+    // ReadAhead State Machine
+    //
+
+    @Override
+    public void onSuccess(List<Entry.Reader> entries) {
+        lastEntryAddedTime.reset().start();
+        for (Entry.Reader entry : entries) {
+            entryQueue.add(entry);
+        }
+        if (!entries.isEmpty()) {
+            Entry.Reader lastEntry = entries.get(entries.size() - 1);
+            nextEntryPosition.advance(lastEntry.getLSSN(), lastEntry.getEntryId() + 1);
+        }
+        // notify on data available
+        notifyStateChangeOnSuccess();
+        if (entryQueue.size() >= maxCachedEntries) {
+            pauseReadAheadOnCacheFull();
+        } else {
+            scheduleReadNext();
+        }
+    }
+
+    @Override
+    public void onFailure(Throwable cause) {
+        if (cause instanceof EndOfLogSegmentException) {
+            // we reach end of the log segment
+            moveToNextLogSegment();
+            return;
+        }
+        if (cause instanceof IOException) {
+            setLastException((IOException) cause);
+        } else {
+            setLastException(new UnexpectedException("Unexpected non I/O exception", cause));
+        }
+    }
+
+    private synchronized void invokeReadAhead() {
+        if (readAheadPaused) {
+            scheduleReadNext();
+            readAheadPaused = false;
+        }
+    }
+
+    private synchronized void pauseReadAheadOnCacheFull() {
+        this.readAheadPaused = true;
+        if (!isCacheFull()) {
+            invokeReadAhead();
+        }
+    }
+
+    private synchronized void pauseReadAheadOnNoMoreLogSegments() {
+        this.readAheadPaused = true;
+    }
+
+    //
+    // Cache Related Methods
+    //
+
+    public Entry.Reader getNextReadAheadEntry(long waitTime, TimeUnit waitTimeUnit) throws IOException {
+        if (null != lastException.get()) {
+            throw lastException.get();
+        }
+        Entry.Reader entry;
+        try {
+            entry = entryQueue.poll(waitTime, waitTimeUnit);
+        } catch (InterruptedException e) {
+            throw new DLInterruptedException("Interrupted on waiting next readahead entry : ", e);
+        }
+        try {
+            return entry;
+        } finally {
+            // resume readahead if the cache becomes empty
+            if (null != entry && !isCacheFull()) {
+                invokeReadAhead();
+            }
+        }
+    }
+
+    /**
+     * Return number cached entries.
+     *
+     * @return number cached entries.
+     */
+    public int getNumCachedEntries() {
+        return entryQueue.size();
+    }
+
+    /**
+     * Return if the cache is full.
+     *
+     * @return true if the cache is full, otherwise false.
+     */
+    public boolean isCacheFull() {
+        return getNumCachedEntries() >= maxCachedEntries;
+    }
+
+    @VisibleForTesting
+    public boolean isCacheEmpty() {
+        return entryQueue.isEmpty();
+    }
+
+    /**
+     * Check whether the readahead becomes stall.
+     *
+     * @param idleReaderErrorThreshold idle reader error threshold
+     * @param timeUnit time unit of the idle reader error threshold
+     * @return true if the readahead becomes stall, otherwise false.
+     */
+    public boolean isReaderIdle(int idleReaderErrorThreshold, TimeUnit timeUnit) {
+        return (lastEntryAddedTime.elapsed(timeUnit) > idleReaderErrorThreshold);
+    }
+
+    //
+    // LogSegment Management
+    //
+
+    void processLogSegments(final List<LogSegmentMetadata> segments) {
+        orderedSubmit(new CloseableRunnable() {
+            @Override
+            void safeRun() {
+                unsafeProcessLogSegments(segments);
+            }
+        });
+    }
+
+    private void unsafeProcessLogSegments(List<LogSegmentMetadata> segments) {
+        if (isInitialized) {
+            unsafeReinitializeLogSegments(segments);
+        } else {
+            unsafeInitializeLogSegments(segments);
+        }
+    }
+
+    /**
+     * Update the log segment metadata.
+     *
+     * @param reader the reader to update the metadata
+     * @param newMetadata the new metadata received
+     * @return true if successfully, false on encountering errors
+     */
+    private boolean updateLogSegmentMetadata(SegmentReader reader,
+                                             LogSegmentMetadata newMetadata) {
+        if (reader.getSegment().getLogSegmentSequenceNumber() != newMetadata.getLogSegmentSequenceNumber()) {
+            setLastException(new DLIllegalStateException("Inconsistent state found in entry reader for "
+                    + streamName + " : current segment = " + reader.getSegment() + ", new segment = " + newMetadata));
+            return false;
+        }
+        if (!reader.getSegment().isInProgress() && newMetadata.isInProgress()) {
+            setLastException(new DLIllegalStateException("An inprogress log segment " + newMetadata
+                    + " received after a closed log segment " + reader.getSegment() + " on reading segment "
+                    + newMetadata.getLogSegmentSequenceNumber() + " @ stream " + streamName));
+            return false;
+        }
+        if (reader.getSegment().isInProgress() && !newMetadata.isInProgress()) {
+            reader.updateLogSegmentMetadata(newMetadata);
+        }
+        return true;
+    }
+
+    /**
+     * Reinitialize the log segments
+     */
+    private void unsafeReinitializeLogSegments(List<LogSegmentMetadata> segments) {
+        logger.info("Reinitialize log segments with {}", segments);
+        int segmentIdx = 0;
+        for (; segmentIdx < segments.size(); segmentIdx++) {
+            LogSegmentMetadata segment = segments.get(segmentIdx);
+            if (segment.getLogSegmentSequenceNumber() < currentSegmentSequenceNumber) {
+                continue;
+            }
+            break;
+        }
+        if (segmentIdx >= segments.size()) {
+            return;
+        }
+        LogSegmentMetadata segment = segments.get(segmentIdx);
+        if (null != currentSegmentReader) {
+            if (!updateLogSegmentMetadata(currentSegmentReader, segment)) {
+                return;
+            }
+        } else {
+            if (currentSegmentSequenceNumber != segment.getLogSegmentSequenceNumber()) {
+                setLastException(new DLIllegalStateException("Inconsistent state found in entry reader for "
+                        + streamName + " : current segment sn = " + currentSegmentSequenceNumber
+                        + ", new segment sn = " + segment.getLogSegmentSequenceNumber()));
+                return;
+            }
+        }
+        segmentIdx++;
+        if (segmentIdx >= segments.size()) {
+            return;
+        }
+        // check next segment
+        segment = segments.get(segmentIdx);
+        if (null != nextSegmentReader) {
+            if (!updateLogSegmentMetadata(nextSegmentReader, segment)) {
+                return;
+            }
+            segmentIdx++;
+        }
+        // check the segment readers in the queue
+        for (int readerIdx = 0;
+             readerIdx < segmentReaders.size() && segmentIdx < segments.size();
+             readerIdx++, segmentIdx++) {
+            SegmentReader reader = segmentReaders.get(readerIdx);
+            segment = segments.get(segmentIdx);
+            if (!updateLogSegmentMetadata(reader, segment)) {
+                return;
+            }
+        }
+        // add the remaining segments to the reader queue
+        for (; segmentIdx < segments.size(); segmentIdx++) {
+            segment = segments.get(segmentIdx);
+            SegmentReader reader = new SegmentReader(segment, 0L);
+            reader.openReader();
+            segmentReaders.add(reader);
+        }
+        if (null == currentSegmentReader) {
+            unsafeMoveToNextLogSegment();
+        }
+        // resume readahead if necessary
+        invokeReadAhead();
+    }
+
+    /**
+     * Initialize the reader with the log <i>segments</i>.
+     *
+     * @param segments list of log segments
+     */
+    private void unsafeInitializeLogSegments(List<LogSegmentMetadata> segments) {
+        if (segments.isEmpty()) {
+            // not initialize the background reader, until the first log segment is notified
+            return;
+        }
+        boolean skipTruncatedLogSegments = true;
+        DLSN dlsnToStart = fromDLSN;
+        // positioning the reader
+        for (int i = 0; i < segments.size(); i++) {
+            LogSegmentMetadata segment = segments.get(i);
+            // skip any log segments that have smaller log segment sequence numbers
+            if (segment.getLogSegmentSequenceNumber() < fromDLSN.getLogSegmentSequenceNo()) {
+                continue;
+            }
+            // if the log segment is truncated, skip it.
+            if (skipTruncatedLogSegments &&
+                    !conf.getIgnoreTruncationStatus() &&
+                    segment.isTruncated()) {
+                continue;
+            }
+            // if the log segment is partially truncated, move the start dlsn to the min active dlsn
+            if (skipTruncatedLogSegments &&
+                    !conf.getIgnoreTruncationStatus() &&
+                    segment.isPartiallyTruncated()) {
+                if (segment.getMinActiveDLSN().compareTo(fromDLSN) > 0) {
+                    dlsnToStart = segment.getMinActiveDLSN();
+                }
+            }
+            skipTruncatedLogSegments = false;
+            if (!isAllowedToPosition(segment, dlsnToStart)) {
+                logger.error("segment {} is not allowed to position at {}", segment, dlsnToStart);
+                return;
+            }
+
+            SegmentReader reader = new SegmentReader(segment,
+                    segment.getLogSegmentSequenceNumber() == dlsnToStart.getLogSegmentSequenceNo()
+                            ? dlsnToStart.getEntryId() : 0L);
+            segmentReaders.add(reader);
+        }
+        if (segmentReaders.isEmpty()) {
+            // not initialize the background reader, until the first log segment is available to read
+            return;
+        }
+        currentSegmentReader = segmentReaders.pollFirst();
+        currentSegmentReader.openReader();
+        currentSegmentReader.startRead();
+        currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
+        unsafeReadNext(currentSegmentReader);
+        if (!segmentReaders.isEmpty()) {
+            for (SegmentReader reader : segmentReaders) {
+                reader.openReader();
+            }
+            unsafePrefetchNextSegment(true);
+        }
+        // mark the reader initialized
+        isInitialized = true;
+    }
+
+    private void unsafePrefetchNextSegment(boolean onlyInprogressLogSegment) {
+        SegmentReader nextReader = segmentReaders.peekFirst();
+        // open the next log segment if it is inprogress
+        if (null != nextReader) {
+            if (onlyInprogressLogSegment && !nextReader.getSegment().isInProgress()) {
+                return;
+            }
+            nextReader.startRead();
+            nextSegmentReader = nextReader;
+            segmentReaders.pollFirst();
+        }
+    }
+
+    /**
+     * Check if we are allowed to position the reader at <i>fromDLSN</i>.
+     *
+     * @return true if it is allowed, otherwise false.
+     */
+    private boolean isAllowedToPosition(LogSegmentMetadata segment, DLSN fromDLSN) {
+        if (segment.isTruncated()
+                && segment.getLastDLSN().compareTo(fromDLSN) >= 0
+                && !conf.getIgnoreTruncationStatus()) {
+            setLastException(new AlreadyTruncatedTransactionException(streamName
+                    + " : trying to position read ahead at " + fromDLSN
+                    + " on a segment " + segment + " that is already marked as truncated"));
+            return false;
+        }
+        if (segment.isPartiallyTruncated() &&
+                segment.getMinActiveDLSN().compareTo(fromDLSN) > 0) {
+            if (conf.getAlertWhenPositioningOnTruncated()) {
+                alertStatsLogger.raise("Trying to position reader on {} when {} is marked partially truncated",
+                    fromDLSN, segment);
+            }
+            if (!conf.getIgnoreTruncationStatus()) {
+                logger.error("{}: Trying to position reader on {} when {} is marked partially truncated",
+                        new Object[]{ streamName, fromDLSN, segment });
+
+                setLastException(new AlreadyTruncatedTransactionException(streamName
+                        + " : trying to position read ahead at " + fromDLSN
+                        + " on a segment " + segment + " that is already marked as truncated"));
+                return false;
+            }
+        }
+        return true;
+    }
+
+    void moveToNextLogSegment() {
+        orderedSubmit(new CloseableRunnable() {
+            @Override
+            void safeRun() {
+                unsafeMoveToNextLogSegment();
+            }
+        });
+    }
+
+    private void unsafeMoveToNextLogSegment() {
+        if (null != currentSegmentReader) {
+            segmentReadersToClose.add(currentSegmentReader);
+            currentSegmentReader.close().ensure(removeClosedSegmentReadersFunc);
+            logger.debug("close current segment reader {}", currentSegmentReader.getSegment());
+            currentSegmentReader = null;
+        }
+        boolean hasSegmentToRead = false;
+        if (null != nextSegmentReader) {
+            currentSegmentReader = nextSegmentReader;
+            logger.debug("move to read segment {}", currentSegmentReader.getSegment());
+            currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
+            nextSegmentReader = null;
+            // start reading
+            unsafeReadNext(currentSegmentReader);
+            unsafePrefetchNextSegment(true);
+            hasSegmentToRead = true;
+        } else {
+            unsafePrefetchNextSegment(false);
+            if (null != nextSegmentReader) {
+                currentSegmentReader = nextSegmentReader;
+                logger.debug("move to read segment {}", currentSegmentReader.getSegment());
+                currentSegmentSequenceNumber = currentSegmentReader.getSegment().getLogSegmentSequenceNumber();
+                nextSegmentReader = null;
+                unsafeReadNext(currentSegmentReader);
+                unsafePrefetchNextSegment(true);
+                hasSegmentToRead = true;
+            }
+        }
+        if (!hasSegmentToRead) { // no more segment to read, wait until new log segment arrive
+            if (isCatchingUp) {
+                logger.info("ReadAhead for {} is caught up and no log segments to read now",
+                        readHandler.getFullyQualifiedName());
+                isCatchingUp = false;
+            }
+            pauseReadAheadOnNoMoreLogSegments();
+        }
+    }
+
+    void scheduleReadNext() {
+        orderedSubmit(new CloseableRunnable() {
+            @Override
+            void safeRun() {
+                if (null == currentSegmentReader) {
+                    pauseReadAheadOnNoMoreLogSegments();
+                    return;
+                }
+                unsafeReadNext(currentSegmentReader);
+            }
+        });
+    }
+
+    private void unsafeReadNext(SegmentReader reader) {
+        reader.readNext().addEventListener(this);
+    }
+
+    @Override
+    public void onSegmentsUpdated(List<LogSegmentMetadata> segments) {
+        if (!started.get()) {
+            return;
+        }
+        logger.info("segments is updated with {}", segments);
+        processLogSegments(segments);
+    }
+
+    @Override
+    public void onLogStreamDeleted() {
+        setLastException(new LogNotFoundException("Log stream "
+                + streamName + " is deleted"));
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java
new file mode 100644
index 0000000..9935d5f
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java
@@ -0,0 +1,782 @@
+/**
+ * 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 java.io.IOException;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.logsegment.LogSegmentRandomAccessEntryReader;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.selector.FirstDLSNNotLessThanSelector;
+import org.apache.distributedlog.selector.FirstTxIdNotLessThanSelector;
+import org.apache.distributedlog.selector.LastRecordSelector;
+import org.apache.distributedlog.selector.LogRecordSelector;
+import org.apache.distributedlog.util.FutureUtils.FutureEventListenerRunnable;
+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.BoxedUnit;
+
+/**
+ * Utility function for readers
+ */
+public class ReadUtils {
+
+    static final Logger LOG = LoggerFactory.getLogger(ReadUtils.class);
+
+    private static final int MIN_SEARCH_BATCH_SIZE = 2;
+
+    //
+    // Read First & Last Record Functions
+    //
+
+    /**
+     * Read last record from a log segment.
+     *
+     * @param streamName
+     *          fully qualified stream name (used for logging)
+     * @param l
+     *          log segment metadata.
+     * @param fence
+     *          whether to fence the log segment.
+     * @param includeControl
+     *          whether to include control record.
+     * @param includeEndOfStream
+     *          whether to include end of stream.
+     * @param scanStartBatchSize
+     *          first num entries used for read last record scan
+     * @param scanMaxBatchSize
+     *          max num entries used for read last record scan
+     * @param numRecordsScanned
+     *          num of records scanned to get last record
+     * @param executorService
+     *          executor service used for processing entries
+     * @param entryStore
+     *          log segment entry store
+     * @return a future with last record.
+     */
+    public static Future<LogRecordWithDLSN> asyncReadLastRecord(
+            final String streamName,
+            final LogSegmentMetadata l,
+            final boolean fence,
+            final boolean includeControl,
+            final boolean includeEndOfStream,
+            final int scanStartBatchSize,
+            final int scanMaxBatchSize,
+            final AtomicInteger numRecordsScanned,
+            final ExecutorService executorService,
+            final LogSegmentEntryStore entryStore) {
+        final LogRecordSelector selector = new LastRecordSelector();
+        return asyncReadRecord(streamName, l, fence, includeControl, includeEndOfStream, scanStartBatchSize,
+                               scanMaxBatchSize, numRecordsScanned, executorService, entryStore,
+                               selector, true /* backward */, 0L);
+    }
+
+    /**
+     * Read first record from a log segment with a DLSN larger than that given.
+     *
+     * @param streamName
+     *          fully qualified stream name (used for logging)
+     * @param l
+     *          log segment metadata.
+     * @param scanStartBatchSize
+     *          first num entries used for read last record scan
+     * @param scanMaxBatchSize
+     *          max num entries used for read last record scan
+     * @param numRecordsScanned
+     *          num of records scanned to get last record
+     * @param executorService
+     *          executor service used for processing entries
+     * @param entryStore
+     *          log segment entry store
+     * @param dlsn
+     *          threshold dlsn
+     * @return a future with last record.
+     */
+    public static Future<LogRecordWithDLSN> asyncReadFirstUserRecord(
+            final String streamName,
+            final LogSegmentMetadata l,
+            final int scanStartBatchSize,
+            final int scanMaxBatchSize,
+            final AtomicInteger numRecordsScanned,
+            final ExecutorService executorService,
+            final LogSegmentEntryStore entryStore,
+            final DLSN dlsn) {
+        long startEntryId = 0L;
+        if (l.getLogSegmentSequenceNumber() == dlsn.getLogSegmentSequenceNo()) {
+            startEntryId = dlsn.getEntryId();
+        }
+        final LogRecordSelector selector = new FirstDLSNNotLessThanSelector(dlsn);
+        return asyncReadRecord(streamName, l, false, false, false, scanStartBatchSize,
+                               scanMaxBatchSize, numRecordsScanned, executorService, entryStore,
+                               selector, false /* backward */, startEntryId);
+    }
+
+    //
+    // Private methods for scanning log segments
+    //
+
+    private static class ScanContext {
+        // variables to about current scan state
+        final AtomicInteger numEntriesToScan;
+        final AtomicLong curStartEntryId;
+        final AtomicLong curEndEntryId;
+
+        // scan settings
+        final long startEntryId;
+        final long endEntryId;
+        final int scanStartBatchSize;
+        final int scanMaxBatchSize;
+        final boolean includeControl;
+        final boolean includeEndOfStream;
+        final boolean backward;
+
+        // number of records scanned
+        final AtomicInteger numRecordsScanned;
+
+        ScanContext(long startEntryId, long endEntryId,
+                    int scanStartBatchSize,
+                    int scanMaxBatchSize,
+                    boolean includeControl,
+                    boolean includeEndOfStream,
+                    boolean backward,
+                    AtomicInteger numRecordsScanned) {
+            this.startEntryId = startEntryId;
+            this.endEntryId = endEntryId;
+            this.scanStartBatchSize = scanStartBatchSize;
+            this.scanMaxBatchSize = scanMaxBatchSize;
+            this.includeControl = includeControl;
+            this.includeEndOfStream = includeEndOfStream;
+            this.backward = backward;
+            // Scan state
+            this.numEntriesToScan = new AtomicInteger(scanStartBatchSize);
+            if (backward) {
+                this.curStartEntryId = new AtomicLong(
+                        Math.max(startEntryId, (endEntryId - scanStartBatchSize + 1)));
+                this.curEndEntryId = new AtomicLong(endEntryId);
+            } else {
+                this.curStartEntryId = new AtomicLong(startEntryId);
+                this.curEndEntryId = new AtomicLong(
+                        Math.min(endEntryId, (startEntryId + scanStartBatchSize - 1)));
+            }
+            this.numRecordsScanned = numRecordsScanned;
+        }
+
+        boolean moveToNextRange() {
+            if (backward) {
+                return moveBackward();
+            } else {
+                return moveForward();
+            }
+        }
+
+        boolean moveBackward() {
+            long nextEndEntryId = curStartEntryId.get() - 1;
+            if (nextEndEntryId < startEntryId) {
+                // no entries to read again
+                return false;
+            }
+            curEndEntryId.set(nextEndEntryId);
+            // update num entries to scan
+            numEntriesToScan.set(
+                    Math.min(numEntriesToScan.get() * 2, scanMaxBatchSize));
+            // update start entry id
+            curStartEntryId.set(Math.max(startEntryId, nextEndEntryId - numEntriesToScan.get() + 1));
+            return true;
+        }
+
+        boolean moveForward() {
+            long nextStartEntryId = curEndEntryId.get() + 1;
+            if (nextStartEntryId > endEntryId) {
+                // no entries to read again
+                return false;
+            }
+            curStartEntryId.set(nextStartEntryId);
+            // update num entries to scan
+            numEntriesToScan.set(
+                    Math.min(numEntriesToScan.get() * 2, scanMaxBatchSize));
+            // update start entry id
+            curEndEntryId.set(Math.min(endEntryId, nextStartEntryId + numEntriesToScan.get() - 1));
+            return true;
+        }
+    }
+
+    private static class SingleEntryScanContext extends ScanContext {
+        SingleEntryScanContext(long entryId) {
+            super(entryId, entryId, 1, 1, true, true, false, new AtomicInteger(0));
+        }
+    }
+
+    /**
+     * Read record from a given range of log segment entries.
+     *
+     * @param streamName
+     *          fully qualified stream name (used for logging)
+     * @param reader
+     *          log segment random access reader
+     * @param executorService
+     *          executor service used for processing entries
+     * @param context
+     *          scan context
+     * @return a future with the log record.
+     */
+    private static Future<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 long startEntryId = context.curStartEntryId.get();
+        final long endEntryId = context.curEndEntryId.get();
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("{} reading entries [{} - {}] from {}.",
+                    new Object[] { streamName, startEntryId, endEntryId, metadata});
+        }
+        FutureEventListener<List<Entry.Reader>> readEntriesListener =
+            new FutureEventListener<List<Entry.Reader>>() {
+                @Override
+                public void onSuccess(final List<Entry.Reader> entries) {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("{} finished reading entries [{} - {}] from {}",
+                                new Object[]{ streamName, startEntryId, endEntryId, metadata});
+                    }
+                    for (Entry.Reader entry : entries) {
+                        try {
+                            visitEntryRecords(entry, context, selector);
+                        } 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);
+                            return;
+                        }
+                    }
+
+                    LogRecordWithDLSN record = selector.result();
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("{} got record from entries [{} - {}] of {} : {}",
+                                new Object[]{streamName, startEntryId, endEntryId,
+                                        metadata, record});
+                    }
+                    promise.setValue(record);
+                }
+
+                @Override
+                public void onFailure(final Throwable cause) {
+                    promise.setException(cause);
+                }
+            };
+        reader.readEntries(startEntryId, endEntryId)
+                .addEventListener(FutureEventListenerRunnable.of(readEntriesListener, executorService));
+        return promise;
+    }
+
+    /**
+     * Process each record using LogRecordSelector.
+     *
+     * @param entry
+     *          ledger entry
+     * @param context
+     *          scan context
+     * @return log record with dlsn inside the ledger entry
+     * @throws IOException
+     */
+    private static void visitEntryRecords(
+            Entry.Reader entry,
+            ScanContext context,
+            LogRecordSelector selector) throws IOException {
+        LogRecordWithDLSN nextRecord = entry.nextRecord();
+        while (nextRecord != null) {
+            LogRecordWithDLSN record = nextRecord;
+            nextRecord = entry.nextRecord();
+            context.numRecordsScanned.incrementAndGet();
+            if (!context.includeControl && record.isControl()) {
+                continue;
+            }
+            if (!context.includeEndOfStream && record.isEndOfStream()) {
+                continue;
+            }
+            selector.process(record);
+        }
+    }
+
+    /**
+     * Scan entries for the given record.
+     *
+     * @param streamName
+     *          fully qualified stream name (used for logging)
+     * @param reader
+     *          log segment random access reader
+     * @param executorService
+     *          executor service used for processing entries
+     * @param promise
+     *          promise to return desired record.
+     * @param context
+     *          scan context
+     */
+    private static void asyncReadRecordFromEntries(
+            final String streamName,
+            final LogSegmentRandomAccessEntryReader reader,
+            final LogSegmentMetadata metadata,
+            final ExecutorService executorService,
+            final Promise<LogRecordWithDLSN> promise,
+            final ScanContext context,
+            final LogRecordSelector selector) {
+        FutureEventListener<LogRecordWithDLSN> readEntriesListener =
+            new FutureEventListener<LogRecordWithDLSN>() {
+                @Override
+                public void onSuccess(LogRecordWithDLSN value) {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("{} read record from [{} - {}] of {} : {}",
+                                new Object[]{streamName, context.curStartEntryId.get(), context.curEndEntryId.get(),
+                                        metadata, value});
+                    }
+                    if (null != value) {
+                        promise.setValue(value);
+                        return;
+                    }
+                    if (!context.moveToNextRange()) {
+                        // no entries to read again
+                        promise.setValue(null);
+                        return;
+                    }
+                    // scan next range
+                    asyncReadRecordFromEntries(streamName,
+                            reader,
+                            metadata,
+                            executorService,
+                            promise,
+                            context,
+                            selector);
+                }
+
+                @Override
+                public void onFailure(Throwable cause) {
+                    promise.setException(cause);
+                }
+            };
+        asyncReadRecordFromEntries(streamName, reader, metadata, executorService, context, selector)
+                .addEventListener(FutureEventListenerRunnable.of(readEntriesListener, executorService));
+    }
+
+    private static void asyncReadRecordFromLogSegment(
+            final String streamName,
+            final LogSegmentRandomAccessEntryReader reader,
+            final LogSegmentMetadata metadata,
+            final ExecutorService executorService,
+            final int scanStartBatchSize,
+            final int scanMaxBatchSize,
+            final boolean includeControl,
+            final boolean includeEndOfStream,
+            final Promise<LogRecordWithDLSN> promise,
+            final AtomicInteger numRecordsScanned,
+            final LogRecordSelector selector,
+            final boolean backward,
+            final long startEntryId) {
+        final long lastAddConfirmed = reader.getLastAddConfirmed();
+        if (lastAddConfirmed < 0) {
+            if (LOG.isDebugEnabled()) {
+                LOG.debug("Log segment {} is empty for {}.", new Object[] { metadata, streamName });
+            }
+            promise.setValue(null);
+            return;
+        }
+        final ScanContext context = new ScanContext(
+                startEntryId, lastAddConfirmed,
+                scanStartBatchSize, scanMaxBatchSize,
+                includeControl, includeEndOfStream, backward, numRecordsScanned);
+        asyncReadRecordFromEntries(streamName, reader, metadata, executorService,
+                                   promise, context, selector);
+    }
+
+    private static Future<LogRecordWithDLSN> asyncReadRecord(
+            final String streamName,
+            final LogSegmentMetadata l,
+            final boolean fence,
+            final boolean includeControl,
+            final boolean includeEndOfStream,
+            final int scanStartBatchSize,
+            final int scanMaxBatchSize,
+            final AtomicInteger numRecordsScanned,
+            final ExecutorService executorService,
+            final LogSegmentEntryStore entryStore,
+            final LogRecordSelector selector,
+            final boolean backward,
+            final long startEntryId) {
+
+        final Promise<LogRecordWithDLSN> promise = new Promise<LogRecordWithDLSN>();
+
+        FutureEventListener<LogSegmentRandomAccessEntryReader> openReaderListener =
+            new FutureEventListener<LogSegmentRandomAccessEntryReader>() {
+                @Override
+                public void onSuccess(final LogSegmentRandomAccessEntryReader reader) {
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("{} Opened log segment {} for reading record",
+                                streamName, l);
+                    }
+                    promise.ensure(new AbstractFunction0<BoxedUnit>() {
+                        @Override
+                        public BoxedUnit apply() {
+                            reader.asyncClose();
+                            return BoxedUnit.UNIT;
+                        }
+                    });
+                    if (LOG.isDebugEnabled()) {
+                        LOG.debug("{} {} scanning {}.", new Object[]{
+                                (backward ? "backward" : "forward"), streamName, l});
+                    }
+                    asyncReadRecordFromLogSegment(
+                            streamName, reader, l, executorService,
+                            scanStartBatchSize, scanMaxBatchSize,
+                            includeControl, includeEndOfStream,
+                            promise, numRecordsScanned, selector, backward, startEntryId);
+                }
+
+                @Override
+                public void onFailure(final Throwable cause) {
+                    promise.setException(cause);
+                }
+            };
+        entryStore.openRandomAccessReader(l, fence)
+                .addEventListener(FutureEventListenerRunnable.of(openReaderListener, executorService));
+        return promise;
+    }
+
+    //
+    // Search Functions
+    //
+
+    /**
+     * Get the log record whose transaction id is not less than provided <code>transactionId</code>.
+     *
+     * <p>
+     * It uses a binary-search like algorithm to find the log record whose transaction id is not less than
+     * provided <code>transactionId</code> within a log <code>segment</code>. You could think of a log segment
+     * in terms of a sequence of records whose transaction ids are non-decreasing.
+     *
+     * - The sequence of records within a log segment is divided into N pieces.
+     * - Find the piece of records that contains a record whose transaction id is not less than provided
+     *   <code>transactionId</code>.
+     *
+     * N could be chosen based on trading off concurrency and latency.
+     * </p>
+     *
+     * @param logName
+     *          name of the log
+     * @param segment
+     *          metadata of the log segment
+     * @param transactionId
+     *          transaction id
+     * @param executorService
+     *          executor service used for processing entries
+     * @param entryStore
+     *          log segment entry store
+     * @param nWays
+     *          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(
+            final String logName,
+            final LogSegmentMetadata segment,
+            final long transactionId,
+            final ExecutorService executorService,
+            final LogSegmentEntryStore entryStore,
+            final int nWays) {
+        if (!segment.isInProgress()) {
+            if (segment.getLastTxId() < transactionId) {
+                // all log records whose transaction id is less than provided transactionId
+                // then return none
+                Optional<LogRecordWithDLSN> noneRecord = Optional.absent();
+                return Future.value(noneRecord);
+            }
+        }
+
+        final Promise<Optional<LogRecordWithDLSN>> promise =
+                new Promise<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;
+                        }
+
+                    });
+                    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);
+                        return;
+                    }
+                    // all log records whose transaction id is not less than provided transactionId
+                    if (segment.getFirstTxId() >= transactionId) {
+                        final FirstTxIdNotLessThanSelector selector =
+                                new FirstTxIdNotLessThanSelector(transactionId);
+                        asyncReadRecordFromEntries(
+                                logName,
+                                reader,
+                                segment,
+                                executorService,
+                                new SingleEntryScanContext(0L),
+                                selector
+                        ).addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+                            @Override
+                            public void onSuccess(LogRecordWithDLSN value) {
+                                promise.setValue(Optional.of(selector.result()));
+                            }
+
+                            @Override
+                            public void onFailure(Throwable cause) {
+                                promise.setException(cause);
+                            }
+                        });
+
+                        return;
+                    }
+                    getLogRecordNotLessThanTxIdFromEntries(
+                            logName,
+                            segment,
+                            transactionId,
+                            executorService,
+                            reader,
+                            Lists.newArrayList(0L, lastEntryId),
+                            nWays,
+                            Optional.<LogRecordWithDLSN>absent(),
+                            promise);
+                }
+
+                @Override
+                public void onFailure(final Throwable cause) {
+                    promise.setException(cause);
+                }
+            };
+
+        entryStore.openRandomAccessReader(segment, false)
+                .addEventListener(FutureEventListenerRunnable.of(openReaderListener, executorService));
+        return promise;
+    }
+
+    /**
+     * Find the log record whose transaction id is not less than provided <code>transactionId</code> from
+     * entries between <code>startEntryId</code> and <code>endEntryId</code>.
+     *
+     * @param logName
+     *          name of the log
+     * @param segment
+     *          log segment
+     * @param transactionId
+     *          provided transaction id to search
+     * @param executorService
+     *          executor service
+     * @param reader
+     *          log segment random access reader
+     * @param entriesToSearch
+     *          list of entries to search
+     * @param nWays
+     *          how many entries to search in parallel
+     * @param prevFoundRecord
+     *          the log record found in previous search
+     * @param promise
+     *          promise to satisfy the result
+     */
+    private static void getLogRecordNotLessThanTxIdFromEntries(
+            final String logName,
+            final LogSegmentMetadata segment,
+            final long transactionId,
+            final ExecutorService executorService,
+            final LogSegmentRandomAccessEntryReader reader,
+            final List<Long> entriesToSearch,
+            final int nWays,
+            final Optional<LogRecordWithDLSN> prevFoundRecord,
+            final Promise<Optional<LogRecordWithDLSN>> promise) {
+        final List<Future<LogRecordWithDLSN>> searchResults =
+                Lists.newArrayListWithExpectedSize(entriesToSearch.size());
+        for (Long entryId : entriesToSearch) {
+            LogRecordSelector selector = new FirstTxIdNotLessThanSelector(transactionId);
+            Future<LogRecordWithDLSN> searchResult = asyncReadRecordFromEntries(
+                    logName,
+                    reader,
+                    segment,
+                    executorService,
+                    new SingleEntryScanContext(entryId),
+                    selector);
+            searchResults.add(searchResult);
+        }
+        FutureEventListener<List<LogRecordWithDLSN>> processSearchResultsListener =
+                new FutureEventListener<List<LogRecordWithDLSN>>() {
+                    @Override
+                    public void onSuccess(List<LogRecordWithDLSN> resultList) {
+                        processSearchResults(
+                                logName,
+                                segment,
+                                transactionId,
+                                executorService,
+                                reader,
+                                resultList,
+                                nWays,
+                                prevFoundRecord,
+                                promise);
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        promise.setException(cause);
+                    }
+                };
+        Future.collect(searchResults).addEventListener(
+                FutureEventListenerRunnable.of(processSearchResultsListener, executorService));
+    }
+
+    /**
+     * Process the search results
+     */
+    static void processSearchResults(
+            final String logName,
+            final LogSegmentMetadata segment,
+            final long transactionId,
+            final ExecutorService executorService,
+            final LogSegmentRandomAccessEntryReader reader,
+            final List<LogRecordWithDLSN> searchResults,
+            final int nWays,
+            final Optional<LogRecordWithDLSN> prevFoundRecord,
+            final Promise<Optional<LogRecordWithDLSN>> promise) {
+        int found = -1;
+        for (int i = 0; i < searchResults.size(); i++) {
+            LogRecordWithDLSN record = searchResults.get(i);
+            if (record.getTransactionId() >= transactionId) {
+                found = i;
+                break;
+            }
+        }
+        if (found == -1) { // all log records' transaction id is less than provided transaction id
+            promise.setValue(prevFoundRecord);
+            return;
+        }
+        // we found a log record
+        LogRecordWithDLSN foundRecord = searchResults.get(found);
+
+        // we found it
+        //   - it is not the first record
+        //   - it is the first record in first search entry
+        //   - its entry is adjacent to previous search entry
+        if (foundRecord.getDlsn().getSlotId() != 0L
+                || found == 0
+                || foundRecord.getDlsn().getEntryId() == (searchResults.get(found - 1).getDlsn().getEntryId() + 1)) {
+            promise.setValue(Optional.of(foundRecord));
+            return;
+        }
+
+        // otherwise, we need to search
+        List<Long> nextSearchBatch = getEntriesToSearch(
+                transactionId,
+                searchResults.get(found - 1),
+                searchResults.get(found),
+                nWays);
+        if (nextSearchBatch.isEmpty()) {
+            promise.setValue(prevFoundRecord);
+            return;
+        }
+        getLogRecordNotLessThanTxIdFromEntries(
+                logName,
+                segment,
+                transactionId,
+                executorService,
+                reader,
+                nextSearchBatch,
+                nWays,
+                Optional.of(foundRecord),
+                promise);
+    }
+
+    /**
+     * Get the entries to search provided <code>transactionId</code> between
+     * <code>firstRecord</code> and <code>lastRecord</code>. <code>firstRecord</code>
+     * and <code>lastRecord</code> are already searched, which the transaction id
+     * of <code>firstRecord</code> is less than <code>transactionId</code> and the
+     * transaction id of <code>lastRecord</code> is not less than <code>transactionId</code>.
+     *
+     * @param transactionId
+     *          transaction id to search
+     * @param firstRecord
+     *          log record that already searched whose transaction id is leass than <code>transactionId</code>.
+     * @param lastRecord
+     *          log record that already searched whose transaction id is not less than <code>transactionId</code>.
+     * @param nWays
+     *          N-ways to search
+     * @return the list of entries to search
+     */
+    static List<Long> getEntriesToSearch(
+            long transactionId,
+            LogRecordWithDLSN firstRecord,
+            LogRecordWithDLSN lastRecord,
+            int nWays) {
+        long txnDiff = lastRecord.getTransactionId() - firstRecord.getTransactionId();
+        if (txnDiff > 0) {
+            if (lastRecord.getTransactionId() == transactionId) {
+                List<Long> entries = getEntriesToSearch(
+                        firstRecord.getDlsn().getEntryId() + 1,
+                        lastRecord.getDlsn().getEntryId() - 2,
+                        Math.max(MIN_SEARCH_BATCH_SIZE, nWays - 1));
+                entries.add(lastRecord.getDlsn().getEntryId() - 1);
+                return entries;
+            } else {
+                // TODO: improve it by estimating transaction ids.
+                return getEntriesToSearch(
+                        firstRecord.getDlsn().getEntryId() + 1,
+                        lastRecord.getDlsn().getEntryId() - 1,
+                        nWays);
+            }
+        } else {
+            // unexpected condition
+            return Lists.newArrayList();
+        }
+    }
+
+    static List<Long> getEntriesToSearch(
+            long startEntryId,
+            long endEntryId,
+            int nWays) {
+        if (startEntryId > endEntryId) {
+            return Lists.newArrayList();
+        }
+        long numEntries = endEntryId - startEntryId + 1;
+        long step = Math.max(1L, numEntries / nWays);
+        List<Long> entryList = Lists.newArrayListWithExpectedSize(nWays);
+        for (long i = startEntryId, j = nWays - 1; i <= endEntryId && j > 0; i += step, j--) {
+            entryList.add(i);
+        }
+        if (entryList.get(entryList.size() - 1) < endEntryId) {
+            entryList.add(endEntryId);
+        }
+        return entryList;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java
new file mode 100644
index 0000000..d25d056
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java
@@ -0,0 +1,62 @@
+/**
+ * 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 org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.util.PermitLimiter;
+
+public class WriteLimiter {
+
+    String streamName;
+    final PermitLimiter streamLimiter;
+    final PermitLimiter globalLimiter;
+
+    public WriteLimiter(String streamName, PermitLimiter streamLimiter, PermitLimiter globalLimiter) {
+        this.streamName = streamName;
+        this.streamLimiter = streamLimiter;
+        this.globalLimiter = globalLimiter;
+    }
+
+    public void acquire() throws OverCapacityException {
+        if (!streamLimiter.acquire()) {
+            throw new OverCapacityException(String.format("Stream write capacity exceeded for stream %s", streamName));
+        }
+        try {
+            if (!globalLimiter.acquire()) {
+                throw new OverCapacityException("Global write capacity exceeded");
+            }
+        } catch (OverCapacityException ex) {
+            streamLimiter.release(1);
+            throw ex;
+        }
+    }
+
+    public void release() {
+        release(1);
+    }
+
+    public void release(int permits) {
+        streamLimiter.release(permits);
+        globalLimiter.release(permits);
+    }
+
+    public void close() {
+        streamLimiter.close();
+        globalLimiter.close();
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java
new file mode 100644
index 0000000..e56a22d
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java
@@ -0,0 +1,402 @@
+/**
+ * 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.base.Stopwatch;
+import org.apache.distributedlog.util.FailpointUtils;
+import org.apache.distributedlog.zk.ZKWatcherManager;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.ACL;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * ZooKeeper Client wrapper over {@link org.apache.bookkeeper.zookeeper.ZooKeeperClient}.
+ * It handles retries on session expires and provides a watcher manager {@link ZKWatcherManager}.
+ *
+ * <h3>Metrics</h3>
+ * <ul>
+ * <li> zookeeper operation stats are exposed under scope <code>zk</code> by
+ * {@link org.apache.bookkeeper.zookeeper.ZooKeeperClient}
+ * <li> stats on zookeeper watched events are exposed under scope <code>watcher</code> by
+ * {@link org.apache.bookkeeper.zookeeper.ZooKeeperWatcherBase}
+ * <li> stats about {@link ZKWatcherManager} are exposed under scope <code>watcher_manager</code>
+ * </ul>
+ */
+public class ZooKeeperClient {
+
+    public static interface Credentials {
+
+        Credentials NONE = new Credentials() {
+            @Override
+            public void authenticate(ZooKeeper zooKeeper) {
+                // noop
+            }
+        };
+
+        void authenticate(ZooKeeper zooKeeper);
+    }
+
+    public static class DigestCredentials implements Credentials {
+
+        String username;
+        String password;
+
+        public DigestCredentials(String username, String password) {
+            this.username = username;
+            this.password = password;
+        }
+
+        @Override
+        public void authenticate(ZooKeeper zooKeeper) {
+            zooKeeper.addAuthInfo("digest", String.format("%s:%s", username, password).getBytes(UTF_8));
+        }
+    }
+
+    public interface ZooKeeperSessionExpireNotifier {
+        void notifySessionExpired();
+    }
+
+    /**
+     * Indicates an error connecting to a zookeeper cluster.
+     */
+    public static class ZooKeeperConnectionException extends IOException {
+        private static final long serialVersionUID = 6682391687004819361L;
+
+        public ZooKeeperConnectionException(String message) {
+            super(message);
+        }
+
+        public ZooKeeperConnectionException(String message, Throwable cause) {
+            super(message, cause);
+        }
+    }
+
+    private static final Logger LOG = LoggerFactory.getLogger(ZooKeeperClient.class.getName());
+
+    private final String name;
+    private final int sessionTimeoutMs;
+    private final int defaultConnectionTimeoutMs;
+    private final String zooKeeperServers;
+    // GuardedBy "this", but still volatile for tests, where we want to be able to see writes
+    // made from within long synchronized blocks.
+    private volatile ZooKeeper zooKeeper = null;
+    private final RetryPolicy retryPolicy;
+    private final StatsLogger statsLogger;
+    private final int retryThreadCount;
+    private final double requestRateLimit;
+    private final Credentials credentials;
+    private volatile boolean authenticated = false;
+    private Stopwatch disconnectedStopwatch = null;
+
+    private boolean closed = false;
+
+    final Set<Watcher> watchers = new CopyOnWriteArraySet<Watcher>();
+
+    // watcher manager to manage watchers
+    private final ZKWatcherManager watcherManager;
+
+    /**
+     * Creates an unconnected client that will lazily attempt to connect on the first call to
+     * {@link #get}.  All successful connections will be authenticated with the given
+     * {@code credentials}.
+     *
+     * @param sessionTimeoutMs
+     *          ZK session timeout in milliseconds
+     * @param connectionTimeoutMs
+     *          ZK connection timeout in milliseconds
+     * @param zooKeeperServers
+     *          the set of servers forming the ZK cluster
+     */
+    ZooKeeperClient(int sessionTimeoutMs, int connectionTimeoutMs, String zooKeeperServers) {
+        this("default", sessionTimeoutMs, connectionTimeoutMs, zooKeeperServers, null, NullStatsLogger.INSTANCE, 1, 0,
+             Credentials.NONE);
+    }
+
+    ZooKeeperClient(String name,
+                    int sessionTimeoutMs,
+                    int connectionTimeoutMs,
+                    String zooKeeperServers,
+                    RetryPolicy retryPolicy,
+                    StatsLogger statsLogger,
+                    int retryThreadCount,
+                    double requestRateLimit,
+                    Credentials credentials) {
+        this.name = name;
+        this.sessionTimeoutMs = sessionTimeoutMs;
+        this.zooKeeperServers = zooKeeperServers;
+        this.defaultConnectionTimeoutMs = connectionTimeoutMs;
+        this.retryPolicy = retryPolicy;
+        this.statsLogger = statsLogger;
+        this.retryThreadCount = retryThreadCount;
+        this.requestRateLimit = requestRateLimit;
+        this.credentials = credentials;
+        this.watcherManager = ZKWatcherManager.newBuilder()
+                .name(name)
+                .zkc(this)
+                .statsLogger(statsLogger.scope("watcher_manager"))
+                .build();
+    }
+
+    public List<ACL> getDefaultACL() {
+        if (Credentials.NONE == credentials) {
+            return ZooDefs.Ids.OPEN_ACL_UNSAFE;
+        } else {
+            return DistributedLogConstants.EVERYONE_READ_CREATOR_ALL;
+        }
+    }
+
+    public ZKWatcherManager getWatcherManager() {
+        return watcherManager;
+    }
+
+    /**
+     * Returns the current active ZK connection or establishes a new one if none has yet been
+     * established or a previous connection was disconnected or had its session time out.
+     *
+     * @return a connected ZooKeeper client
+     * @throws ZooKeeperConnectionException if there was a problem connecting to the ZK cluster
+     * @throws InterruptedException if interrupted while waiting for a connection to be established
+     * @throws TimeoutException if a connection could not be established within the configured
+     * session timeout
+     */
+    public synchronized ZooKeeper get()
+        throws ZooKeeperConnectionException, InterruptedException {
+
+        try {
+            FailpointUtils.checkFailPoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss);
+        } catch (IOException ioe) {
+            throw new ZooKeeperConnectionException("Client " + name + " failed on establishing zookeeper connection", ioe);
+        }
+
+        // This indicates that the client was explictly closed
+        if (closed) {
+            throw new ZooKeeperConnectionException("Client " + name + " has already been closed");
+        }
+
+        // the underneath zookeeper is retryable zookeeper
+        if (zooKeeper != null && retryPolicy != null) {
+            if (zooKeeper.getState().equals(ZooKeeper.States.CONNECTED)) {
+                // the zookeeper client is connected
+                disconnectedStopwatch = null;
+            } else {
+                if (disconnectedStopwatch == null) {
+                    disconnectedStopwatch = Stopwatch.createStarted();
+                } else {
+                    long disconnectedMs = disconnectedStopwatch.elapsed(TimeUnit.MILLISECONDS);
+                    if (disconnectedMs > defaultConnectionTimeoutMs) {
+                        closeInternal();
+                        authenticated = false;
+                    }
+                }
+            }
+        }
+
+        if (zooKeeper == null) {
+            zooKeeper = buildZooKeeper();
+            disconnectedStopwatch = null;
+        }
+
+        // In case authenticate throws an exception, the caller can try to recover the client by
+        // calling get again.
+        if (!authenticated) {
+            credentials.authenticate(zooKeeper);
+            authenticated = true;
+        }
+
+        return zooKeeper;
+    }
+
+    private ZooKeeper buildZooKeeper()
+        throws ZooKeeperConnectionException, InterruptedException {
+        Watcher watcher = new Watcher() {
+            @Override
+            public void process(WatchedEvent event) {
+                switch (event.getType()) {
+                    case None:
+                        switch (event.getState()) {
+                            case Expired:
+                                if (null == retryPolicy) {
+                                    LOG.info("ZooKeeper {}' session expired. Event: {}", name, event);
+                                    closeInternal();
+                                }
+                                authenticated = false;
+                                break;
+                            case Disconnected:
+                                if (null == retryPolicy) {
+                                    LOG.info("ZooKeeper {} is disconnected from zookeeper now," +
+                                            " but it is OK unless we received EXPIRED event.", name);
+                                }
+                                // Mark as not authenticated if expired or disconnected. In both cases
+                                // we lose any attached auth info. Relying on Expired/Disconnected is
+                                // sufficient since all Expired/Disconnected events are processed before
+                                // all SyncConnected events, and the underlying member is not updated until
+                                // SyncConnected is received.
+                                authenticated = false;
+                                break;
+                            default:
+                                break;
+                        }
+                }
+
+                try {
+                    for (Watcher watcher : watchers) {
+                        try {
+                            watcher.process(event);
+                        } catch (Throwable t) {
+                            LOG.warn("Encountered unexpected exception from watcher {} : ", watcher, t);
+                        }
+                    }
+                } catch (Throwable t) {
+                    LOG.warn("Encountered unexpected exception when firing watched event {} : ", event, t);
+                }
+            }
+        };
+
+        Set<Watcher> watchers = new HashSet<Watcher>();
+        watchers.add(watcher);
+
+        ZooKeeper zk;
+        try {
+            RetryPolicy opRetryPolicy = null == retryPolicy ?
+                    new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0) : retryPolicy;
+            RetryPolicy connectRetryPolicy = null == retryPolicy ?
+                    new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, 0) :
+                    new BoundExponentialBackoffRetryPolicy(sessionTimeoutMs, sessionTimeoutMs, Integer.MAX_VALUE);
+            zk = org.apache.bookkeeper.zookeeper.ZooKeeperClient.newBuilder()
+                    .connectString(zooKeeperServers)
+                    .sessionTimeoutMs(sessionTimeoutMs)
+                    .watchers(watchers)
+                    .operationRetryPolicy(opRetryPolicy)
+                    .connectRetryPolicy(connectRetryPolicy)
+                    .statsLogger(statsLogger)
+                    .retryThreadCount(retryThreadCount)
+                    .requestRateLimit(requestRateLimit)
+                    .build();
+        } catch (KeeperException e) {
+            throw new ZooKeeperConnectionException("Problem connecting to servers: " + zooKeeperServers, e);
+        } catch (IOException e) {
+            throw new ZooKeeperConnectionException("Problem connecting to servers: " + zooKeeperServers, e);
+        }
+        return zk;
+    }
+
+    /**
+     * Clients that need to re-establish state after session expiration can register an
+     * {@code onExpired} command to execute.
+     *
+     * @param onExpired the {@code Command} to register
+     * @return the new {@link Watcher} which can later be passed to {@link #unregister} for
+     *         removal.
+     */
+    public Watcher registerExpirationHandler(final ZooKeeperSessionExpireNotifier onExpired) {
+        Watcher watcher = new Watcher() {
+            @Override
+            public void process(WatchedEvent event) {
+                if (event.getType() == EventType.None && event.getState() == KeeperState.Expired) {
+                    try {
+                        onExpired.notifySessionExpired();
+                    } catch (Exception exc) {
+                        // do nothing
+                    }
+                }
+            }
+        };
+        register(watcher);
+        return watcher;
+    }
+
+    /**
+     * Clients that need to register a top-level {@code Watcher} should do so using this method.  The
+     * registered {@code watcher} will remain registered across re-connects and session expiration
+     * events.
+     *
+     * @param watcher the {@code Watcher to register}
+     */
+    public void register(Watcher watcher) {
+        if (null != watcher) {
+            watchers.add(watcher);
+        }
+    }
+
+    /**
+     * Clients can attempt to unregister a top-level {@code Watcher} that has previously been
+     * registered.
+     *
+     * @param watcher the {@code Watcher} to unregister as a top-level, persistent watch
+     * @return whether the given {@code Watcher} was found and removed from the active set
+     */
+    public boolean unregister(Watcher watcher) {
+        return null != watcher && watchers.remove(watcher);
+    }
+
+    /**
+     * Closes the current connection if any expiring the current ZooKeeper session.  Any subsequent
+     * calls to this method will no-op until the next successful {@link #get}.
+     */
+    public synchronized void closeInternal() {
+        if (zooKeeper != null) {
+            try {
+                LOG.info("Closing zookeeper client {}.", name);
+                zooKeeper.close();
+                LOG.info("Closed zookeeper client {}.", name);
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                LOG.warn("Interrupted trying to close zooKeeper {} : ", name, e);
+            } finally {
+                zooKeeper = null;
+            }
+        }
+    }
+
+    /**
+     * Closes the the underlying zookeeper instance.
+     * Subsequent attempts to {@link #get} will fail
+     */
+    public synchronized void close() {
+        if (closed) {
+            return;
+        }
+        LOG.info("Close zookeeper client {}.", name);
+        closeInternal();
+        // unregister gauges to prevent GC spiral
+        this.watcherManager.unregisterGauges();
+        closed = true;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java b/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java
new file mode 100644
index 0000000..0c200ce
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java
@@ -0,0 +1,233 @@
+/**
+ * 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.base.Preconditions;
+import org.apache.distributedlog.ZooKeeperClient.Credentials;
+import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+
+/**
+ * Builder to build zookeeper client.
+ */
+public class ZooKeeperClientBuilder {
+
+    static final Logger LOG = LoggerFactory.getLogger(ZooKeeperClientBuilder.class);
+
+    /**
+     * Create a zookeeper client builder to build zookeeper clients.
+     *
+     * @return zookeeper client builder.
+     */
+    public static ZooKeeperClientBuilder newBuilder() {
+        return new ZooKeeperClientBuilder();
+    }
+
+    // name
+    private String name = "default";
+    // sessionTimeoutMs
+    private int sessionTimeoutMs = -1;
+    // conectionTimeoutMs
+    private int conectionTimeoutMs = -1;
+    // zkServers
+    private String zkServers = null;
+    // retry policy
+    private RetryPolicy retryPolicy = null;
+    // stats logger
+    private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
+    // retry executor thread count
+    private int retryThreadCount = 1;
+    // zookeeper access requestRateLimit limit
+    private double requestRateLimit = 0;
+    // Did call the zkAclId setter on the builder, used to ensure the setter is set.
+    private boolean zkAclIdSet = false;
+    private String zkAclId;
+
+    // Cached ZooKeeper Client
+    private ZooKeeperClient cachedClient = null;
+
+    private ZooKeeperClientBuilder() {}
+
+    /**
+     * Set zookeeper client name
+     *
+     * @param name zookeeper client name
+     * @return zookeeper client builder
+     */
+    public synchronized ZooKeeperClientBuilder name(String name) {
+        this.name = name;
+        return this;
+    }
+
+    /**
+     * Set zookeeper session timeout in milliseconds.
+     *
+     * @param sessionTimeoutMs
+     *          session timeout in milliseconds.
+     * @return zookeeper client builder.
+     */
+    public synchronized ZooKeeperClientBuilder sessionTimeoutMs(int sessionTimeoutMs) {
+        this.sessionTimeoutMs = sessionTimeoutMs;
+        if (this.conectionTimeoutMs <= 0) {
+            this.conectionTimeoutMs = 2 * sessionTimeoutMs;
+        }
+        return this;
+    }
+
+    public synchronized ZooKeeperClientBuilder retryThreadCount(int retryThreadCount) {
+        this.retryThreadCount = retryThreadCount;
+        return this;
+    }
+
+    public synchronized ZooKeeperClientBuilder requestRateLimit(double requestRateLimit) {
+        this.requestRateLimit = requestRateLimit;
+        return this;
+    }
+
+    /**
+     * Set zookeeper connection timeout in milliseconds
+     *
+     * @param connectionTimeoutMs
+     *          connection timeout ms.
+     * @return builder
+     */
+    public synchronized ZooKeeperClientBuilder connectionTimeoutMs(int connectionTimeoutMs) {
+        this.conectionTimeoutMs = connectionTimeoutMs;
+        return this;
+    }
+
+    /**
+     * Set ZooKeeper Connect String.
+     *
+     * @param zkServers
+     *          zookeeper servers to connect.
+     * @return builder
+     */
+    public synchronized ZooKeeperClientBuilder zkServers(String zkServers) {
+        this.zkServers = zkServers;
+        return this;
+    }
+
+    /**
+     * Set DistributedLog URI.
+     *
+     * @param uri
+     *          distributedlog uri.
+     * @return builder.
+     */
+    public synchronized ZooKeeperClientBuilder uri(URI uri) {
+        this.zkServers = BKNamespaceDriver.getZKServersFromDLUri(uri);
+        return this;
+    }
+
+    /**
+     * Build zookeeper client using existing <i>zkc</i> client.
+     *
+     * @param zkc
+     *          zookeeper client.
+     * @return builder
+     */
+    public synchronized ZooKeeperClientBuilder zkc(ZooKeeperClient zkc) {
+        this.cachedClient = zkc;
+        return this;
+    }
+
+    /**
+     * Build zookeeper client with given retry policy <i>retryPolicy</i>.
+     *
+     * @param retryPolicy
+     *          retry policy
+     * @return builder
+     */
+    public synchronized ZooKeeperClientBuilder retryPolicy(RetryPolicy retryPolicy) {
+        this.retryPolicy = retryPolicy;
+        return this;
+    }
+
+    /**
+     * Build zookeeper client with given stats logger <i>statsLogger</i>.
+     *
+     * @param statsLogger
+     *          stats logger to expose zookeeper stats
+     * @return builder
+     */
+    public synchronized ZooKeeperClientBuilder statsLogger(StatsLogger statsLogger) {
+        this.statsLogger = statsLogger;
+        return this;
+    }
+
+    /**
+     * * Build zookeeper client with given zk acl digest id <i>zkAclId</i>.
+     */
+    public synchronized ZooKeeperClientBuilder zkAclId(String zkAclId) {
+        this.zkAclIdSet = true;
+        this.zkAclId = zkAclId;
+        return this;
+    }
+
+    private void validateParameters() {
+        Preconditions.checkNotNull(zkServers, "No zk servers provided.");
+        Preconditions.checkArgument(conectionTimeoutMs > 0,
+                "Invalid connection timeout : %d", conectionTimeoutMs);
+        Preconditions.checkArgument(sessionTimeoutMs > 0,
+                "Invalid session timeout : %d", sessionTimeoutMs);
+        Preconditions.checkNotNull(statsLogger, "No stats logger provided.");
+        Preconditions.checkArgument(zkAclIdSet, "Zookeeper acl id not set.");
+    }
+
+    /**
+     * Build a zookeeper client.
+     *
+     * @return zookeeper client.
+     */
+    public synchronized ZooKeeperClient build() {
+        if (null == cachedClient) {
+            cachedClient = buildClient();
+        }
+        return cachedClient;
+    }
+
+    private ZooKeeperClient buildClient() {
+        validateParameters();
+
+        Credentials credentials = Credentials.NONE;
+        if (null != zkAclId) {
+            credentials = new DigestCredentials(zkAclId, zkAclId);
+        }
+
+        return new ZooKeeperClient(
+                name,
+                sessionTimeoutMs,
+                conectionTimeoutMs,
+                zkServers,
+                retryPolicy,
+                statsLogger,
+                retryThreadCount,
+                requestRateLimit,
+                credentials
+        );
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/acl/AccessControlManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/AccessControlManager.java
new file mode 100644
index 0000000..2c3e738
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/AccessControlManager.java
@@ -0,0 +1,74 @@
+/**
+ * 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.acl;
+
+/**
+ * Access Control on stream operations
+ */
+public interface AccessControlManager {
+
+    /**
+     * Whether allowing writing to a stream.
+     *
+     * @param stream
+     *          Stream to write
+     * @return true if allowing writing to the given stream, otherwise false.
+     */
+    boolean allowWrite(String stream);
+
+    /**
+     * Whether allowing truncating a given stream.
+     *
+     * @param stream
+     *          Stream to truncate
+     * @return true if allowing truncating a given stream.
+     */
+    boolean allowTruncate(String stream);
+
+    /**
+     * Whether allowing deleting a given stream.
+     *
+     * @param stream
+     *          Stream to delete
+     * @return true if allowing deleting a given stream.
+     */
+    boolean allowDelete(String stream);
+
+    /**
+     * Whether allowing proxies to acquire a given stream.
+     *
+     * @param stream
+     *          stream to acquire
+     * @return true if allowing proxies to acquire the given stream.
+     */
+    boolean allowAcquire(String stream);
+
+    /**
+     * Whether allowing proxies to release ownership for a given stream.
+     *
+     * @param stream
+     *          stream to release
+     * @return true if allowing proxies to release a given stream.
+     */
+    boolean allowRelease(String stream);
+
+    /**
+     * Close the access control manager.
+     */
+    void close();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/acl/DefaultAccessControlManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/DefaultAccessControlManager.java
new file mode 100644
index 0000000..bf3352a
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/DefaultAccessControlManager.java
@@ -0,0 +1,55 @@
+/**
+ * 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.acl;
+
+public class DefaultAccessControlManager implements AccessControlManager {
+
+    public static final DefaultAccessControlManager INSTANCE = new DefaultAccessControlManager();
+
+    private DefaultAccessControlManager() {
+    }
+
+    @Override
+    public boolean allowWrite(String stream) {
+        return true;
+    }
+
+    @Override
+    public boolean allowTruncate(String stream) {
+        return true;
+    }
+
+    @Override
+    public boolean allowDelete(String stream) {
+        return true;
+    }
+
+    @Override
+    public boolean allowAcquire(String stream) {
+        return true;
+    }
+
+    @Override
+    public boolean allowRelease(String stream) {
+        return true;
+    }
+
+    @Override
+    public void close() {
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/acl/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/package-info.java
new file mode 100644
index 0000000..4218bfc
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/acl/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Access Control for distributedlog streams.
+ */
+package org.apache.distributedlog.acl;
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
new file mode 100644
index 0000000..4e94984
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
@@ -0,0 +1,921 @@
+/**
+ * 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.admin;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.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.impl.BKNamespaceDriver;
+import org.apache.distributedlog.impl.acl.ZKAccessControl;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.impl.federated.FederatedZKLogMetadataStore;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.metadata.DLMetadata;
+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.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.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Admin Tool for DistributedLog.
+ */
+public class DistributedLogAdmin extends DistributedLogTool {
+
+    static final Logger LOG = LoggerFactory.getLogger(DistributedLogAdmin.class);
+
+    /**
+     * Fix inprogress segment with lower ledger sequence number.
+     *
+     * @param namespace
+     *          dl namespace
+     * @param metadataUpdater
+     *          metadata updater.
+     * @param streamName
+     *          stream name.
+     * @param verbose
+     *          print verbose messages.
+     * @param interactive
+     *          is confirmation needed before executing actual action.
+     * @throws IOException
+     */
+    public static void fixInprogressSegmentWithLowerSequenceNumber(final DistributedLogNamespace namespace,
+                                                                   final MetadataUpdater metadataUpdater,
+                                                                   final String streamName,
+                                                                   final boolean verbose,
+                                                                   final boolean interactive) throws IOException {
+        DistributedLogManager dlm = namespace.openLog(streamName);
+        try {
+            List<LogSegmentMetadata> segments = dlm.getLogSegments();
+            if (verbose) {
+                System.out.println("LogSegments for " + streamName + " : ");
+                for (LogSegmentMetadata segment : segments) {
+                    System.out.println(segment.getLogSegmentSequenceNumber() + "\t: " + segment);
+                }
+            }
+            LOG.info("Get log segments for {} : {}", streamName, segments);
+            // validate log segments
+            long maxCompletedLogSegmentSequenceNumber = -1L;
+            LogSegmentMetadata inprogressSegment = null;
+            for (LogSegmentMetadata segment : segments) {
+                if (!segment.isInProgress()) {
+                    maxCompletedLogSegmentSequenceNumber = Math.max(maxCompletedLogSegmentSequenceNumber, segment.getLogSegmentSequenceNumber());
+                } else {
+                    // we already found an inprogress segment
+                    if (null != inprogressSegment) {
+                        throw new DLIllegalStateException("Multiple inprogress segments found for stream " + streamName + " : " + segments);
+                    }
+                    inprogressSegment = segment;
+                }
+            }
+            if (null == inprogressSegment || inprogressSegment.getLogSegmentSequenceNumber() > maxCompletedLogSegmentSequenceNumber) {
+                // nothing to fix
+                return;
+            }
+            final long newLogSegmentSequenceNumber = maxCompletedLogSegmentSequenceNumber + 1;
+            if (interactive && !IOUtils.confirmPrompt("Confirm to fix (Y/N), Ctrl+C to break : ")) {
+                return;
+            }
+            final LogSegmentMetadata newSegment =
+                    FutureUtils.result(metadataUpdater.changeSequenceNumber(inprogressSegment, newLogSegmentSequenceNumber));
+            LOG.info("Fixed {} : {} -> {} ",
+                     new Object[] { streamName, inprogressSegment, newSegment });
+            if (verbose) {
+                System.out.println("Fixed " + streamName + " : " + inprogressSegment.getZNodeName()
+                                   + " -> " + newSegment.getZNodeName());
+                System.out.println("\t old: " + inprogressSegment);
+                System.out.println("\t new: " + newSegment);
+                System.out.println();
+            }
+        } finally {
+            dlm.close();
+        }
+    }
+
+    private static class LogSegmentCandidate {
+        final LogSegmentMetadata metadata;
+        final LogRecordWithDLSN lastRecord;
+
+        LogSegmentCandidate(LogSegmentMetadata metadata, LogRecordWithDLSN lastRecord) {
+            this.metadata = metadata;
+            this.lastRecord = lastRecord;
+        }
+
+        @Override
+        public String toString() {
+            return "LogSegmentCandidate[ metadata = " + metadata + ", last record = " + lastRecord + " ]";
+        }
+
+    }
+
+    private static final Comparator<LogSegmentCandidate> LOG_SEGMENT_CANDIDATE_COMPARATOR =
+            new Comparator<LogSegmentCandidate>() {
+                @Override
+                public int compare(LogSegmentCandidate o1, LogSegmentCandidate o2) {
+                    return LogSegmentMetadata.COMPARATOR.compare(o1.metadata, o2.metadata);
+                }
+            };
+
+    private static class StreamCandidate {
+
+        final String streamName;
+        final SortedSet<LogSegmentCandidate> segmentCandidates =
+                new TreeSet<LogSegmentCandidate>(LOG_SEGMENT_CANDIDATE_COMPARATOR);
+
+        StreamCandidate(String streamName) {
+            this.streamName = streamName;
+        }
+
+        synchronized void addLogSegmentCandidate(LogSegmentCandidate segmentCandidate) {
+            segmentCandidates.add(segmentCandidate);
+        }
+
+        @Override
+        public String toString() {
+            return "StreamCandidate[ name = " + streamName + ", segments = " + segmentCandidates + " ]";
+        }
+    }
+
+    public static void checkAndRepairDLNamespace(final URI uri,
+                                                 final DistributedLogNamespace namespace,
+                                                 final MetadataUpdater metadataUpdater,
+                                                 final OrderedScheduler scheduler,
+                                                 final boolean verbose,
+                                                 final boolean interactive) throws IOException {
+        checkAndRepairDLNamespace(uri, namespace, metadataUpdater, scheduler, verbose, interactive, 1);
+    }
+
+    public static void checkAndRepairDLNamespace(final URI uri,
+                                                 final DistributedLogNamespace namespace,
+                                                 final MetadataUpdater metadataUpdater,
+                                                 final OrderedScheduler scheduler,
+                                                 final boolean verbose,
+                                                 final boolean interactive,
+                                                 final int concurrency) throws IOException {
+        Preconditions.checkArgument(concurrency > 0, "Invalid concurrency " + concurrency + " found.");
+        // 0. getting streams under a given uri.
+        Iterator<String> streamsIter = namespace.getLogs();
+        List<String> streams = Lists.newArrayList();
+        while (streamsIter.hasNext()) {
+            streams.add(streamsIter.next());
+        }
+        if (verbose) {
+            System.out.println("- 0. checking streams under " + uri);
+        }
+        if (streams.size() == 0) {
+            System.out.println("+ 0. nothing to check. quit.");
+            return;
+        }
+        Map<String, StreamCandidate> streamCandidates =
+                checkStreams(namespace, streams, scheduler, concurrency);
+        if (verbose) {
+            System.out.println("+ 0. " + streamCandidates.size() + " corrupted streams found.");
+        }
+        if (interactive && !IOUtils.confirmPrompt("Do you want to fix all " + streamCandidates.size() + " corrupted streams (Y/N) : ")) {
+            return;
+        }
+        if (verbose) {
+            System.out.println("- 1. repairing " + streamCandidates.size() + " corrupted streams.");
+        }
+        for (StreamCandidate candidate : streamCandidates.values()) {
+            if (!repairStream(metadataUpdater, candidate, verbose, interactive)) {
+                if (verbose) {
+                    System.out.println("* 1. aborted repairing corrupted streams.");
+                }
+                return;
+            }
+        }
+        if (verbose) {
+            System.out.println("+ 1. repaired " + streamCandidates.size() + " corrupted streams.");
+        }
+    }
+
+    private static Map<String, StreamCandidate> checkStreams(
+            final DistributedLogNamespace namespace,
+            final Collection<String> streams,
+            final OrderedScheduler scheduler,
+            final int concurrency) throws IOException {
+        final LinkedBlockingQueue<String> streamQueue =
+                new LinkedBlockingQueue<String>();
+        streamQueue.addAll(streams);
+        final Map<String, StreamCandidate> candidateMap =
+                new ConcurrentSkipListMap<String, StreamCandidate>();
+        final AtomicInteger numPendingStreams = new AtomicInteger(streams.size());
+        final CountDownLatch doneLatch = new CountDownLatch(1);
+        Runnable checkRunnable = new Runnable() {
+            @Override
+            public void run() {
+                while (!streamQueue.isEmpty()) {
+                    String stream;
+                    try {
+                        stream = streamQueue.take();
+                    } catch (InterruptedException e) {
+                        Thread.currentThread().interrupt();
+                        break;
+                    }
+                    StreamCandidate candidate;
+                    try {
+                        LOG.info("Checking stream {}.", stream);
+                        candidate = checkStream(namespace, stream, scheduler);
+                        LOG.info("Checked stream {} - {}.", stream, candidate);
+                    } catch (IOException e) {
+                        LOG.error("Error on checking stream {} : ", stream, e);
+                        doneLatch.countDown();
+                        break;
+                    }
+                    if (null != candidate) {
+                        candidateMap.put(stream, candidate);
+                    }
+                    if (numPendingStreams.decrementAndGet() == 0) {
+                        doneLatch.countDown();
+                    }
+                }
+            }
+        };
+        Thread[] threads = new Thread[concurrency];
+        for (int i = 0; i < concurrency; i++) {
+            threads[i] = new Thread(checkRunnable, "check-thread-" + i);
+            threads[i].start();
+        }
+        try {
+            doneLatch.await();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+        }
+        if (numPendingStreams.get() != 0) {
+            throw new IOException(numPendingStreams.get() + " streams left w/o checked");
+        }
+        for (int i = 0; i < concurrency; i++) {
+            threads[i].interrupt();
+            try {
+                threads[i].join();
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+            }
+        }
+        return candidateMap;
+    }
+
+    private static StreamCandidate checkStream(
+            final DistributedLogNamespace namespace,
+            final String streamName,
+            final OrderedScheduler scheduler) throws IOException {
+        DistributedLogManager dlm = namespace.openLog(streamName);
+        try {
+            List<LogSegmentMetadata> segments = dlm.getLogSegments();
+            if (segments.isEmpty()) {
+                return null;
+            }
+            List<Future<LogSegmentCandidate>> futures =
+                    new ArrayList<Future<LogSegmentCandidate>>(segments.size());
+            for (LogSegmentMetadata segment : segments) {
+                futures.add(checkLogSegment(namespace, streamName, segment, scheduler));
+            }
+            List<LogSegmentCandidate> segmentCandidates;
+            try {
+                segmentCandidates = Await.result(Future.collect(futures));
+            } catch (Exception e) {
+                throw new IOException("Failed on checking stream " + streamName, e);
+            }
+            StreamCandidate streamCandidate = new StreamCandidate(streamName);
+            for (LogSegmentCandidate segmentCandidate: segmentCandidates) {
+                if (null != segmentCandidate) {
+                    streamCandidate.addLogSegmentCandidate(segmentCandidate);
+                }
+            }
+            if (streamCandidate.segmentCandidates.isEmpty()) {
+                return null;
+            }
+            return streamCandidate;
+        } finally {
+            dlm.close();
+        }
+    }
+
+    private static Future<LogSegmentCandidate> checkLogSegment(
+            final DistributedLogNamespace namespace,
+            final String streamName,
+            final LogSegmentMetadata metadata,
+            final OrderedScheduler scheduler) {
+        if (metadata.isInProgress()) {
+            return Future.value(null);
+        }
+
+        final LogSegmentEntryStore entryStore = namespace.getNamespaceDriver()
+                .getLogSegmentEntryStore(NamespaceDriver.Role.READER);
+        return ReadUtils.asyncReadLastRecord(
+                streamName,
+                metadata,
+                true,
+                false,
+                true,
+                4,
+                16,
+                new AtomicInteger(0),
+                scheduler,
+                entryStore
+        ).map(new Function<LogRecordWithDLSN, LogSegmentCandidate>() {
+            @Override
+            public LogSegmentCandidate apply(LogRecordWithDLSN record) {
+                if (null != record &&
+                    (record.getDlsn().compareTo(metadata.getLastDLSN()) > 0 ||
+                     record.getTransactionId() > metadata.getLastTxId() ||
+                     !metadata.isRecordPositionWithinSegmentScope(record))) {
+                    return new LogSegmentCandidate(metadata, record);
+                } else {
+                    return null;
+                }
+            }
+        });
+    }
+
+    private static boolean repairStream(MetadataUpdater metadataUpdater,
+                                        StreamCandidate streamCandidate,
+                                        boolean verbose,
+                                        boolean interactive) throws IOException {
+        if (verbose) {
+            System.out.println("Stream " + streamCandidate.streamName + " : ");
+            for (LogSegmentCandidate segmentCandidate : streamCandidate.segmentCandidates) {
+                System.out.println("  " + segmentCandidate.metadata.getLogSegmentSequenceNumber()
+                        + " : metadata = " + segmentCandidate.metadata + ", last dlsn = "
+                        + segmentCandidate.lastRecord.getDlsn());
+            }
+            System.out.println("-------------------------------------------");
+        }
+        if (interactive && !IOUtils.confirmPrompt("Do you want to fix the stream " + streamCandidate.streamName + " (Y/N) : ")) {
+            return false;
+        }
+        for (LogSegmentCandidate segmentCandidate : streamCandidate.segmentCandidates) {
+            LogSegmentMetadata newMetadata = FutureUtils.result(
+                    metadataUpdater.updateLastRecord(segmentCandidate.metadata, segmentCandidate.lastRecord));
+            if (verbose) {
+                System.out.println("  Fixed segment " + segmentCandidate.metadata.getLogSegmentSequenceNumber() + " : ");
+                System.out.println("    old metadata : " + segmentCandidate.metadata);
+                System.out.println("    new metadata : " + newMetadata);
+            }
+        }
+        if (verbose) {
+            System.out.println("-------------------------------------------");
+        }
+        return true;
+    }
+
+    //
+    // Commands
+    //
+
+    /**
+     * Unbind the bookkeeper environment for a given distributedlog uri.
+     *
+     * TODO: move unbind operation to namespace driver
+     */
+    class UnbindCommand extends OptsCommand {
+
+        Options options = new Options();
+
+        UnbindCommand() {
+            super("unbind", "unbind the bookkeeper environment bound for a given distributedlog instance.");
+            options.addOption("f", "force", false, "Force unbinding without prompt.");
+        }
+
+        @Override
+        protected Options getOptions() {
+            return options;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "unbind [options] <distributedlog uri>";
+        }
+
+        @Override
+        protected int runCmd(CommandLine cmdline) throws Exception {
+            String[] args = cmdline.getArgs();
+            if (args.length <= 0) {
+                System.err.println("No distributedlog uri specified.");
+                printUsage();
+                return -1;
+            }
+            boolean force = cmdline.hasOption("f");
+            URI uri = URI.create(args[0]);
+            // resolving the uri to see if there is another bindings in this uri.
+            ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder().uri(uri)
+                    .sessionTimeoutMs(10000).build();
+            BKDLConfig bkdlConfig;
+            try {
+                bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
+            } catch (IOException ie) {
+                bkdlConfig = null;
+            }
+            if (null == bkdlConfig) {
+                System.out.println("No bookkeeper is bound to " + uri);
+                return 0;
+            } else {
+                System.out.println("There is bookkeeper bound to " + uri + " : ");
+                System.out.println("");
+                System.out.println(bkdlConfig.toString());
+                System.out.println("");
+                if (!force && !IOUtils.confirmPrompt("Do you want to unbind " + uri + " :\n")) {
+                    return 0;
+                }
+            }
+            DLMetadata.unbind(uri);
+            System.out.println("Unbound on " + uri + ".");
+            return 0;
+        }
+    }
+
+    /**
+     * Bind Command to bind bookkeeper environment for a given distributed uri.
+     *
+     * TODO: move bind to namespace driver
+     */
+    class BindCommand extends OptsCommand {
+
+        Options options = new Options();
+
+        BindCommand() {
+            super("bind", "bind the bookkeeper environment settings for a given distributedlog instance.");
+            options.addOption("l", "bkLedgers", true, "ZooKeeper ledgers path for bookkeeper instance.");
+            options.addOption("s", "bkZkServers", true, "ZooKeeper servers used for bookkeeper for writers.");
+            options.addOption("bkzr", "bkZkServersForReader", true, "ZooKeeper servers used for bookkeeper for readers.");
+            options.addOption("dlzw", "dlZkServersForWriter", true, "ZooKeeper servers used for distributedlog for writers.");
+            options.addOption("dlzr", "dlZkServersForReader", true, "ZooKeeper servers used for distributedlog for readers.");
+            options.addOption("i", "sanityCheckTxnID", true, "Flag to sanity check highest txn id.");
+            options.addOption("r", "encodeRegionID", true, "Flag to encode region id.");
+            options.addOption("seqno", "firstLogSegmentSeqNo", true, "The first log segment sequence number to use after upgrade");
+            options.addOption("fns", "federatedNamespace", false, "Flag to turn a namespace to federated namespace");
+            options.addOption("f", "force", false, "Force binding without prompt.");
+            options.addOption("c", "creation", false, "Whether is it a creation binding.");
+            options.addOption("q", "query", false, "Query the bookkeeper bindings");
+        }
+
+        @Override
+        protected Options getOptions() {
+            return options;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "bind [options] <distributedlog uri>";
+        }
+
+        @Override
+        protected int runCmd(CommandLine cmdline) throws Exception {
+            boolean isQuery = cmdline.hasOption("q");
+            if (!isQuery && (!cmdline.hasOption("l") || !cmdline.hasOption("s"))) {
+                System.err.println("Error: Neither zkServers nor ledgersPath specified for bookkeeper environment.");
+                printUsage();
+                return -1;
+            }
+            String[] args = cmdline.getArgs();
+            if (args.length <= 0) {
+                System.err.println("No distributedlog uri specified.");
+                printUsage();
+                return -1;
+            }
+            boolean force = cmdline.hasOption("f");
+            boolean creation = cmdline.hasOption("c");
+            String bkLedgersPath = cmdline.getOptionValue("l");
+            String bkZkServersForWriter = cmdline.getOptionValue("s");
+            boolean sanityCheckTxnID =
+                    !cmdline.hasOption("i") || Boolean.parseBoolean(cmdline.getOptionValue("i"));
+            boolean encodeRegionID =
+                    cmdline.hasOption("r") && Boolean.parseBoolean(cmdline.getOptionValue("r"));
+
+            String bkZkServersForReader;
+            if (cmdline.hasOption("bkzr")) {
+                bkZkServersForReader = cmdline.getOptionValue("bkzr");
+            } else {
+                bkZkServersForReader = bkZkServersForWriter;
+            }
+
+            URI uri = URI.create(args[0]);
+
+            String dlZkServersForWriter;
+            String dlZkServersForReader;
+            if (cmdline.hasOption("dlzw")) {
+                dlZkServersForWriter = cmdline.getOptionValue("dlzw");
+            } else {
+                dlZkServersForWriter = BKNamespaceDriver.getZKServersFromDLUri(uri);
+            }
+            if (cmdline.hasOption("dlzr")) {
+                dlZkServersForReader = cmdline.getOptionValue("dlzr");
+            } else {
+                dlZkServersForReader = dlZkServersForWriter;
+            }
+
+            // resolving the uri to see if there is another bindings in this uri.
+            ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder().uri(uri).zkAclId(null)
+                    .sessionTimeoutMs(10000).build();
+            try {
+                BKDLConfig newBKDLConfig =
+                        new BKDLConfig(dlZkServersForWriter, dlZkServersForReader,
+                                       bkZkServersForWriter, bkZkServersForReader, bkLedgersPath)
+                                .setSanityCheckTxnID(sanityCheckTxnID)
+                                .setEncodeRegionID(encodeRegionID);
+
+                if (cmdline.hasOption("seqno")) {
+                    newBKDLConfig = newBKDLConfig.setFirstLogSegmentSeqNo(Long.parseLong(cmdline.getOptionValue("seqno")));
+                }
+
+                if (cmdline.hasOption("fns")) {
+                    newBKDLConfig = newBKDLConfig.setFederatedNamespace(true);
+                }
+
+                BKDLConfig bkdlConfig;
+                try {
+                    bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
+                } catch (IOException ie) {
+                    bkdlConfig = null;
+                }
+                if (null == bkdlConfig) {
+                    System.out.println("No bookkeeper is bound to " + uri);
+                } else {
+                    System.out.println("There is bookkeeper bound to " + uri + " : ");
+                    System.out.println("");
+                    System.out.println(bkdlConfig.toString());
+                    System.out.println("");
+                    if (!isQuery) {
+                        if (newBKDLConfig.equals(bkdlConfig)) {
+                            System.out.println("No bookkeeper binding needs to be updated. Quit.");
+                            return 0;
+                        } else if(!newBKDLConfig.isFederatedNamespace() && bkdlConfig.isFederatedNamespace()) {
+                            System.out.println("You can't turn a federated namespace back to non-federated.");
+                            return 0;
+                        } else {
+                            if (!force && !IOUtils.confirmPrompt("Do you want to bind " + uri
+                                        + " with new bookkeeper instance :\n" + newBKDLConfig)) {
+                                return 0;
+                            }
+                        }
+                    }
+                }
+                if (isQuery) {
+                    System.out.println("Done.");
+                    return 0;
+                }
+                DLMetadata dlMetadata = DLMetadata.create(newBKDLConfig);
+                if (creation) {
+                    try {
+                        dlMetadata.create(uri);
+                        System.out.println("Created binding on " + uri + ".");
+                    } catch (IOException ie) {
+                        System.err.println("Failed to create binding on " + uri + " : " + ie.getMessage());
+                    }
+                } else {
+                    try {
+                        dlMetadata.update(uri);
+                        System.out.println("Updated binding on " + uri + " : ");
+                        System.out.println("");
+                        System.out.println(newBKDLConfig.toString());
+                        System.out.println("");
+                    } catch (IOException ie) {
+                        System.err.println("Failed to update binding on " + uri + " : " + ie.getMessage());
+                    }
+                }
+                if (newBKDLConfig.isFederatedNamespace()) {
+                    try {
+                        FederatedZKLogMetadataStore.createFederatedNamespace(uri, zkc);
+                    } catch (KeeperException.NodeExistsException nee) {
+                        // ignore node exists exception
+                    }
+                }
+                return 0;
+            } finally {
+                zkc.close();
+            }
+        }
+    }
+
+    static class RepairSeqNoCommand extends PerDLCommand {
+
+        boolean dryrun = false;
+        boolean verbose = false;
+        final List<String> streams = new ArrayList<String>();
+
+        RepairSeqNoCommand() {
+            super("repairseqno", "Repair a stream whose inprogress log segment has lower sequence number.");
+            options.addOption("d", "dryrun", false, "Dry run without repairing");
+            options.addOption("l", "list", true, "List of streams to repair, separated by comma");
+            options.addOption("v", "verbose", false, "Print verbose messages");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            dryrun = cmdline.hasOption("d");
+            verbose = cmdline.hasOption("v");
+            force = !dryrun && cmdline.hasOption("f");
+            if (!cmdline.hasOption("l")) {
+                throw new ParseException("No streams provided to repair");
+            }
+            String streamsList = cmdline.getOptionValue("l");
+            Collections.addAll(streams, streamsList.split(","));
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            MetadataUpdater metadataUpdater = dryrun ?
+                    new DryrunLogSegmentMetadataStoreUpdater(getConf(),
+                            getLogSegmentMetadataStore()) :
+                    LogSegmentMetadataStoreUpdater.createMetadataUpdater(getConf(),
+                            getLogSegmentMetadataStore());
+            System.out.println("List of streams : ");
+            System.out.println(streams);
+            if (!IOUtils.confirmPrompt("Do you want to repair all these streams (Y/N):")) {
+                return -1;
+            }
+            for (String stream : streams) {
+                fixInprogressSegmentWithLowerSequenceNumber(getNamespace(), metadataUpdater, stream, verbose, !getForce());
+            }
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "repairseqno [options]";
+        }
+    }
+
+    static class DLCKCommand extends PerDLCommand {
+
+        boolean dryrun = false;
+        boolean verbose = false;
+        int concurrency = 1;
+
+        DLCKCommand() {
+            super("dlck", "Check and repair a distributedlog namespace");
+            options.addOption("d", "dryrun", false, "Dry run without repairing");
+            options.addOption("v", "verbose", false, "Print verbose messages");
+            options.addOption("cy", "concurrency", true, "Concurrency on checking streams");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            dryrun = cmdline.hasOption("d");
+            verbose = cmdline.hasOption("v");
+            if (cmdline.hasOption("cy")) {
+                try {
+                    concurrency = Integer.parseInt(cmdline.getOptionValue("cy"));
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid concurrency value : " + cmdline.getOptionValue("cy"));
+                }
+            }
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            MetadataUpdater metadataUpdater = dryrun ?
+                    new DryrunLogSegmentMetadataStoreUpdater(getConf(),
+                            getLogSegmentMetadataStore()) :
+                    LogSegmentMetadataStoreUpdater.createMetadataUpdater(getConf(),
+                            getLogSegmentMetadataStore());
+            OrderedScheduler scheduler = OrderedScheduler.newBuilder()
+                    .name("dlck-scheduler")
+                    .corePoolSize(Runtime.getRuntime().availableProcessors())
+                    .build();
+            ExecutorService executorService = Executors.newCachedThreadPool();
+            try {
+                checkAndRepairDLNamespace(getUri(), getNamespace(), metadataUpdater, scheduler,
+                                          verbose, !getForce(), concurrency);
+            } finally {
+                SchedulerUtils.shutdownScheduler(executorService, 5, TimeUnit.MINUTES);
+            }
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "dlck [options]";
+        }
+    }
+
+    static class DeleteStreamACLCommand extends PerDLCommand {
+
+        String stream = null;
+
+        DeleteStreamACLCommand() {
+            super("delete_stream_acl", "Delete ACL for a given stream");
+            options.addOption("s", "stream", true, "Stream to set ACL");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("s")) {
+                throw new ParseException("No stream to set ACL");
+            }
+            stream = cmdline.getOptionValue("s");
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(getZooKeeperClient(), getUri());
+            if (null == bkdlConfig.getACLRootPath()) {
+                // acl isn't enabled for this namespace.
+                System.err.println("ACL isn't enabled for namespace " + getUri());
+                return -1;
+            }
+            String zkPath = getUri() + "/" + bkdlConfig.getACLRootPath() + "/" + stream;
+            ZKAccessControl.delete(getZooKeeperClient(), zkPath);
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return null;
+        }
+    }
+
+    static class SetStreamACLCommand extends SetACLCommand {
+
+        String stream = null;
+
+        SetStreamACLCommand() {
+            super("set_stream_acl", "Set Default ACL for a given stream");
+            options.addOption("s", "stream", true, "Stream to set ACL");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("s")) {
+                throw new ParseException("No stream to set ACL");
+            }
+            stream = cmdline.getOptionValue("s");
+        }
+
+        @Override
+        protected String getZKPath(String zkRootPath) {
+            return zkRootPath + "/" + stream;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "set_stream_acl [options]";
+        }
+    }
+
+    static class SetDefaultACLCommand extends SetACLCommand {
+
+        SetDefaultACLCommand() {
+            super("set_default_acl", "Set Default ACL for a namespace");
+        }
+
+        @Override
+        protected String getZKPath(String zkRootPath) {
+            return zkRootPath;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "set_default_acl [options]";
+        }
+    }
+
+    static abstract class SetACLCommand extends PerDLCommand {
+
+        boolean denyWrite = false;
+        boolean denyTruncate = false;
+        boolean denyDelete = false;
+        boolean denyAcquire = false;
+        boolean denyRelease = false;
+
+        protected SetACLCommand(String name, String description) {
+            super(name, description);
+            options.addOption("dw", "deny-write", false, "Deny write/bulkWrite requests");
+            options.addOption("dt", "deny-truncate", false, "Deny truncate requests");
+            options.addOption("dd", "deny-delete", false, "Deny delete requests");
+            options.addOption("da", "deny-acquire", false, "Deny acquire requests");
+            options.addOption("dr", "deny-release", false, "Deny release requests");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            denyWrite = cmdline.hasOption("dw");
+            denyTruncate = cmdline.hasOption("dt");
+            denyDelete = cmdline.hasOption("dd");
+            denyAcquire = cmdline.hasOption("da");
+            denyRelease = cmdline.hasOption("dr");
+        }
+
+        protected abstract String getZKPath(String zkRootPath);
+
+        protected ZKAccessControl getZKAccessControl(ZooKeeperClient zkc, String zkPath) throws Exception {
+            ZKAccessControl accessControl;
+            try {
+                accessControl = Await.result(ZKAccessControl.read(zkc, zkPath, null));
+            } catch (KeeperException.NoNodeException nne) {
+                accessControl = new ZKAccessControl(new AccessControlEntry(), zkPath);
+            }
+            return accessControl;
+        }
+
+        protected void setZKAccessControl(ZooKeeperClient zkc, ZKAccessControl accessControl) throws Exception {
+            String zkPath = accessControl.getZKPath();
+            if (null == zkc.get().exists(zkPath, false)) {
+                accessControl.create(zkc);
+            } else {
+                accessControl.update(zkc);
+            }
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(getZooKeeperClient(), getUri());
+            if (null == bkdlConfig.getACLRootPath()) {
+                // acl isn't enabled for this namespace.
+                System.err.println("ACL isn't enabled for namespace " + getUri());
+                return -1;
+            }
+            String zkPath = getZKPath(getUri().getPath() + "/" + bkdlConfig.getACLRootPath());
+            ZKAccessControl accessControl = getZKAccessControl(getZooKeeperClient(), zkPath);
+            AccessControlEntry acl = accessControl.getAccessControlEntry();
+            acl.setDenyWrite(denyWrite);
+            acl.setDenyTruncate(denyTruncate);
+            acl.setDenyDelete(denyDelete);
+            acl.setDenyAcquire(denyAcquire);
+            acl.setDenyRelease(denyRelease);
+            setZKAccessControl(getZooKeeperClient(), accessControl);
+            return 0;
+        }
+
+    }
+
+    public DistributedLogAdmin() {
+        super();
+        commands.clear();
+        addCommand(new HelpCommand());
+        addCommand(new BindCommand());
+        addCommand(new UnbindCommand());
+        addCommand(new RepairSeqNoCommand());
+        addCommand(new DLCKCommand());
+        addCommand(new SetDefaultACLCommand());
+        addCommand(new SetStreamACLCommand());
+        addCommand(new DeleteStreamACLCommand());
+    }
+
+    @Override
+    protected String getName() {
+        return "dlog_admin";
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/admin/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/package-info.java
new file mode 100644
index 0000000..d708111
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/admin/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Admin Tools for DistributedLog
+ */
+package org.apache.distributedlog.admin;
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
new file mode 100644
index 0000000..56a4f2e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
@@ -0,0 +1,630 @@
+/**
+ * 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.auditor;
+
+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 org.apache.distributedlog.BookKeeperClient;
+import org.apache.distributedlog.BookKeeperClientBuilder;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.LogSegmentMetadata;
+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.namespace.NamespaceDriver;
+import org.apache.distributedlog.util.DLUtils;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAccessor;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * DL Auditor will audit DL namespace, e.g. find leaked ledger, report disk usage by streams.
+ */
+public class DLAuditor {
+
+    private static final Logger logger = LoggerFactory.getLogger(DLAuditor.class);
+
+    private final DistributedLogConfiguration conf;
+
+    public DLAuditor(DistributedLogConfiguration conf) {
+        this.conf = conf;
+    }
+
+    private ZooKeeperClient getZooKeeperClient(DistributedLogNamespace namespace) {
+        NamespaceDriver driver = namespace.getNamespaceDriver();
+        assert(driver instanceof BKNamespaceDriver);
+        return ((BKNamespaceDriver) driver).getWriterZKC();
+    }
+
+    private BookKeeperClient getBookKeeperClient(DistributedLogNamespace namespace) {
+        NamespaceDriver driver = namespace.getNamespaceDriver();
+        assert(driver instanceof BKNamespaceDriver);
+        return ((BKNamespaceDriver) driver).getReaderBKC();
+    }
+
+    private String validateAndGetZKServers(List<URI> uris) {
+        URI firstURI = uris.get(0);
+        String zkServers = BKNamespaceDriver.getZKServersFromDLUri(firstURI);
+        for (URI uri : uris) {
+            if (!zkServers.equalsIgnoreCase(BKNamespaceDriver.getZKServersFromDLUri(uri))) {
+                throw new IllegalArgumentException("Uris don't belong to same zookeeper cluster");
+            }
+        }
+        return zkServers;
+    }
+
+    private BKDLConfig resolveBKDLConfig(ZooKeeperClient zkc, List<URI> uris) throws IOException {
+        URI firstURI = uris.get(0);
+        BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, firstURI);
+        for (URI uri : uris) {
+            BKDLConfig anotherConfig = BKDLConfig.resolveDLConfig(zkc, uri);
+            if (!(Objects.equal(bkdlConfig.getBkLedgersPath(), anotherConfig.getBkLedgersPath())
+                    && Objects.equal(bkdlConfig.getBkZkServersForWriter(), anotherConfig.getBkZkServersForWriter()))) {
+                throw new IllegalArgumentException("Uris don't use same bookkeeper cluster");
+            }
+        }
+        return bkdlConfig;
+    }
+
+    public Pair<Set<Long>, Set<Long>> collectLedgers(List<URI> uris, List<List<String>> allocationPaths)
+            throws IOException {
+        Preconditions.checkArgument(uris.size() > 0, "No uri provided to audit");
+
+        String zkServers = validateAndGetZKServers(uris);
+        RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
+                conf.getZKRetryBackoffStartMillis(),
+                conf.getZKRetryBackoffMaxMillis(),
+                Integer.MAX_VALUE);
+        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
+                .name("DLAuditor-ZK")
+                .zkServers(zkServers)
+                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                .retryPolicy(retryPolicy)
+                .zkAclId(conf.getZkAclId())
+                .build();
+        ExecutorService executorService = Executors.newCachedThreadPool();
+        try {
+            BKDLConfig bkdlConfig = resolveBKDLConfig(zkc, uris);
+            logger.info("Resolved bookkeeper config : {}", bkdlConfig);
+
+            BookKeeperClient bkc = BookKeeperClientBuilder.newBuilder()
+                    .name("DLAuditor-BK")
+                    .dlConfig(conf)
+                    .zkServers(bkdlConfig.getBkZkServersForWriter())
+                    .ledgersPath(bkdlConfig.getBkLedgersPath())
+                    .build();
+            try {
+                Set<Long> bkLedgers = collectLedgersFromBK(bkc, executorService);
+                Set<Long> dlLedgers = collectLedgersFromDL(uris, allocationPaths);
+                return Pair.of(bkLedgers, dlLedgers);
+            } finally {
+                bkc.close();
+            }
+        } finally {
+            zkc.close();
+            executorService.shutdown();
+        }
+    }
+
+    /**
+     * Find leak ledgers phase 1: collect ledgers set.
+     */
+    private Set<Long> collectLedgersFromBK(BookKeeperClient bkc,
+                                           final ExecutorService executorService)
+            throws IOException {
+        LedgerManager lm = BookKeeperAccessor.getLedgerManager(bkc.get());
+
+        final Set<Long> ledgers = new HashSet<Long>();
+        final SettableFuture<Void> doneFuture = SettableFuture.create();
+
+        BookkeeperInternalCallbacks.Processor<Long> collector =
+                new BookkeeperInternalCallbacks.Processor<Long>() {
+            @Override
+            public void process(Long lid,
+                                final AsyncCallback.VoidCallback cb) {
+                synchronized (ledgers) {
+                    ledgers.add(lid);
+                    if (0 == ledgers.size() % 1000) {
+                        logger.info("Collected {} ledgers", ledgers.size());
+                    }
+                }
+                executorService.submit(new Runnable() {
+                    @Override
+                    public void run() {
+                        cb.processResult(BKException.Code.OK, null, null);
+                    }
+                });
+
+            }
+        };
+        AsyncCallback.VoidCallback finalCb = new AsyncCallback.VoidCallback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx) {
+                if (BKException.Code.OK == rc) {
+                    doneFuture.set(null);
+                } else {
+                    doneFuture.setException(BKException.create(rc));
+                }
+            }
+        };
+        lm.asyncProcessLedgers(collector, finalCb, null, BKException.Code.OK,
+                BKException.Code.ZKException);
+        try {
+            doneFuture.get();
+            logger.info("Collected total {} ledgers", ledgers.size());
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new DLInterruptedException("Interrupted on collecting ledgers : ", e);
+        } catch (ExecutionException e) {
+            if (e.getCause() instanceof IOException) {
+                throw (IOException)(e.getCause());
+            } else {
+                throw new IOException("Failed to collect ledgers : ", e.getCause());
+            }
+        }
+        return ledgers;
+    }
+
+    /**
+     * Find leak ledgers phase 2: collect ledgers from uris.
+     */
+    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());
+        try {
+            for (URI uri : uris) {
+                namespaces.add(
+                        DistributedLogNamespaceBuilder.newBuilder()
+                                .conf(conf)
+                                .uri(uri)
+                                .build());
+            }
+            final CountDownLatch doneLatch = new CountDownLatch(uris.size());
+            final AtomicInteger numFailures = new AtomicInteger(0);
+            ExecutorService executor = Executors.newFixedThreadPool(uris.size());
+            try {
+                int i = 0;
+                for (final DistributedLogNamespace namespace : namespaces) {
+                    final DistributedLogNamespace dlNamespace = namespace;
+                    final URI uri = uris.get(i);
+                    final List<String> aps = allocationPaths.get(i);
+                    i++;
+                    executor.submit(new Runnable() {
+                        @Override
+                        public void run() {
+                            try {
+                                logger.info("Collecting ledgers from {} : {}", uri, aps);
+                                collectLedgersFromAllocator(uri, namespace, aps, ledgers);
+                                synchronized (ledgers) {
+                                    logger.info("Collected {} ledgers from allocators for {} : {} ",
+                                            new Object[]{ledgers.size(), uri, ledgers});
+                                }
+                                collectLedgersFromDL(uri, namespace, ledgers);
+                            } catch (IOException e) {
+                                numFailures.incrementAndGet();
+                                logger.info("Error to collect ledgers from DL : ", e);
+                            }
+                            doneLatch.countDown();
+                        }
+                    });
+                }
+                try {
+                    doneLatch.await();
+                    if (numFailures.get() > 0) {
+                        throw new IOException(numFailures.get() + " errors to collect ledgers from DL");
+                    }
+                } catch (InterruptedException e) {
+                    Thread.currentThread().interrupt();
+                    logger.warn("Interrupted on collecting ledgers from DL : ", e);
+                    throw new DLInterruptedException("Interrupted on collecting ledgers from DL : ", e);
+                }
+            } finally {
+                executor.shutdown();
+            }
+        } finally {
+            for (DistributedLogNamespace namespace : namespaces) {
+                namespace.close();
+            }
+        }
+        return ledgers;
+    }
+
+    private void collectLedgersFromAllocator(final URI uri,
+                                             final DistributedLogNamespace namespace,
+                                             final List<String> allocationPaths,
+                                             final Set<Long> ledgers) throws IOException {
+        final LinkedBlockingQueue<String> poolQueue =
+                new LinkedBlockingQueue<String>();
+        for (String allocationPath : allocationPaths) {
+            String rootPath = uri.getPath() + "/" + allocationPath;
+            try {
+                List<String> pools = getZooKeeperClient(namespace).get().getChildren(rootPath, false);
+                for (String pool : pools) {
+                    poolQueue.add(rootPath + "/" + pool);
+                }
+            } catch (KeeperException e) {
+                throw new ZKException("Failed to get list of pools from " + rootPath, e);
+            } catch (InterruptedException e) {
+                Thread.currentThread().interrupt();
+                throw new DLInterruptedException("Interrupted on getting list of pools from " + rootPath, e);
+            }
+        }
+
+
+        logger.info("Collecting ledgers from allocators for {} : {}", uri, poolQueue);
+
+        executeAction(poolQueue, 10, new Action<String>() {
+            @Override
+            public void execute(String poolPath) throws IOException {
+                try {
+                    collectLedgersFromPool(poolPath);
+                } catch (InterruptedException e) {
+                    throw new DLInterruptedException("Interrupted on collecting ledgers from allocation pool " + poolPath, e);
+                } catch (KeeperException e) {
+                    throw new ZKException("Failed to collect ledgers from allocation pool " + poolPath, e.code());
+                }
+            }
+
+            private void collectLedgersFromPool(String poolPath)
+                    throws InterruptedException, ZooKeeperClient.ZooKeeperConnectionException, KeeperException {
+                List<String> allocators = getZooKeeperClient(namespace).get()
+                                        .getChildren(poolPath, false);
+                for (String allocator : allocators) {
+                    String allocatorPath = poolPath + "/" + allocator;
+                    byte[] data = getZooKeeperClient(namespace).get().getData(allocatorPath, false, new Stat());
+                    if (null != data && data.length > 0) {
+                        try {
+                            long ledgerId = DLUtils.bytes2LogSegmentId(data);
+                            synchronized (ledgers) {
+                                ledgers.add(ledgerId);
+                            }
+                        } catch (NumberFormatException nfe) {
+                            logger.warn("Invalid ledger found in allocator path {} : ", allocatorPath, nfe);
+                        }
+                    }
+                }
+            }
+        });
+
+        logger.info("Collected ledgers from allocators for {}.", uri);
+    }
+
+    private void collectLedgersFromDL(final URI uri,
+                                      final DistributedLogNamespace namespace,
+                                      final Set<Long> ledgers) throws IOException {
+        logger.info("Enumerating {} to collect streams.", uri);
+        Iterator<String> streams = namespace.getLogs();
+        final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
+        while (streams.hasNext()) {
+            streamQueue.add(streams.next());
+        }
+
+        logger.info("Collected {} streams from uri {} : {}",
+                    new Object[] { streamQueue.size(), uri, streams });
+
+        executeAction(streamQueue, 10, new Action<String>() {
+            @Override
+            public void execute(String stream) throws IOException {
+                collectLedgersFromStream(namespace, stream, ledgers);
+            }
+        });
+    }
+
+    private List<Long> collectLedgersFromStream(DistributedLogNamespace namespace,
+                                                String stream,
+                                                Set<Long> ledgers)
+            throws IOException {
+        DistributedLogManager dlm = namespace.openLog(stream);
+        try {
+            List<LogSegmentMetadata> segments = dlm.getLogSegments();
+            List<Long> sLedgers = new ArrayList<Long>();
+            for (LogSegmentMetadata segment : segments) {
+                synchronized (ledgers) {
+                    ledgers.add(segment.getLogSegmentId());
+                }
+                sLedgers.add(segment.getLogSegmentId());
+            }
+            return sLedgers;
+        } finally {
+            dlm.close();
+        }
+    }
+
+    /**
+     * Calculating stream space usage from given <i>uri</i>.
+     *
+     * @param uri dl uri
+     * @throws IOException
+     */
+    public Map<String, Long> calculateStreamSpaceUsage(final URI uri) throws IOException {
+        logger.info("Collecting stream space usage for {}.", uri);
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build();
+        try {
+            return calculateStreamSpaceUsage(uri, namespace);
+        } finally {
+            namespace.close();
+        }
+    }
+
+    private Map<String, Long> calculateStreamSpaceUsage(
+            final URI uri, final DistributedLogNamespace namespace)
+        throws IOException {
+        Iterator<String> streams = namespace.getLogs();
+        final LinkedBlockingQueue<String> streamQueue = new LinkedBlockingQueue<String>();
+        while (streams.hasNext()) {
+            streamQueue.add(streams.next());
+        }
+
+        final Map<String, Long> streamSpaceUsageMap =
+                new ConcurrentSkipListMap<String, Long>();
+        final AtomicInteger numStreamsCollected = new AtomicInteger(0);
+
+        executeAction(streamQueue, 10, new Action<String>() {
+            @Override
+            public void execute(String stream) throws IOException {
+                streamSpaceUsageMap.put(stream,
+                        calculateStreamSpaceUsage(namespace, stream));
+                if (numStreamsCollected.incrementAndGet() % 1000 == 0) {
+                    logger.info("Calculated {} streams from uri {}.", numStreamsCollected.get(), uri);
+                }
+            }
+        });
+
+        return streamSpaceUsageMap;
+    }
+
+    private long calculateStreamSpaceUsage(final DistributedLogNamespace namespace,
+                                           final String stream) throws IOException {
+        DistributedLogManager dlm = namespace.openLog(stream);
+        long totalBytes = 0;
+        try {
+            List<LogSegmentMetadata> segments = dlm.getLogSegments();
+            for (LogSegmentMetadata segment : segments) {
+                try {
+                    LedgerHandle lh = getBookKeeperClient(namespace).get().openLedgerNoRecovery(segment.getLogSegmentId(),
+                            BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
+                    totalBytes += lh.getLength();
+                    lh.close();
+                } catch (BKException e) {
+                    logger.error("Failed to open ledger {} : ", segment.getLogSegmentId(), e);
+                    throw new IOException("Failed to open ledger " + segment.getLogSegmentId(), e);
+                } catch (InterruptedException e) {
+                    logger.warn("Interrupted on opening ledger {} : ", segment.getLogSegmentId(), e);
+                    Thread.currentThread().interrupt();
+                    throw new DLInterruptedException("Interrupted on opening ledger " + segment.getLogSegmentId(), e);
+                }
+            }
+        } finally {
+            dlm.close();
+        }
+        return totalBytes;
+    }
+
+    public long calculateLedgerSpaceUsage(URI uri) throws IOException {
+        List<URI> uris = Lists.newArrayList(uri);
+        String zkServers = validateAndGetZKServers(uris);
+        RetryPolicy retryPolicy = new BoundExponentialBackoffRetryPolicy(
+                conf.getZKRetryBackoffStartMillis(),
+                conf.getZKRetryBackoffMaxMillis(),
+                Integer.MAX_VALUE);
+        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
+                .name("DLAuditor-ZK")
+                .zkServers(zkServers)
+                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                .retryPolicy(retryPolicy)
+                .zkAclId(conf.getZkAclId())
+                .build();
+        ExecutorService executorService = Executors.newCachedThreadPool();
+        try {
+            BKDLConfig bkdlConfig = resolveBKDLConfig(zkc, uris);
+            logger.info("Resolved bookkeeper config : {}", bkdlConfig);
+
+            BookKeeperClient bkc = BookKeeperClientBuilder.newBuilder()
+                    .name("DLAuditor-BK")
+                    .dlConfig(conf)
+                    .zkServers(bkdlConfig.getBkZkServersForWriter())
+                    .ledgersPath(bkdlConfig.getBkLedgersPath())
+                    .build();
+            try {
+                return calculateLedgerSpaceUsage(bkc, executorService);
+            } finally {
+                bkc.close();
+            }
+        } finally {
+            zkc.close();
+            executorService.shutdown();
+        }
+    }
+
+    private long calculateLedgerSpaceUsage(BookKeeperClient bkc,
+                                           final ExecutorService executorService)
+        throws IOException {
+        final AtomicLong totalBytes = new AtomicLong(0);
+        final AtomicLong totalEntries = new AtomicLong(0);
+        final AtomicLong numLedgers = new AtomicLong(0);
+
+        LedgerManager lm = BookKeeperAccessor.getLedgerManager(bkc.get());
+
+        final SettableFuture<Void> doneFuture = SettableFuture.create();
+        final BookKeeper bk = bkc.get();
+
+        BookkeeperInternalCallbacks.Processor<Long> collector =
+                new BookkeeperInternalCallbacks.Processor<Long>() {
+            @Override
+            public void process(final Long lid,
+                                final AsyncCallback.VoidCallback cb) {
+                numLedgers.incrementAndGet();
+                executorService.submit(new Runnable() {
+                    @Override
+                    public void run() {
+                        bk.asyncOpenLedgerNoRecovery(lid, BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8),
+                                new org.apache.bookkeeper.client.AsyncCallback.OpenCallback() {
+                            @Override
+                            public void openComplete(int rc, LedgerHandle lh, Object ctx) {
+                                final int cbRc;
+                                if (BKException.Code.OK == rc) {
+                                    totalBytes.addAndGet(lh.getLength());
+                                    totalEntries.addAndGet(lh.getLastAddConfirmed() + 1);
+                                    cbRc = rc;
+                                } else {
+                                    cbRc = BKException.Code.ZKException;
+                                }
+                                executorService.submit(new Runnable() {
+                                    @Override
+                                    public void run() {
+                                        cb.processResult(cbRc, null, null);
+                                    }
+                                });
+                            }
+                        }, null);
+                    }
+                });
+            }
+        };
+        AsyncCallback.VoidCallback finalCb = new AsyncCallback.VoidCallback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx) {
+                if (BKException.Code.OK == rc) {
+                    doneFuture.set(null);
+                } else {
+                    doneFuture.setException(BKException.create(rc));
+                }
+            }
+        };
+        lm.asyncProcessLedgers(collector, finalCb, null, BKException.Code.OK, BKException.Code.ZKException);
+        try {
+            doneFuture.get();
+            logger.info("calculated {} ledgers\n\ttotal bytes = {}\n\ttotal entries = {}",
+                    new Object[] { numLedgers.get(), totalBytes.get(), totalEntries.get() });
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new DLInterruptedException("Interrupted on calculating ledger space : ", e);
+        } catch (ExecutionException e) {
+            if (e.getCause() instanceof IOException) {
+                throw (IOException)(e.getCause());
+            } else {
+                throw new IOException("Failed to calculate ledger space : ", e.getCause());
+            }
+        }
+        return totalBytes.get();
+    }
+
+    public void close() {
+        // no-op
+    }
+
+    static interface Action<T> {
+        void execute(T item) throws IOException ;
+    }
+
+    static <T> void executeAction(final LinkedBlockingQueue<T> queue,
+                                  final int numThreads,
+                                  final Action<T> action) throws IOException {
+        final CountDownLatch failureLatch = new CountDownLatch(1);
+        final CountDownLatch doneLatch = new CountDownLatch(queue.size());
+        final AtomicInteger numFailures = new AtomicInteger(0);
+        final AtomicInteger completedThreads = new AtomicInteger(0);
+
+        ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
+        try {
+            for (int i = 0 ; i < numThreads; i++) {
+                executorService.submit(new Runnable() {
+                    @Override
+                    public void run() {
+                        while (true) {
+                            T item = queue.poll();
+                            if (null == item) {
+                                break;
+                            }
+                            try {
+                                action.execute(item);
+                            } catch (IOException ioe) {
+                                logger.error("Failed to execute action on item '{}'", item, ioe);
+                                numFailures.incrementAndGet();
+                                failureLatch.countDown();
+                                break;
+                            }
+                            doneLatch.countDown();
+                        }
+                        if (numFailures.get() == 0 && completedThreads.incrementAndGet() == numThreads) {
+                            failureLatch.countDown();
+                        }
+                    }
+                });
+            }
+            try {
+                failureLatch.await();
+                if (numFailures.get() > 0) {
+                    throw new IOException("Encountered " + numFailures.get() + " failures on executing action.");
+                }
+                doneLatch.await();
+            } catch (InterruptedException ie) {
+                Thread.currentThread().interrupt();
+                logger.warn("Interrupted on executing action", ie);
+                throw new DLInterruptedException("Interrupted on executing action", ie);
+            }
+        } finally {
+            executorService.shutdown();
+        }
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/DynamicQuorumConfigProvider.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/DynamicQuorumConfigProvider.java
new file mode 100644
index 0000000..ced4363
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/DynamicQuorumConfigProvider.java
@@ -0,0 +1,37 @@
+/**
+ * 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.bk;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+
+/**
+ * Provider returns quorum configs based on dynamic configuration.
+ */
+public class DynamicQuorumConfigProvider implements QuorumConfigProvider {
+
+    private final DynamicDistributedLogConfiguration conf;
+
+    public DynamicQuorumConfigProvider(DynamicDistributedLogConfiguration conf) {
+        this.conf = conf;
+    }
+
+    @Override
+    public QuorumConfig getQuorumConfig() {
+        return conf.getQuorumConfig();
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/ImmutableQuorumConfigProvider.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/ImmutableQuorumConfigProvider.java
new file mode 100644
index 0000000..463cdf9
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/ImmutableQuorumConfigProvider.java
@@ -0,0 +1,35 @@
+/**
+ * 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.bk;
+
+/**
+ * Provider that returns an immutable quorum config.
+ */
+public class ImmutableQuorumConfigProvider implements QuorumConfigProvider {
+
+    private final QuorumConfig quorumConfig;
+
+    public ImmutableQuorumConfigProvider(QuorumConfig quorumConfig) {
+        this.quorumConfig = quorumConfig;
+    }
+
+    @Override
+    public QuorumConfig getQuorumConfig() {
+        return quorumConfig;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocator.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocator.java
new file mode 100644
index 0000000..cd3f0ab
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocator.java
@@ -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.
+ */
+package org.apache.distributedlog.bk;
+
+import org.apache.distributedlog.util.Allocator;
+import org.apache.bookkeeper.client.LedgerHandle;
+
+import java.io.IOException;
+
+public interface LedgerAllocator extends Allocator<LedgerHandle, Object> {
+
+    /**
+     * Start the ledger allocator. The implementaion should not be blocking call.
+     */
+    void start() throws IOException;
+
+}
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
new file mode 100644
index 0000000..6ea248b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorDelegator.java
@@ -0,0 +1,83 @@
+/**
+ * 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.bk;
+
+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;
+
+/**
+ * Delegator of the underlying allocator. If it owns the allocator, it takes
+ * the responsibility of start the allocator and close the allocator.
+ */
+public class LedgerAllocatorDelegator implements LedgerAllocator {
+
+    private final LedgerAllocator allocator;
+    private final boolean ownAllocator;
+
+    /**
+     * Create an allocator's delegator.
+     *
+     * @param allocator
+     *          the underlying allocator
+     * @param ownAllocator
+     *          whether to own the allocator
+     */
+    public LedgerAllocatorDelegator(LedgerAllocator allocator,
+                                    boolean ownAllocator)
+            throws IOException {
+        this.allocator = allocator;
+        this.ownAllocator = ownAllocator;
+        if (this.ownAllocator) {
+            this.allocator.start();
+        }
+    }
+
+    @Override
+    public void start() throws IOException {
+        // no-op
+    }
+
+    @Override
+    public Future<Void> delete() {
+        return Future.exception(new UnsupportedOperationException("Can't delete an allocator by delegator"));
+    }
+
+    @Override
+    public void allocate() throws IOException {
+        this.allocator.allocate();
+    }
+
+    @Override
+    public Future<LedgerHandle> tryObtain(Transaction<Object> txn,
+                                          OpListener<LedgerHandle> listener) {
+        return this.allocator.tryObtain(txn, listener);
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        if (ownAllocator) {
+            return this.allocator.asyncClose();
+        } else {
+            return Future.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
new file mode 100644
index 0000000..4fff2f6
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorPool.java
@@ -0,0 +1,458 @@
+/**
+ * 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.bk;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+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.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;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+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;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+public class LedgerAllocatorPool implements LedgerAllocator {
+
+    static final Logger logger = LoggerFactory.getLogger(LedgerAllocatorPool.class);
+
+    private final DistributedLogConfiguration conf;
+    private final QuorumConfigProvider quorumConfigProvider;
+    private final BookKeeperClient bkc;
+    private final ZooKeeperClient zkc;
+    private final ScheduledExecutorService scheduledExecutorService;
+    private final String poolPath;
+    private final int corePoolSize;
+
+    private final LinkedList<SimpleLedgerAllocator> pendingList =
+            new LinkedList<SimpleLedgerAllocator>();
+    private final LinkedList<SimpleLedgerAllocator> allocatingList =
+            new LinkedList<SimpleLedgerAllocator>();
+    private final Map<String, SimpleLedgerAllocator> rescueMap =
+            new HashMap<String, SimpleLedgerAllocator>();
+    private final Map<LedgerHandle, SimpleLedgerAllocator> obtainMap =
+            new HashMap<LedgerHandle, SimpleLedgerAllocator>();
+    private final Map<SimpleLedgerAllocator, LedgerHandle> reverseObtainMap =
+            new HashMap<SimpleLedgerAllocator, LedgerHandle>();
+
+    public LedgerAllocatorPool(String poolPath, int corePoolSize,
+                               DistributedLogConfiguration conf,
+                               ZooKeeperClient zkc,
+                               BookKeeperClient bkc,
+                               ScheduledExecutorService scheduledExecutorService) throws IOException {
+        this.poolPath = poolPath;
+        this.corePoolSize = corePoolSize;
+        this.conf = conf;
+        this.quorumConfigProvider =
+                new ImmutableQuorumConfigProvider(conf.getQuorumConfig());
+        this.zkc = zkc;
+        this.bkc = bkc;
+        this.scheduledExecutorService = scheduledExecutorService;
+        initializePool();
+    }
+
+    @Override
+    public void start() throws IOException {
+        for (LedgerAllocator allocator : pendingList) {
+            // issue allocating requests during initialize
+            allocator.allocate();
+        }
+    }
+
+    @VisibleForTesting
+    synchronized int pendingListSize() {
+        return pendingList.size();
+    }
+
+    @VisibleForTesting
+    synchronized int allocatingListSize() {
+        return allocatingList.size();
+    }
+
+    @VisibleForTesting
+    public synchronized int obtainMapSize() {
+        return obtainMap.size();
+    }
+
+    @VisibleForTesting
+    synchronized int rescueSize() {
+        return rescueMap.size();
+    }
+
+    @VisibleForTesting
+    synchronized SimpleLedgerAllocator getLedgerAllocator(LedgerHandle lh) {
+        return obtainMap.get(lh);
+    }
+
+    private void initializePool() throws IOException {
+        try {
+            List<String> allocators;
+            try {
+                allocators = zkc.get().getChildren(poolPath, false);
+            } catch (KeeperException.NoNodeException e) {
+                logger.info("Allocator Pool {} doesn't exist. Creating it.", poolPath);
+                ZkUtils.createFullPathOptimistic(zkc.get(), poolPath, new byte[0], zkc.getDefaultACL(),
+                        CreateMode.PERSISTENT);
+                allocators = zkc.get().getChildren(poolPath, false);
+            }
+            if (null == allocators) {
+                allocators = new ArrayList<String>();
+            }
+            if (allocators.size() < corePoolSize) {
+                createAllocators(corePoolSize - allocators.size());
+                allocators = zkc.get().getChildren(poolPath, false);
+            }
+            initializeAllocators(allocators);
+        } catch (InterruptedException ie) {
+            throw new DLInterruptedException("Interrupted when ensuring " + poolPath + " created : ", ie);
+        } catch (KeeperException ke) {
+            throw new IOException("Encountered zookeeper exception when initializing pool " + poolPath + " : ", ke);
+        }
+    }
+
+    private void createAllocators(int numAllocators) throws InterruptedException, IOException {
+        final AtomicInteger numPendings = new AtomicInteger(numAllocators);
+        final AtomicInteger numFailures = new AtomicInteger(0);
+        final CountDownLatch latch = new CountDownLatch(1);
+        AsyncCallback.StringCallback createCallback = new AsyncCallback.StringCallback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx, String name) {
+                if (KeeperException.Code.OK.intValue() != rc) {
+                    numFailures.incrementAndGet();
+                    latch.countDown();
+                    return;
+                }
+                if (numPendings.decrementAndGet() == 0 && numFailures.get() == 0) {
+                    latch.countDown();
+                }
+            }
+        };
+        for (int i = 0; i < numAllocators; i++) {
+            zkc.get().create(poolPath + "/A", new byte[0],
+                             zkc.getDefaultACL(),
+                             CreateMode.PERSISTENT_SEQUENTIAL,
+                             createCallback, null);
+        }
+        latch.await();
+        if (numFailures.get() > 0) {
+            throw new IOException("Failed to create " + numAllocators + " allocators.");
+        }
+    }
+
+    /**
+     * Initialize simple allocators with given list of allocator names <i>allocators</i>.
+     * It initializes a simple allocator with its simple allocator path.
+     */
+    private void initializeAllocators(List<String> allocators) throws IOException, InterruptedException {
+        final AtomicInteger numPendings = new AtomicInteger(allocators.size());
+        final AtomicInteger numFailures = new AtomicInteger(0);
+        final CountDownLatch latch = new CountDownLatch(numPendings.get() > 0 ? 1 : 0);
+        AsyncCallback.DataCallback dataCallback = new AsyncCallback.DataCallback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
+                if (KeeperException.Code.OK.intValue() != rc) {
+                    numFailures.incrementAndGet();
+                    latch.countDown();
+                    return;
+                }
+                Versioned<byte[]> allocatorData =
+                        new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
+                SimpleLedgerAllocator allocator =
+                        new SimpleLedgerAllocator(path, allocatorData, quorumConfigProvider, zkc, bkc);
+                allocator.start();
+                pendingList.add(allocator);
+                if (numPendings.decrementAndGet() == 0 && numFailures.get() == 0) {
+                    latch.countDown();
+                }
+            }
+        };
+        for (String name : allocators) {
+            String path = poolPath + "/" + name;
+            zkc.get().getData(path, false, dataCallback, null);
+        }
+        latch.await();
+        if (numFailures.get() > 0) {
+            throw new IOException("Failed to initialize allocators : " + allocators);
+        }
+    }
+
+    private void scheduleAllocatorRescue(final SimpleLedgerAllocator ledgerAllocator) {
+        try {
+            scheduledExecutorService.schedule(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        rescueAllocator(ledgerAllocator);
+                    } catch (DLInterruptedException dle) {
+                        Thread.currentThread().interrupt();
+                    }
+                }
+            }, conf.getZKRetryBackoffStartMillis(), TimeUnit.MILLISECONDS);
+        } catch (RejectedExecutionException ree) {
+            logger.warn("Failed to schedule rescuing ledger allocator {} : ", ledgerAllocator.allocatePath, ree);
+        }
+    }
+
+    /**
+     * Rescue a ledger allocator from an ERROR state
+     * @param ledgerAllocator
+     *          ledger allocator to rescue
+     */
+    private void rescueAllocator(final SimpleLedgerAllocator ledgerAllocator) throws DLInterruptedException {
+        SimpleLedgerAllocator oldAllocator;
+        synchronized (this) {
+            oldAllocator = rescueMap.put(ledgerAllocator.allocatePath, ledgerAllocator);
+        }
+        if (oldAllocator != null) {
+            logger.info("ledger allocator {} is being rescued.", ledgerAllocator.allocatePath);
+            return;
+        }
+        try {
+            zkc.get().getData(ledgerAllocator.allocatePath, false, new AsyncCallback.DataCallback() {
+                @Override
+                public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
+                    boolean retry = false;
+                    SimpleLedgerAllocator newAllocator = null;
+                    if (KeeperException.Code.OK.intValue() == rc) {
+                        Versioned<byte[]> allocatorData =
+                                new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
+                        logger.info("Rescuing ledger allocator {}.", path);
+                        newAllocator = new SimpleLedgerAllocator(path, allocatorData, quorumConfigProvider, zkc, bkc);
+                        newAllocator.start();
+                        logger.info("Rescued ledger allocator {}.", path);
+                    } else if (KeeperException.Code.NONODE.intValue() == rc) {
+                        logger.info("Ledger allocator {} doesn't exist, skip rescuing it.", path);
+                    } else {
+                        retry = true;
+                    }
+                    synchronized (LedgerAllocatorPool.this) {
+                        rescueMap.remove(ledgerAllocator.allocatePath);
+                        if (null != newAllocator) {
+                            pendingList.addLast(newAllocator);
+                        }
+                    }
+                    if (retry) {
+                        scheduleAllocatorRescue(ledgerAllocator);
+                    }
+                }
+            }, null);
+        } catch (InterruptedException ie) {
+            logger.warn("Interrupted on rescuing ledger allocator {} : ", ledgerAllocator.allocatePath, ie);
+            synchronized (LedgerAllocatorPool.this) {
+                rescueMap.remove(ledgerAllocator.allocatePath);
+            }
+            throw new DLInterruptedException("Interrupted on rescuing ledger allocator " + ledgerAllocator.allocatePath, ie);
+        } catch (IOException ioe) {
+            logger.warn("Failed to rescue ledger allocator {}, retry rescuing it later : ", ledgerAllocator.allocatePath, ioe);
+            synchronized (LedgerAllocatorPool.this) {
+                rescueMap.remove(ledgerAllocator.allocatePath);
+            }
+            scheduleAllocatorRescue(ledgerAllocator);
+        }
+    }
+
+    @Override
+    public void allocate() throws IOException {
+        SimpleLedgerAllocator allocator;
+        synchronized (this) {
+            if (pendingList.isEmpty()) {
+                // if no ledger allocator available, we should fail it immediately, which the request will be redirected to other
+                // proxies
+                throw new IOException("No ledger allocator available under " + poolPath + ".");
+            } else {
+                allocator = pendingList.removeFirst();
+            }
+        }
+        boolean success = false;
+        try {
+            allocator.allocate();
+            synchronized (this) {
+                allocatingList.addLast(allocator);
+            }
+            success = true;
+        } finally {
+            if (!success) {
+                rescueAllocator(allocator);
+            }
+        }
+    }
+
+    @Override
+    public Future<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 + "."));
+            } else {
+                allocator = allocatingList.removeFirst();
+            }
+        }
+
+        final Promise<LedgerHandle> tryObtainPromise = new Promise<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);
+                }
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                try {
+                    rescueAllocator(allocator);
+                } catch (IOException ioe) {
+                    logger.info("Failed to rescue allocator {}", allocator.allocatePath, ioe);
+                }
+                tryObtainPromise.setException(cause);
+            }
+        };
+
+        allocator.tryObtain(txn, new Transaction.OpListener<LedgerHandle>() {
+            @Override
+            public void onCommit(LedgerHandle lh) {
+                confirmObtain(allocator);
+                listener.onCommit(lh);
+            }
+
+            @Override
+            public void onAbort(Throwable t) {
+                abortObtain(allocator);
+                listener.onAbort(t);
+            }
+        }).addEventListener(tryObtainListener);
+        return tryObtainPromise;
+    }
+
+    void confirmObtain(SimpleLedgerAllocator allocator) {
+        synchronized (this) {
+            LedgerHandle lh = reverseObtainMap.remove(allocator);
+            if (null != lh) {
+                obtainMap.remove(lh);
+            }
+        }
+        synchronized (this) {
+            pendingList.addLast(allocator);
+        }
+    }
+
+    void abortObtain(SimpleLedgerAllocator allocator) {
+        synchronized (this) {
+            LedgerHandle lh = reverseObtainMap.remove(allocator);
+            if (null != lh) {
+                obtainMap.remove(lh);
+            }
+        }
+        // if a ledger allocator is aborted, it is better to rescue it. since the ledger allocator might
+        // already encounter BadVersion exception.
+        try {
+            rescueAllocator(allocator);
+        } catch (DLInterruptedException e) {
+            logger.warn("Interrupted on rescuing ledger allocator pool {} : ", poolPath, e);
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        List<LedgerAllocator> allocatorsToClose;
+        synchronized (this) {
+            allocatorsToClose = Lists.newArrayListWithExpectedSize(
+                    pendingList.size() + allocatingList.size() + obtainMap.size());
+            for (LedgerAllocator allocator : pendingList) {
+                allocatorsToClose.add(allocator);
+            }
+            for (LedgerAllocator allocator : allocatingList) {
+                allocatorsToClose.add(allocator);
+            }
+            for (LedgerAllocator allocator : obtainMap.values()) {
+                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;
+            }
+        });
+    }
+
+    @Override
+    public Future<Void> delete() {
+        List<LedgerAllocator> allocatorsToDelete;
+        synchronized (this) {
+            allocatorsToDelete = Lists.newArrayListWithExpectedSize(
+                    pendingList.size() + allocatingList.size() + obtainMap.size());
+            for (LedgerAllocator allocator : pendingList) {
+                allocatorsToDelete.add(allocator);
+            }
+            for (LedgerAllocator allocator : allocatingList) {
+                allocatorsToDelete.add(allocator);
+            }
+            for (LedgerAllocator allocator : obtainMap.values()) {
+                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));
+            }
+        });
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorUtils.java
new file mode 100644
index 0000000..f6baac4
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/LedgerAllocatorUtils.java
@@ -0,0 +1,54 @@
+/**
+ * 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.bk;
+
+import org.apache.distributedlog.BookKeeperClient;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.ZooKeeperClient;
+
+import java.io.IOException;
+import java.util.concurrent.ScheduledExecutorService;
+
+public class LedgerAllocatorUtils {
+
+    /**
+     * Create ledger allocator pool.
+     *
+     * @param poolPath
+     *          ledger allocator pool path.
+     * @param corePoolSize
+     *          ledger allocator pool core size.
+     * @param conf
+     *          distributedlog configuration.
+     * @param zkc
+     *          zookeeper client
+     * @param bkc
+     *          bookkeeper client
+     * @return ledger allocator
+     * @throws IOException
+     */
+    public static LedgerAllocator createLedgerAllocatorPool(
+            String poolPath,
+            int corePoolSize,
+            DistributedLogConfiguration conf,
+            ZooKeeperClient zkc,
+            BookKeeperClient bkc,
+            ScheduledExecutorService scheduledExecutorService) throws IOException {
+        return new LedgerAllocatorPool(poolPath, corePoolSize, conf, zkc, bkc, scheduledExecutorService);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfig.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfig.java
new file mode 100644
index 0000000..07573ff
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfig.java
@@ -0,0 +1,92 @@
+/**
+ * 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.bk;
+
+import com.google.common.base.Objects;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Configuration for quorums
+ */
+public class QuorumConfig {
+
+    private static final Logger logger = LoggerFactory.getLogger(QuorumConfig.class);
+
+    private final int ensembleSize;
+    private final int writeQuorumSize;
+    private final int ackQuorumSize;
+
+    public QuorumConfig(int ensembleSize,
+                        int writeQuorumSize,
+                        int ackQuorumSize) {
+        this.ensembleSize = ensembleSize;
+        if (this.ensembleSize < writeQuorumSize) {
+            this.writeQuorumSize = this.ensembleSize;
+            logger.warn("Setting write quorum size {} greater than ensemble size {}",
+                    writeQuorumSize, this.ensembleSize);
+        } else {
+            this.writeQuorumSize = writeQuorumSize;
+        }
+        if (this.writeQuorumSize < ackQuorumSize) {
+            this.ackQuorumSize = this.writeQuorumSize;
+            logger.warn("Setting write ack quorum size {} greater than write quorum size {}",
+                    ackQuorumSize, this.writeQuorumSize);
+        } else {
+            this.ackQuorumSize = ackQuorumSize;
+        }
+    }
+
+    public int getEnsembleSize() {
+        return ensembleSize;
+    }
+
+    public int getWriteQuorumSize() {
+        return writeQuorumSize;
+    }
+
+    public int getAckQuorumSize() {
+        return ackQuorumSize;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hashCode(ensembleSize, writeQuorumSize, ackQuorumSize);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof QuorumConfig)) {
+            return false;
+        }
+        QuorumConfig other = (QuorumConfig) obj;
+        return ensembleSize == other.ensembleSize
+                && writeQuorumSize == other.writeQuorumSize
+                && ackQuorumSize == other.ackQuorumSize;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("QuorumConfig[ensemble=")
+          .append(ensembleSize).append(", write quorum=")
+          .append(writeQuorumSize).append(", ack quorum=")
+          .append(ackQuorumSize).append("]");
+        return sb.toString();
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfigProvider.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfigProvider.java
new file mode 100644
index 0000000..ccf90a9
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/QuorumConfigProvider.java
@@ -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.
+ */
+package org.apache.distributedlog.bk;
+
+/**
+ * Provider to provide quorum config
+ */
+public interface QuorumConfigProvider {
+
+    /**
+     * Get the quorum config for a given log stream.
+     *
+     * @return quorum config
+     */
+    QuorumConfig getQuorumConfig();
+
+}
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
new file mode 100644
index 0000000..e0102f3
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/SimpleLedgerAllocator.java
@@ -0,0 +1,536 @@
+/**
+ * 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.bk;
+
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.BookKeeperClient;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.util.DLUtils;
+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.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;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+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;
+import java.util.List;
+
+/**
+ * Allocator to allocate ledgers.
+ */
+public class SimpleLedgerAllocator implements LedgerAllocator, FutureEventListener<LedgerHandle>, OpListener<Version> {
+
+    static final Logger LOG = LoggerFactory.getLogger(SimpleLedgerAllocator.class);
+
+    static enum Phase {
+        ALLOCATING, ALLOCATED, HANDING_OVER, HANDED_OVER, ERROR
+    }
+
+    static class AllocationException extends IOException {
+
+        private static final long serialVersionUID = -1111397872059426882L;
+
+        private final Phase phase;
+
+        public AllocationException(Phase phase, String msg) {
+            super(msg);
+            this.phase = phase;
+        }
+
+        public Phase getPhase() {
+            return this.phase;
+        }
+
+    }
+
+    static class ConcurrentObtainException extends AllocationException {
+
+        private static final long serialVersionUID = -8532471098537176913L;
+
+        public ConcurrentObtainException(Phase phase, String msg) {
+            super(phase, msg);
+        }
+    }
+
+    // zookeeper client
+    final ZooKeeperClient zkc;
+    // bookkeeper client
+    final BookKeeperClient bkc;
+    // znode path
+    final String allocatePath;
+    // allocation phase
+    Phase phase = Phase.HANDED_OVER;
+    // version
+    ZkVersion version = new ZkVersion(-1);
+    // outstanding allocation
+    Promise<LedgerHandle> allocatePromise;
+    // outstanding tryObtain transaction
+    Transaction<Object> tryObtainTxn = null;
+    OpListener<LedgerHandle> tryObtainListener = null;
+    // ledger id left from previous allocation
+    Long ledgerIdLeftFromPrevAllocation = null;
+    // Allocated Ledger
+    LedgerHandle allocatedLh = null;
+
+    Future<Void> closeFuture = null;
+    final LinkedList<Future<Void>> ledgerDeletions =
+            new LinkedList<Future<Void>>();
+
+    // Ledger configuration
+    private final QuorumConfigProvider quorumConfigProvider;
+
+    static Future<Versioned<byte[]>> getAndCreateAllocationData(final String allocatePath,
+                                                                final ZooKeeperClient zkc) {
+        return Utils.zkGetData(zkc, allocatePath, false)
+                .flatMap(new AbstractFunction1<Versioned<byte[]>, Future<Versioned<byte[]>>>() {
+            @Override
+            public Future<Versioned<byte[]>> apply(Versioned<byte[]> result) {
+                if (null != result && null != result.getVersion() && null != result.getValue()) {
+                    return Future.value(result);
+                }
+                return createAllocationData(allocatePath, zkc);
+            }
+        });
+    }
+
+    private static Future<Versioned<byte[]>> createAllocationData(final String allocatePath,
+                                                                  final ZooKeeperClient zkc) {
+        try {
+            final Promise<Versioned<byte[]>> promise = new Promise<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,
+                                        new ZkVersion(stat.getVersion())));
+                            } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
+                                Utils.zkGetData(zkc, allocatePath, false).proxyTo(promise);
+                            } else {
+                                promise.setException(FutureUtils.zkException(
+                                        KeeperException.create(KeeperException.Code.get(rc)), allocatePath));
+                            }
+                        }
+                    }, null);
+            return promise;
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            return Future.exception(FutureUtils.zkException(e, allocatePath));
+        } catch (InterruptedException e) {
+            return Future.exception(FutureUtils.zkException(e, allocatePath));
+        }
+    }
+
+    public static Future<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,
+                    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);
+            }
+        });
+    }
+
+    /**
+     * Construct a ledger allocator.
+     *
+     * @param allocatePath
+     *          znode path to store the allocated ledger.
+     * @param allocationData
+     *          allocation data.
+     * @param quorumConfigProvider
+     *          Quorum configuration provider.
+     * @param zkc
+     *          zookeeper client.
+     * @param bkc
+     *          bookkeeper client.
+     */
+    public SimpleLedgerAllocator(String allocatePath,
+                                 Versioned<byte[]> allocationData,
+                                 QuorumConfigProvider quorumConfigProvider,
+                                 ZooKeeperClient zkc,
+                                 BookKeeperClient bkc) {
+        this.zkc = zkc;
+        this.bkc = bkc;
+        this.allocatePath = allocatePath;
+        this.quorumConfigProvider = quorumConfigProvider;
+        initialize(allocationData);
+    }
+
+    /**
+     * Initialize the allocator.
+     *
+     * @param allocationData
+     *          Allocation Data.
+     */
+    private void initialize(Versioned<byte[]> allocationData) {
+        setVersion((ZkVersion) allocationData.getVersion());
+        byte[] data = allocationData.getValue();
+        if (null != data && data.length > 0) {
+            // delete the allocated ledger since this is left by last allocation.
+            try {
+                ledgerIdLeftFromPrevAllocation = DLUtils.bytes2LogSegmentId(data);
+            } catch (NumberFormatException nfe) {
+                LOG.warn("Invalid data found in allocator path {} : ", allocatePath, nfe);
+            }
+        }
+
+    }
+
+    private synchronized void deleteLedgerLeftFromPreviousAllocationIfNecessary() {
+        if (null != ledgerIdLeftFromPrevAllocation) {
+            LOG.info("Deleting allocated-but-unused ledger left from previous allocation {}.", ledgerIdLeftFromPrevAllocation);
+            deleteLedger(ledgerIdLeftFromPrevAllocation);
+            ledgerIdLeftFromPrevAllocation = null;
+        }
+    }
+
+    @Override
+    public synchronized void allocate() throws IOException {
+        if (Phase.ERROR == phase) {
+            throw new AllocationException(Phase.ERROR, "Error on ledger allocator for " + allocatePath);
+        }
+        if (Phase.HANDED_OVER == phase) {
+            // issue an allocate request when ledger is already handed over.
+            allocateLedger();
+        }
+    }
+
+    @Override
+    public synchronized Future<LedgerHandle> tryObtain(final Transaction<Object> txn,
+                                                       final OpListener<LedgerHandle> listener) {
+        if (Phase.ERROR == phase) {
+            return Future.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,
+                    "Ledger handle is handling over to another thread : " + phase));
+        }
+        tryObtainTxn = txn;
+        tryObtainListener = listener;
+        if (null != allocatedLh) {
+            completeAllocation(allocatedLh);
+        }
+        return allocatePromise;
+    }
+
+    @Override
+    public void onCommit(Version r) {
+        confirmObtain((ZkVersion) r);
+    }
+
+    private void confirmObtain(ZkVersion zkVersion) {
+        boolean shouldAllocate = false;
+        OpListener<LedgerHandle> listenerToNotify = null;
+        LedgerHandle lhToNotify = null;
+        synchronized (this) {
+            if (Phase.HANDING_OVER == phase) {
+                setPhase(Phase.HANDED_OVER);
+                setVersion(zkVersion);
+                listenerToNotify = tryObtainListener;
+                lhToNotify = allocatedLh;
+                // reset the state
+                allocatedLh = null;
+                allocatePromise = null;
+                tryObtainTxn = null;
+                tryObtainListener = null;
+                // mark flag to issue an allocation request
+                shouldAllocate = true;
+            }
+        }
+        if (null != listenerToNotify && null != lhToNotify) {
+            // notify the listener
+            listenerToNotify.onCommit(lhToNotify);
+        }
+        if (shouldAllocate) {
+            // issue an allocation request
+            allocateLedger();
+        }
+    }
+
+    @Override
+    public void onAbort(Throwable t) {
+        OpListener<LedgerHandle> listenerToNotify;
+        synchronized (this) {
+            listenerToNotify = tryObtainListener;
+            if (t instanceof KeeperException &&
+                    ((KeeperException) t).code() == KeeperException.Code.BADVERSION) {
+                LOG.info("Set ledger allocator {} to ERROR state after hit bad version : version = {}",
+                        allocatePath, getVersion());
+                setPhase(Phase.ERROR);
+            } else {
+                if (Phase.HANDING_OVER == phase) {
+                    setPhase(Phase.ALLOCATED);
+                    tryObtainTxn = null;
+                    tryObtainListener = null;
+                }
+            }
+        }
+        if (null != listenerToNotify) {
+            listenerToNotify.onAbort(t);
+        }
+    }
+
+    private synchronized void setPhase(Phase phase) {
+        this.phase = phase;
+        LOG.info("Ledger allocator {} moved to phase {} : version = {}.",
+                new Object[] { allocatePath, phase, version });
+    }
+
+    private synchronized void allocateLedger() {
+        // make sure previous allocation is already handed over.
+        if (Phase.HANDED_OVER != phase) {
+            LOG.error("Trying allocate ledger for {} in phase {}, giving up.", allocatePath, phase);
+            return;
+        }
+        setPhase(Phase.ALLOCATING);
+        allocatePromise = new Promise<LedgerHandle>();
+        QuorumConfig quorumConfig = quorumConfigProvider.getQuorumConfig();
+        bkc.createLedger(
+                quorumConfig.getEnsembleSize(),
+                quorumConfig.getWriteQuorumSize(),
+                quorumConfig.getAckQuorumSize()
+        ).addEventListener(this);
+    }
+
+    private synchronized void completeAllocation(LedgerHandle lh) {
+        allocatedLh = lh;
+        if (null == tryObtainTxn) {
+            return;
+        }
+        org.apache.zookeeper.Op zkSetDataOp = org.apache.zookeeper.Op.setData(
+                allocatePath, DistributedLogConstants.EMPTY_BYTES, version.getZnodeVersion());
+        ZKVersionedSetOp commitOp = new ZKVersionedSetOp(zkSetDataOp, this);
+        tryObtainTxn.addOp(commitOp);
+        setPhase(Phase.HANDING_OVER);
+        FutureUtils.setValue(allocatePromise, lh);
+    }
+
+    private synchronized void failAllocation(Throwable cause) {
+        FutureUtils.setException(allocatePromise, cause);
+    }
+
+    @Override
+    public void onSuccess(LedgerHandle lh) {
+        // a ledger is created, update the ledger to allocation path before handling it over for usage.
+        markAsAllocated(lh);
+    }
+
+    @Override
+    public void onFailure(Throwable cause) {
+        LOG.error("Error creating ledger for allocating {} : ", allocatePath, cause);
+        setPhase(Phase.ERROR);
+        failAllocation(cause);
+    }
+
+    private synchronized ZkVersion getVersion() {
+        return version;
+    }
+
+    private synchronized void setVersion(ZkVersion newVersion) {
+        Version.Occurred occurred = newVersion.compare(version);
+        if (occurred == Version.Occurred.AFTER) {
+            LOG.info("Ledger allocator for {} moved version from {} to {}.",
+                    new Object[] { allocatePath, version, newVersion });
+            version = newVersion;
+        } else {
+            LOG.warn("Ledger allocator for {} received an old version {}, current version is {}.",
+                    new Object[] { allocatePath, newVersion , version });
+        }
+    }
+
+    private void markAsAllocated(final LedgerHandle lh) {
+        byte[] data = DLUtils.logSegmentId2Bytes(lh.getId());
+        Utils.zkSetData(zkc, allocatePath, data, getVersion())
+            .addEventListener(new FutureEventListener<ZkVersion>() {
+                @Override
+                public void onSuccess(ZkVersion version) {
+                    // we only issue deleting ledger left from previous allocation when we could allocate first ledger
+                    // as zookeeper version could prevent us doing stupid things.
+                    deleteLedgerLeftFromPreviousAllocationIfNecessary();
+                    setVersion(version);
+                    setPhase(Phase.ALLOCATED);
+                    // complete the allocation after it is marked as allocated
+                    completeAllocation(lh);
+                }
+
+                @Override
+                public void onFailure(Throwable cause) {
+                    setPhase(Phase.ERROR);
+                    deleteLedger(lh.getId());
+                    LOG.error("Fail mark ledger {} as allocated under {} : ",
+                            new Object[] { lh.getId(), allocatePath, cause });
+                    // fail the allocation since failed to mark it as allocated
+                    failAllocation(cause);
+                }
+            });
+    }
+
+    void deleteLedger(final long ledgerId) {
+        final Future<Void> deleteFuture = bkc.deleteLedger(ledgerId, true);
+        synchronized (ledgerDeletions) {
+            ledgerDeletions.add(deleteFuture);
+        }
+        deleteFuture.onFailure(new AbstractFunction1<Throwable, BoxedUnit>() {
+            @Override
+            public BoxedUnit apply(Throwable 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;
+            }
+        });
+    }
+
+    private synchronized boolean isClosing() {
+        return closeFuture != null;
+    }
+
+    private Future<Void> closeInternal(boolean cleanup) {
+        Promise<Void> closePromise;
+        synchronized (this) {
+            if (null != closeFuture) {
+                return closeFuture;
+            }
+            closePromise = new Promise<Void>();
+            closeFuture = closePromise;
+        }
+        if (!cleanup) {
+            LOG.info("Abort ledger allocator without cleaning up on {}.", allocatePath);
+            FutureUtils.setValue(closePromise, null);
+            return closePromise;
+        }
+        cleanupAndClose(closePromise);
+        return closePromise;
+    }
+
+    private void cleanupAndClose(final Promise<Void> closePromise) {
+        LOG.info("Closing ledger allocator on {}.", allocatePath);
+        final ZKTransaction txn = new ZKTransaction(zkc);
+        // try obtain ledger handle
+        tryObtain(txn, new OpListener<LedgerHandle>() {
+            @Override
+            public void onCommit(LedgerHandle r) {
+                // no-op
+                complete();
+            }
+
+            @Override
+            public void onAbort(Throwable t) {
+                // no-op
+                complete();
+            }
+
+            private void complete() {
+                FutureUtils.setValue(closePromise, null);
+                LOG.info("Closed ledger allocator on {}.", allocatePath);
+            }
+        }).addEventListener(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;
+                synchronized (ledgerDeletions) {
+                    outstandingDeletions = Lists.newArrayList(ledgerDeletions);
+                }
+                Future.collect(outstandingDeletions).addEventListener(new FutureEventListener<List<Void>>() {
+                    @Override
+                    public void onSuccess(List<Void> values) {
+                        txn.execute();
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        LOG.debug("Fail to obtain the allocated ledger handle when closing the allocator : ", cause);
+                        FutureUtils.setValue(closePromise, null);
+                    }
+                });
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                LOG.debug("Fail to obtain the allocated ledger handle when closing the allocator : ", cause);
+                FutureUtils.setValue(closePromise, null);
+            }
+        });
+
+    }
+
+    @Override
+    public void start() {
+        // nop
+    }
+
+    @Override
+    public Future<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());
+            }
+        });
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/bk/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/package-info.java
new file mode 100644
index 0000000..6173acd
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/bk/package-info.java
@@ -0,0 +1,24 @@
+/**
+ * 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.
+ */
+/**
+ * BookKeeper related util functions.
+ * <p>
+ * <h2>Ledger Allocator</h2>
+ *
+ */
+package org.apache.distributedlog.bk;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentListener.java b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentListener.java
new file mode 100644
index 0000000..20ed48b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentListener.java
@@ -0,0 +1,42 @@
+/**
+ * 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.callback;
+
+import org.apache.distributedlog.LogSegmentMetadata;
+
+import java.util.List;
+
+/**
+ * Listener on log segments changes for a given stream used by {@link org.apache.distributedlog.BKLogReadHandler}
+ */
+public interface LogSegmentListener {
+
+    /**
+     * Notified when <i>segments</i> updated. The new sorted log segments
+     * list is returned in this method.
+     *
+     * @param segments
+     *          updated list of segments.
+     */
+    void onSegmentsUpdated(List<LogSegmentMetadata> segments);
+
+    /**
+     * Notified when the log stream is deleted.
+     */
+    void onLogStreamDeleted();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentNamesListener.java b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentNamesListener.java
new file mode 100644
index 0000000..21fd155
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/LogSegmentNamesListener.java
@@ -0,0 +1,42 @@
+/**
+ * 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.callback;
+
+import org.apache.bookkeeper.versioning.Versioned;
+
+import java.util.List;
+
+/**
+ * Listener on list of log segments changes for a given stream used by
+ * {@link org.apache.distributedlog.logsegment.LogSegmentMetadataStore}.
+ */
+public interface LogSegmentNamesListener {
+    /**
+     * Notified when <i>segments</i> updated. The new log segments
+     * list is returned in this method.
+     *
+     * @param segments
+     *          updated list of segments.
+     */
+    void onSegmentsUpdated(Versioned<List<String>> segments);
+
+    /**
+     * Notified when the log stream is deleted.
+     */
+    void onLogStreamDeleted();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/NamespaceListener.java b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/NamespaceListener.java
new file mode 100644
index 0000000..443c1f8
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/NamespaceListener.java
@@ -0,0 +1,34 @@
+/**
+ * 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.callback;
+
+import com.google.common.annotations.Beta;
+
+import java.util.Iterator;
+
+@Beta
+public interface NamespaceListener {
+
+    /**
+     * Updated with latest streams.
+     *
+     * @param streams
+     *          latest list of streams under a given namespace.
+     */
+    void onStreamsChanged(Iterator<String> streams);
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/ReadAheadCallback.java b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/ReadAheadCallback.java
new file mode 100644
index 0000000..dccd2e8
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/ReadAheadCallback.java
@@ -0,0 +1,25 @@
+/**
+ * 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.callback;
+
+/**
+ * ReadAhead Callback
+ */
+public interface ReadAheadCallback {
+    void resumeReadAhead();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/callback/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/package-info.java
new file mode 100644
index 0000000..24cca9a
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/callback/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Callbacks for distributedlog operations.
+ */
+package org.apache.distributedlog.callback;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConcurrentBaseConfiguration.java b/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConcurrentBaseConfiguration.java
new file mode 100644
index 0000000..f189ad3
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConcurrentBaseConfiguration.java
@@ -0,0 +1,76 @@
+/**
+ * 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.config;
+
+import com.google.common.base.Preconditions;
+
+import org.apache.commons.configuration.AbstractConfiguration;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.Iterator;
+import java.util.concurrent.ConcurrentHashMap;
+
+/**
+ * Configuration view built on concurrent hash map for fast thread-safe access.
+ * Notes:
+ * 1. Multi-property list aggregation will not work in this class. I.e. commons config
+ * normally combines all properties with the same key into one list property automatically.
+ * 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;
+
+    public ConcurrentBaseConfiguration() {
+        this.map = new ConcurrentHashMap<String, Object>();
+    }
+
+    @Override
+    protected void addPropertyDirect(String key, Object value) {
+        Preconditions.checkNotNull(value);
+        map.put(key, value);
+    }
+
+    @Override
+    public Object getProperty(String key) {
+        return map.get(key);
+    }
+
+    @Override
+    public Iterator getKeys() {
+        return map.keySet().iterator();
+    }
+
+    @Override
+    public boolean containsKey(String key) {
+        return map.containsKey(key);
+    }
+
+    @Override
+    public boolean isEmpty() {
+        return map.isEmpty();
+    }
+
+    @Override
+    protected void clearPropertyDirect(String key) {
+        map.remove(key);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConcurrentConstConfiguration.java b/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConcurrentConstConfiguration.java
new file mode 100644
index 0000000..4e7f886
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConcurrentConstConfiguration.java
@@ -0,0 +1,31 @@
+/**
+ * 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.config;
+
+import com.google.common.base.Preconditions;
+import org.apache.commons.configuration.Configuration;
+
+/**
+ * Invariant thread-safe view of some configuration.
+ */
+public class ConcurrentConstConfiguration extends ConcurrentBaseConfiguration {
+    public ConcurrentConstConfiguration(Configuration conf) {
+        Preconditions.checkNotNull(conf);
+        copy(conf);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConfigurationListener.java b/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConfigurationListener.java
new file mode 100644
index 0000000..70059d4
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConfigurationListener.java
@@ -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.
+ */
+package org.apache.distributedlog.config;
+
+/**
+ * Configuration listener triggered when reloading configuration settings.
+ */
+public interface ConfigurationListener {
+
+    /**
+     * Reload the configuration.
+     *
+     * @param conf configuration to reload
+     */
+    void onReload(ConcurrentBaseConfiguration conf);
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConfigurationSubscription.java b/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConfigurationSubscription.java
new file mode 100644
index 0000000..0e5c897
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/ConfigurationSubscription.java
@@ -0,0 +1,186 @@
+/**
+ * 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.config;
+
+import java.io.FileNotFoundException;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArraySet;
+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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * ConfigurationSubscription publishes a reloading, thread-safe view of file configuration. The class
+ * periodically calls FileConfiguration.reload on the underlying conf, and propagates changes to the
+ * concurrent config. The configured FileChangedReloadingStrategy ensures that file config will only
+ * be reloaded if something changed.
+ * Notes:
+ * 1. Reload schedule is never terminated. The assumption is a finite number of these are started
+ * at the calling layer, and terminated only once the executor service is shut down.
+ * 2. The underlying FileConfiguration is not at all thread-safe, so its important to ensure access
+ * to this object is always single threaded.
+ */
+public class ConfigurationSubscription {
+    static final Logger LOG = LoggerFactory.getLogger(ConfigurationSubscription.class);
+
+    private final ConcurrentBaseConfiguration viewConfig;
+    private final ScheduledExecutorService executorService;
+    private final int reloadPeriod;
+    private final TimeUnit reloadUnit;
+    private final List<FileConfigurationBuilder> fileConfigBuilders;
+    private final List<FileConfiguration> fileConfigs;
+    private final CopyOnWriteArraySet<ConfigurationListener> confListeners;
+
+    public ConfigurationSubscription(ConcurrentBaseConfiguration viewConfig,
+                                     List<FileConfigurationBuilder> fileConfigBuilders,
+                                     ScheduledExecutorService executorService,
+                                     int reloadPeriod,
+                                     TimeUnit reloadUnit)
+            throws ConfigurationException {
+        Preconditions.checkNotNull(fileConfigBuilders);
+        Preconditions.checkArgument(!fileConfigBuilders.isEmpty());
+        Preconditions.checkNotNull(executorService);
+        Preconditions.checkNotNull(viewConfig);
+        this.viewConfig = viewConfig;
+        this.executorService = executorService;
+        this.reloadPeriod = reloadPeriod;
+        this.reloadUnit = reloadUnit;
+        this.fileConfigBuilders = fileConfigBuilders;
+        this.fileConfigs = Lists.newArrayListWithExpectedSize(this.fileConfigBuilders.size());
+        this.confListeners = new CopyOnWriteArraySet<ConfigurationListener>();
+        reload();
+        scheduleReload();
+    }
+
+    public void registerListener(ConfigurationListener listener) {
+        this.confListeners.add(listener);
+    }
+
+    public void unregisterListener(ConfigurationListener listener) {
+        this.confListeners.remove(listener);
+    }
+
+    private boolean initConfig() {
+        if (fileConfigs.isEmpty()) {
+            try {
+                for (FileConfigurationBuilder fileConfigBuilder : fileConfigBuilders) {
+                    FileConfiguration fileConfig = fileConfigBuilder.getConfiguration();
+                    FileChangedReloadingStrategy reloadingStrategy = new FileChangedReloadingStrategy();
+                    reloadingStrategy.setRefreshDelay(0);
+                    fileConfig.setReloadingStrategy(reloadingStrategy);
+                    fileConfigs.add(fileConfig);
+                }
+            } catch (ConfigurationException ex) {
+                if (!fileNotFound(ex)) {
+                    LOG.error("Config init failed {}", ex);
+                }
+            }
+        }
+        return !fileConfigs.isEmpty();
+    }
+
+    private void scheduleReload() {
+        executorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                reload();
+            }
+        }, 0, reloadPeriod, reloadUnit);
+    }
+
+    @VisibleForTesting
+    void reload() {
+        // No-op if already loaded.
+        if (!initConfig()) {
+            return;
+        }
+        // Reload if config exists.
+        Set<String> confKeys = Sets.newHashSet();
+        for (FileConfiguration fileConfig : fileConfigs) {
+            LOG.debug("Check and reload config, file={}, lastModified={}", fileConfig.getFile(),
+                    fileConfig.getFile().lastModified());
+            fileConfig.reload();
+            // load keys
+            Iterator keyIter = fileConfig.getKeys();
+            while (keyIter.hasNext()) {
+                String key = (String) keyIter.next();
+                confKeys.add(key);
+            }
+        }
+        // clear unexisted keys
+        Iterator viewIter = viewConfig.getKeys();
+        while (viewIter.hasNext()) {
+            String key = (String) viewIter.next();
+            if (!confKeys.contains(key)) {
+                clearViewProperty(key);
+            }
+        }
+        LOG.info("Reload features : {}", confKeys);
+        // load keys from files
+        for (FileConfiguration fileConfig : fileConfigs) {
+            try {
+                loadView(fileConfig);
+            } catch (Exception ex) {
+                if (!fileNotFound(ex)) {
+                    LOG.error("Config reload failed for file {}", fileConfig.getFileName(), ex);
+                }
+            }
+        }
+        for (ConfigurationListener listener : confListeners) {
+            listener.onReload(viewConfig);
+        }
+    }
+
+    private boolean fileNotFound(Exception ex) {
+        return ex instanceof FileNotFoundException ||
+                ex.getCause() != null && ex.getCause() instanceof FileNotFoundException;
+    }
+
+    private void loadView(FileConfiguration fileConfig) {
+        Iterator fileIter = fileConfig.getKeys();
+        while (fileIter.hasNext()) {
+            String key = (String) fileIter.next();
+            setViewProperty(fileConfig, key, fileConfig.getProperty(key));
+        }
+    }
+
+    private void clearViewProperty(String key) {
+        LOG.debug("Removing property, key={}", key);
+        viewConfig.clearProperty(key);
+    }
+
+    private void setViewProperty(FileConfiguration fileConfig,
+                                 String key,
+                                 Object value) {
+        if (!viewConfig.containsKey(key) || !viewConfig.getProperty(key).equals(value)) {
+            LOG.debug("Setting property, key={} value={}", key, fileConfig.getProperty(key));
+            viewConfig.setProperty(key, fileConfig.getProperty(key));
+        }
+    }
+}
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
new file mode 100644
index 0000000..c77778a
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicConfigurationFactory.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.config;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+
+import com.google.common.collect.Lists;
+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;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.commons.configuration.ConfigurationException;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Encapsulates creation of DynamicDistributedLogConfiguration instances. Ensures one instance per
+ * factory.
+ * Notes:
+ * Once loaded, stays loaded until shutdown. Caller ensures small finite number of configs are created.
+ */
+public class DynamicConfigurationFactory {
+    private static final Logger LOG = LoggerFactory.getLogger(DynamicConfigurationFactory.class);
+
+    private final Map<String, DynamicDistributedLogConfiguration> dynamicConfigs;
+    private final List<ConfigurationSubscription> subscriptions;
+    private final ScheduledExecutorService executorService;
+    private final int reloadPeriod;
+    private final TimeUnit reloadUnit;
+
+    public DynamicConfigurationFactory(ScheduledExecutorService executorService, int reloadPeriod, TimeUnit reloadUnit) {
+        this.executorService = executorService;
+        this.reloadPeriod = reloadPeriod;
+        this.reloadUnit = reloadUnit;
+        this.dynamicConfigs = new HashMap<String, DynamicDistributedLogConfiguration>();
+        this.subscriptions = new LinkedList<ConfigurationSubscription>();
+    }
+
+    public synchronized Optional<DynamicDistributedLogConfiguration> getDynamicConfiguration(
+            String configPath,
+            ConcurrentBaseConfiguration defaultConf) throws ConfigurationException {
+        Preconditions.checkNotNull(configPath);
+        try {
+            if (!dynamicConfigs.containsKey(configPath)) {
+                File configFile = new File(configPath);
+                FileConfigurationBuilder properties =
+                        new PropertiesConfigurationBuilder(configFile.toURI().toURL());
+                DynamicDistributedLogConfiguration dynConf =
+                        new DynamicDistributedLogConfiguration(defaultConf);
+                List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(properties);
+                ConfigurationSubscription subscription = new ConfigurationSubscription(
+                        dynConf, fileConfigBuilders, executorService, reloadPeriod, reloadUnit);
+                subscriptions.add(subscription);
+                dynamicConfigs.put(configPath, dynConf);
+                LOG.info("Loaded dynamic configuration at {}", configPath);
+            }
+            return Optional.of(dynamicConfigs.get(configPath));
+        } catch (MalformedURLException ex) {
+            throw new ConfigurationException(ex);
+        }
+    }
+
+    public synchronized Optional<DynamicDistributedLogConfiguration> getDynamicConfiguration(String configPath) throws ConfigurationException {
+        return getDynamicConfiguration(configPath, new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
+    }
+}
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
new file mode 100644
index 0000000..ea7f4a7
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/DynamicDistributedLogConfiguration.java
@@ -0,0 +1,356 @@
+/**
+ * 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.config;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.bk.QuorumConfig;
+
+import static org.apache.distributedlog.DistributedLogConfiguration.*;
+
+/**
+ * Whitelist dynamic configuration by adding an accessor to this class.
+ */
+public class DynamicDistributedLogConfiguration extends ConcurrentBaseConfiguration {
+
+    private final ConcurrentBaseConfiguration defaultConfig;
+
+    public DynamicDistributedLogConfiguration(ConcurrentBaseConfiguration defaultConfig) {
+        this.defaultConfig = defaultConfig;
+    }
+
+    private static int getInt(ConcurrentBaseConfiguration configuration,
+                              String newKey,
+                              String oldKey,
+                              int defaultValue) {
+        return configuration.getInt(newKey, configuration.getInt(oldKey, defaultValue));
+    }
+
+    /**
+     * Get retention period in hours
+     *
+     * @return retention period in hours
+     */
+    public int getRetentionPeriodHours() {
+        return getInt(
+                this,
+                BKDL_RETENTION_PERIOD_IN_HOURS,
+                BKDL_RETENTION_PERIOD_IN_HOURS_OLD,
+                getInt(defaultConfig,
+                        BKDL_RETENTION_PERIOD_IN_HOURS,
+                        BKDL_RETENTION_PERIOD_IN_HOURS_OLD,
+                        BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT)
+        );
+    }
+
+    /**
+     * A lower threshold bytes per second limit on writes to the distributedlog proxy.
+     *
+     * @return Bytes per second write limit
+     */
+    public int getBpsSoftWriteLimit() {
+        return getInt(DistributedLogConfiguration.BKDL_BPS_SOFT_WRITE_LIMIT,
+            defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_SOFT_WRITE_LIMIT,
+                DistributedLogConfiguration.BKDL_BPS_SOFT_WRITE_LIMIT_DEFAULT));
+    }
+
+    /**
+     * An upper threshold bytes per second limit on writes to the distributedlog proxy.
+     *
+     * @return Bytes per second write limit
+     */
+    public int getBpsHardWriteLimit() {
+        return getInt(DistributedLogConfiguration.BKDL_BPS_HARD_WRITE_LIMIT,
+            defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_HARD_WRITE_LIMIT,
+                DistributedLogConfiguration.BKDL_BPS_HARD_WRITE_LIMIT_DEFAULT));
+    }
+
+    /**
+     * A lower threshold requests per second limit on writes to the distributedlog proxy.
+     *
+     * @return Requests per second write limit
+     */
+    public int getRpsSoftWriteLimit() {
+        return getInt(DistributedLogConfiguration.BKDL_RPS_SOFT_WRITE_LIMIT,
+            defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_SOFT_WRITE_LIMIT,
+                DistributedLogConfiguration.BKDL_RPS_SOFT_WRITE_LIMIT_DEFAULT));
+    }
+
+    /**
+     * An upper threshold requests per second limit on writes to the distributedlog proxy.
+     *
+     * @return Requests per second write limit
+     */
+    public int getRpsHardWriteLimit() {
+        return getInt(DistributedLogConfiguration.BKDL_RPS_HARD_WRITE_LIMIT,
+            defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_HARD_WRITE_LIMIT,
+                DistributedLogConfiguration.BKDL_RPS_HARD_WRITE_LIMIT_DEFAULT));
+    }
+
+    /**
+     * A lower threshold requests per second limit on writes to the distributedlog proxy globally.
+     *
+     * @return Requests per second write limit
+     */
+    public int getRpsSoftServiceLimit() {
+        return getInt(DistributedLogConfiguration.BKDL_RPS_SOFT_SERVICE_LIMIT,
+            defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_SOFT_SERVICE_LIMIT,
+                DistributedLogConfiguration.BKDL_RPS_SOFT_SERVICE_LIMIT_DEFAULT));
+    }
+
+    /**
+     * An upper threshold requests per second limit on writes to the distributedlog proxy globally.
+     *
+     * @return Requests per second write limit
+     */
+    public int getRpsHardServiceLimit() {
+        return getInt(DistributedLogConfiguration.BKDL_RPS_HARD_SERVICE_LIMIT,
+            defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_HARD_SERVICE_LIMIT,
+                DistributedLogConfiguration.BKDL_RPS_HARD_SERVICE_LIMIT_DEFAULT));
+    }
+
+    /**
+     * When 60min average rps for the entire service instance hits this value, new streams will be
+     * rejected.
+     *
+     * @return Requests per second limit
+     */
+    public int getRpsStreamAcquireServiceLimit() {
+        return getInt(DistributedLogConfiguration.BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT,
+            defaultConfig.getInt(DistributedLogConfiguration.BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT,
+                DistributedLogConfiguration.BKDL_RPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT));
+    }
+
+    /**
+     * A lower threshold bytes per second limit on writes to the distributedlog proxy globally.
+     *
+     * @return Bytes per second write limit
+     */
+    public int getBpsSoftServiceLimit() {
+        return getInt(DistributedLogConfiguration.BKDL_BPS_SOFT_SERVICE_LIMIT,
+            defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_SOFT_SERVICE_LIMIT,
+                DistributedLogConfiguration.BKDL_BPS_SOFT_SERVICE_LIMIT_DEFAULT));
+    }
+
+    /**
+     * An upper threshold bytes per second limit on writes to the distributedlog proxy globally.
+     *
+     * @return Bytes per second write limit
+     */
+    public int getBpsHardServiceLimit() {
+        return getInt(DistributedLogConfiguration.BKDL_BPS_HARD_SERVICE_LIMIT,
+            defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_HARD_SERVICE_LIMIT,
+                DistributedLogConfiguration.BKDL_BPS_HARD_SERVICE_LIMIT_DEFAULT));
+    }
+
+    /**
+     * When 60min average bps for the entire service instance hits this value, new streams will be
+     * rejected.
+     *
+     * @return Bytes per second limit
+     */
+    public int getBpsStreamAcquireServiceLimit() {
+        return getInt(DistributedLogConfiguration.BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT,
+            defaultConfig.getInt(DistributedLogConfiguration.BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT,
+                DistributedLogConfiguration.BKDL_BPS_STREAM_ACQUIRE_SERVICE_LIMIT_DEFAULT));
+    }
+
+    /**
+     * Get percent of write bytes which should be delayed by BKDL_EI_INJECTED_WRITE_DELAY_MS.
+     *
+     * @return percent of writes to delay.
+     */
+    public double getEIInjectedWriteDelayPercent() {
+        return getDouble(DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_PERCENT,
+            defaultConfig.getDouble(DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_PERCENT,
+                DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_PERCENT_DEFAULT));
+    }
+
+    /**
+     * Get amount of time to delay writes for in writer failure injection.
+     *
+     * @return millis to delay writes for.
+     */
+    public int getEIInjectedWriteDelayMs() {
+        return getInt(DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_MS,
+            defaultConfig.getInt(DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_MS,
+                DistributedLogConfiguration.BKDL_EI_INJECTED_WRITE_DELAY_MS_DEFAULT));
+    }
+
+    /**
+     * Get output buffer size
+     *
+     * @return buffer size
+     */
+    public int getOutputBufferSize() {
+        return getInt(
+                this,
+                BKDL_OUTPUT_BUFFER_SIZE,
+                BKDL_OUTPUT_BUFFER_SIZE_OLD,
+                getInt(defaultConfig,
+                        BKDL_OUTPUT_BUFFER_SIZE,
+                        BKDL_OUTPUT_BUFFER_SIZE_OLD,
+                        BKDL_OUTPUT_BUFFER_SIZE_DEFAULT)
+        );
+    }
+
+    /**
+     * Get Periodic Log Flush Frequency in seconds
+     *
+     * @return periodic flush frequency
+     */
+    public int getPeriodicFlushFrequencyMilliSeconds() {
+        return getInt(DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
+            defaultConfig.getInt(DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS,
+                DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT));
+    }
+
+    /**
+     * Get the number of entries that readahead worker reads as a batch from bookkeeper
+     *
+     * @return the batch size
+     */
+    public int getReadAheadBatchSize() {
+        return getInt(
+                this,
+                BKDL_READAHEAD_BATCHSIZE,
+                BKDL_READAHEAD_BATCHSIZE_OLD,
+                getInt(defaultConfig,
+                        BKDL_READAHEAD_BATCHSIZE,
+                        BKDL_READAHEAD_BATCHSIZE_OLD,
+                        BKDL_READAHEAD_BATCHSIZE_DEFAULT)
+        );
+    }
+
+    /**
+     * Get the maximum number of {@link org.apache.distributedlog.LogRecord } that readahead worker will cache.
+     *
+     * @return the maximum number
+     */
+    public int getReadAheadMaxRecords() {
+        return getInt(
+                this,
+                BKDL_READAHEAD_MAX_RECORDS,
+                BKDL_READAHEAD_MAX_RECORDS_OLD,
+                getInt(defaultConfig,
+                        BKDL_READAHEAD_MAX_RECORDS,
+                        BKDL_READAHEAD_MAX_RECORDS_OLD,
+                        BKDL_READAHEAD_MAX_RECORDS_DEFAULT)
+        );
+    }
+
+    /**
+     * Whether to enable ledger allocator pool or not.
+     * It is disabled by default.
+     *
+     * @return whether using ledger allocator pool or not.
+     */
+    public boolean getEnableLedgerAllocatorPool() {
+        return getBoolean(BKDL_ENABLE_LEDGER_ALLOCATOR_POOL,
+                defaultConfig.getBoolean(
+                        BKDL_ENABLE_LEDGER_ALLOCATOR_POOL,
+                        BKDL_ENABLE_LEDGER_ALLOCATOR_POOL_DEFAULT));
+    }
+
+    /**
+     * Get the quorum config.
+     *
+     * @return quorum config
+     */
+    public QuorumConfig getQuorumConfig() {
+        int ensembleSize = getInt(
+                this,
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE,
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD,
+                getInt(defaultConfig,
+                        BKDL_BOOKKEEPER_ENSEMBLE_SIZE,
+                        BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD,
+                        BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT));
+        int writeQuorumSize = getInt(
+                this,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
+                getInt(defaultConfig,
+                        BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
+                        BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
+                        BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT));
+        int ackQuorumSize = getInt(
+                this,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
+                getInt(defaultConfig,
+                        BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
+                        BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
+                        BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT));
+        return new QuorumConfig(ensembleSize, writeQuorumSize, ackQuorumSize);
+    }
+
+    /**
+     * Get the maximum number of partitions of each stream allowed to be acquired per proxy.
+     *
+     * @return maximum number of partitions of each stream allowed to be acquired
+     * @see DistributedLogConfiguration#getMaxAcquiredPartitionsPerProxy()
+     */
+    public int getMaxAcquiredPartitionsPerProxy() {
+        return getInt(
+                BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY,
+                defaultConfig.getInt(
+                        BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY,
+                        BKDL_MAX_ACQUIRED_PARTITIONS_PER_PROXY_DEFAULT)
+        );
+    }
+
+    /**
+     * Get the maximum number of partitions of each stream allowed to cache per proxy.
+     *
+     * @return maximum number of partitions of each stream allowed to cache
+     * @see DistributedLogConfiguration#getMaxAcquiredPartitionsPerProxy()
+     */
+    public int getMaxCachedPartitionsPerProxy() {
+        return getInt(
+                BKDL_MAX_CACHED_PARTITIONS_PER_PROXY,
+                defaultConfig.getInt(
+                        BKDL_MAX_CACHED_PARTITIONS_PER_PROXY,
+                        BKDL_MAX_CACHED_PARTITIONS_PER_PROXY_DEFAULT)
+        );
+    }
+
+    /**
+     * Check whether the durable write is enabled.
+     *
+     * @return true if durable write is enabled. otherwise, false.
+     */
+    public boolean isDurableWriteEnabled() {
+        return getBoolean(BKDL_IS_DURABLE_WRITE_ENABLED,
+                defaultConfig.getBoolean(
+                        BKDL_IS_DURABLE_WRITE_ENABLED,
+                        BKDL_IS_DURABLE_WRITE_ENABLED_DEFAULT));
+    }
+
+    /**
+     * Get the flag whether to deserialize recordset on reads.
+     *
+     * @return flag whether to deserialize recordset on reads.
+     */
+    public boolean getDeserializeRecordSetOnReads() {
+        return getBoolean(BKDL_DESERIALIZE_RECORDSET_ON_READS,
+                defaultConfig.getBoolean(
+                        BKDL_DESERIALIZE_RECORDSET_ON_READS,
+                        BKDL_DESERIALIZE_RECORDSET_ON_READS_DEFAULT));
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/FileConfigurationBuilder.java b/distributedlog-core/src/main/java/org/apache/distributedlog/config/FileConfigurationBuilder.java
new file mode 100644
index 0000000..dbf8fe7
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/FileConfigurationBuilder.java
@@ -0,0 +1,28 @@
+/**
+ * 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.config;
+
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.FileConfiguration;
+
+/**
+ * Abstract out FileConfiguration subclass construction.
+ */
+public interface FileConfigurationBuilder {
+    FileConfiguration getConfiguration() throws ConfigurationException;
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/PropertiesConfigurationBuilder.java b/distributedlog-core/src/main/java/org/apache/distributedlog/config/PropertiesConfigurationBuilder.java
new file mode 100644
index 0000000..df1408c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/PropertiesConfigurationBuilder.java
@@ -0,0 +1,40 @@
+/**
+ * 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.config;
+
+import java.net.URL;
+
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.FileConfiguration;
+import org.apache.commons.configuration.PropertiesConfiguration;
+
+/**
+ * Hide PropertiesConfiguration dependency.
+ */
+public class PropertiesConfigurationBuilder implements FileConfigurationBuilder {
+    private URL url;
+
+    public PropertiesConfigurationBuilder(URL url) {
+        this.url = url;
+    }
+
+    @Override
+    public FileConfiguration getConfiguration() throws ConfigurationException {
+        return new PropertiesConfiguration(url);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/config/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/config/package-info.java
new file mode 100644
index 0000000..bc9c512
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/config/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * DistributedLog Configuration
+ */
+package org.apache.distributedlog.config;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/ZKException.java b/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/ZKException.java
new file mode 100644
index 0000000..b9e9d12
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/exceptions/ZKException.java
@@ -0,0 +1,54 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.KeeperException.Code;
+
+/**
+ * TODO: move ZKException to distributedlog-protocol
+ */
+public class ZKException extends DLException {
+
+    private static final long serialVersionUID = 7542748595054923600L;
+
+    final KeeperException.Code code;
+
+    public ZKException(String msg, Code code) {
+        super(StatusCode.ZOOKEEPER_ERROR, msg + " : " + code);
+        this.code = code;
+    }
+
+    public ZKException(String msg, KeeperException exception) {
+        super(StatusCode.ZOOKEEPER_ERROR, msg, exception);
+        this.code = exception.code();
+    }
+
+    public Code getKeeperExceptionCode() {
+        return this.code;
+    }
+
+    public static boolean isRetryableZKException(ZKException zke) {
+        KeeperException.Code code = zke.getKeeperExceptionCode();
+        return KeeperException.Code.CONNECTIONLOSS == code ||
+                KeeperException.Code.OPERATIONTIMEOUT == code ||
+                KeeperException.Code.SESSIONEXPIRED == code ||
+                KeeperException.Code.SESSIONMOVED == code;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java
new file mode 100644
index 0000000..df92b95
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/AbstractFeatureProvider.java
@@ -0,0 +1,103 @@
+/**
+ * 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.feature;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.bookkeeper.feature.CacheableFeatureProvider;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.commons.configuration.ConfigurationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+
+/**
+ * Decider based feature provider
+ */
+public abstract class AbstractFeatureProvider<T extends Feature> extends CacheableFeatureProvider<T> {
+
+    protected static final Logger logger = LoggerFactory.getLogger(AbstractFeatureProvider.class);
+
+    public static FeatureProvider getFeatureProvider(String rootScope,
+                                                     DistributedLogConfiguration conf,
+                                                     StatsLogger statsLogger)
+            throws IOException {
+        Class<? extends FeatureProvider> featureProviderClass;
+        try {
+            featureProviderClass = conf.getFeatureProviderClass();
+        } catch (ConfigurationException e) {
+            throw new IOException("Can't initialize the feature provider : ", e);
+        }
+        // create feature provider
+        Constructor<? extends FeatureProvider> constructor;
+        try {
+            constructor = featureProviderClass.getDeclaredConstructor(
+                    String.class,
+                    DistributedLogConfiguration.class,
+                    StatsLogger.class);
+        } catch (NoSuchMethodException e) {
+            throw new IOException("No constructor found for feature provider class " + featureProviderClass + " : ", e);
+        }
+        try {
+            return constructor.newInstance(rootScope, conf, statsLogger);
+        } catch (InstantiationException e) {
+            throw new IOException("Failed to instantiate feature provider : ", e);
+        } catch (IllegalAccessException e) {
+            throw new IOException("Encountered illegal access when instantiating feature provider : ", e);
+        } catch (InvocationTargetException e) {
+            Throwable targetException = e.getTargetException();
+            if (targetException instanceof IOException) {
+                throw (IOException) targetException;
+            } else {
+                throw new IOException("Encountered invocation target exception while instantiating feature provider : ", e);
+            }
+        }
+    }
+
+    protected final DistributedLogConfiguration conf;
+    protected final StatsLogger statsLogger;
+
+    protected AbstractFeatureProvider(String rootScope,
+                                      DistributedLogConfiguration conf,
+                                      StatsLogger statsLogger) {
+        super(rootScope);
+        this.conf = conf;
+        this.statsLogger = statsLogger;
+    }
+
+    /**
+     * Start the feature provider.
+     *
+     * @throws IOException when failed to start the feature provider.
+     */
+    public void start() throws IOException {
+        // no-op
+    }
+
+    /**
+     * Stop the feature provider.
+     */
+    public void stop() {
+        // no-op
+    }
+
+}
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
new file mode 100644
index 0000000..83cac22
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/ConfigurationFeatureProvider.java
@@ -0,0 +1,76 @@
+/**
+ * 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.feature;
+
+import org.apache.distributedlog.config.ConcurrentBaseConfiguration;
+import org.apache.bookkeeper.feature.CacheableFeatureProvider;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.feature.SettableFeature;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Feature Provider that load features from configuration
+ */
+class ConfigurationFeatureProvider extends CacheableFeatureProvider {
+
+    private static final Logger logger = LoggerFactory.getLogger(ConfigurationFeatureProvider.class);
+
+    static SettableFeature makeFeature(ConcurrentBaseConfiguration featuresConf,
+                                       ConcurrentMap<String, SettableFeature> features,
+                                       String featureName) {
+        SettableFeature feature = features.get(featureName);
+        if (null == feature) {
+            int availability = featuresConf.getInt(featureName, 0);
+            feature = new SettableFeature(featureName, availability);
+            SettableFeature oldFeature =
+                    features.putIfAbsent(featureName, feature);
+            if (null != oldFeature) {
+                feature = oldFeature;
+            } else {
+                logger.info("Load feature {}={}", featureName, availability);
+            }
+        }
+        return feature;
+    }
+
+    private final ConcurrentBaseConfiguration featuresConf;
+    private final ConcurrentMap<String, SettableFeature> features;
+
+    ConfigurationFeatureProvider(String rootScope,
+                                 ConcurrentBaseConfiguration featuresConf,
+                                 ConcurrentMap<String, SettableFeature> features) {
+        super(rootScope);
+        this.featuresConf = featuresConf;
+        this.features = features;
+    }
+
+    @Override
+    protected Feature makeFeature(String featureName) {
+        return makeFeature(featuresConf, features, featureName);
+    }
+
+    @Override
+    protected FeatureProvider makeProvider(String fullScopeName) {
+        return new ConfigurationFeatureProvider(
+                fullScopeName, featuresConf, features);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/CoreFeatureKeys.java b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/CoreFeatureKeys.java
new file mode 100644
index 0000000..bc531ef
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/CoreFeatureKeys.java
@@ -0,0 +1,29 @@
+/**
+ * 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.feature;
+
+/**
+ * List of feature keys used by distributedlog core
+ */
+public enum CoreFeatureKeys {
+    // @Deprecated: bkc features are managed by bookkeeper prefixed with a scope
+    DISABLE_DURABILITY_ENFORCEMENT,
+    // disabling logsegment rolling
+    DISABLE_LOGSEGMENT_ROLLING,
+    DISABLE_WRITE_LIMIT,
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DefaultFeatureProvider.java b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DefaultFeatureProvider.java
new file mode 100644
index 0000000..6c30ab3
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DefaultFeatureProvider.java
@@ -0,0 +1,47 @@
+/**
+ * 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.feature;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.feature.SettableFeature;
+import org.apache.bookkeeper.feature.SettableFeatureProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * Default feature provider which disable all features by default.
+ */
+public class DefaultFeatureProvider extends AbstractFeatureProvider {
+
+    public DefaultFeatureProvider(String rootScope,
+                                  DistributedLogConfiguration conf,
+                                  StatsLogger statsLogger) {
+        super(rootScope, conf, statsLogger);
+    }
+
+    @Override
+    protected Feature makeFeature(String featureName) {
+        return new SettableFeature(featureName, 0);
+    }
+
+    @Override
+    protected FeatureProvider makeProvider(String fullScopeName) {
+        return new SettableFeatureProvider(fullScopeName, 0);
+    }
+}
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
new file mode 100644
index 0000000..4689d51
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/DynamicConfigurationFeatureProvider.java
@@ -0,0 +1,132 @@
+/**
+ * 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.feature;
+
+import com.google.common.base.Preconditions;
+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.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.feature.SettableFeature;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.commons.configuration.ConfigurationException;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Feature Provider based dynamic configuration.
+ */
+public class DynamicConfigurationFeatureProvider extends AbstractFeatureProvider
+        implements ConfigurationListener {
+
+    private final ConcurrentBaseConfiguration featuresConf;
+    private ConfigurationSubscription featuresConfSubscription;
+    private final ConcurrentMap<String, SettableFeature> features;
+    private final ScheduledExecutorService executorService;
+
+    public DynamicConfigurationFeatureProvider(String rootScope,
+                                               DistributedLogConfiguration conf,
+                                               StatsLogger statsLogger) {
+        super(rootScope, conf, statsLogger);
+        this.features = new ConcurrentHashMap<String, SettableFeature>();
+        this.featuresConf = new ConcurrentBaseConfiguration();
+        this.executorService = Executors.newSingleThreadScheduledExecutor(
+                new ThreadFactoryBuilder().setNameFormat("DynamicConfigurationFeatureProvider-%d").build());
+    }
+
+    ConcurrentBaseConfiguration getFeatureConf() {
+        return featuresConf;
+    }
+
+    ConfigurationSubscription getFeatureConfSubscription() {
+        return featuresConfSubscription;
+    }
+
+    @Override
+    public void start() throws IOException {
+        List<FileConfigurationBuilder> fileConfigBuilders =
+                Lists.newArrayListWithExpectedSize(2);
+        String baseConfigPath = conf.getFileFeatureProviderBaseConfigPath();
+        Preconditions.checkNotNull(baseConfigPath);
+        File baseConfigFile = new File(baseConfigPath);
+        FileConfigurationBuilder baseProperties =
+                new PropertiesConfigurationBuilder(baseConfigFile.toURI().toURL());
+        fileConfigBuilders.add(baseProperties);
+        String overlayConfigPath = conf.getFileFeatureProviderOverlayConfigPath();
+        if (null != overlayConfigPath) {
+            File overlayConfigFile = new File(overlayConfigPath);
+            FileConfigurationBuilder overlayProperties =
+                    new PropertiesConfigurationBuilder(overlayConfigFile.toURI().toURL());
+            fileConfigBuilders.add(overlayProperties);
+        }
+        try {
+            this.featuresConfSubscription = new ConfigurationSubscription(
+                    this.featuresConf,
+                    fileConfigBuilders,
+                    executorService,
+                    conf.getDynamicConfigReloadIntervalSec(),
+                    TimeUnit.SECONDS);
+        } catch (ConfigurationException e) {
+            throw new IOException("Failed to register subscription on features configuration");
+        }
+        this.featuresConfSubscription.registerListener(this);
+    }
+
+    @Override
+    public void stop() {
+        this.executorService.shutdown();
+    }
+
+    @Override
+    public void onReload(ConcurrentBaseConfiguration conf) {
+        for (Map.Entry<String, SettableFeature> feature : features.entrySet()) {
+            String featureName = feature.getKey();
+            int availability = conf.getInt(featureName, 0);
+            if (availability != feature.getValue().availability()) {
+                feature.getValue().set(availability);
+                logger.info("Reload feature {}={}", featureName, availability);
+            }
+        }
+    }
+
+    @Override
+    protected Feature makeFeature(String featureName) {
+        return ConfigurationFeatureProvider.makeFeature(
+                featuresConf, features, featureName);
+    }
+
+    @Override
+    protected FeatureProvider makeProvider(String fullScopeName) {
+        return new ConfigurationFeatureProvider(
+                fullScopeName, featuresConf, features);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/feature/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/package-info.java
new file mode 100644
index 0000000..4abe31b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/feature/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Feature & FeatureProvider for distributedlog
+ */
+package org.apache.distributedlog.feature;
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
new file mode 100644
index 0000000..b1adf4a
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/function/CloseAsyncCloseableFunction.java
@@ -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.
+ */
+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
new file mode 100644
index 0000000..6360f2c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/function/DefaultValueMapFunction.java
@@ -0,0 +1,41 @@
+/**
+ * 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
new file mode 100644
index 0000000..1bf620c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetLastTxIdFunction.java
@@ -0,0 +1,43 @@
+/**
+ * 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.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 static final GetLastTxIdFunction INSTANCE = new GetLastTxIdFunction();
+
+    private GetLastTxIdFunction() {}
+
+    @Override
+    public Long apply(List<LogSegmentMetadata> segmentList) {
+        long lastTxId = DistributedLogConstants.INVALID_TXID;
+        for (LogSegmentMetadata l : segmentList) {
+            lastTxId = Math.max(lastTxId, l.getLastTxId());
+        }
+        return lastTxId;
+    }
+}
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
new file mode 100644
index 0000000..98164de
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/function/GetVersionedValueFunction.java
@@ -0,0 +1,39 @@
+/**
+ * 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
new file mode 100644
index 0000000..79f9c32
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/function/VoidFunctions.java
@@ -0,0 +1,34 @@
+/**
+ * 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/function/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/function/package-info.java
new file mode 100644
index 0000000..ed792d9
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/function/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Common Functions for DistributedLog
+ */
+package org.apache.distributedlog.function;
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
new file mode 100644
index 0000000..e132b64
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/BKNamespaceDriver.java
@@ -0,0 +1,631 @@
+/**
+ * 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.impl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.distributedlog.BookKeeperClient;
+import org.apache.distributedlog.BookKeeperClientBuilder;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.MetadataAccessor;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.acl.DefaultAccessControlManager;
+import org.apache.distributedlog.impl.acl.ZKAccessControlManager;
+import org.apache.distributedlog.bk.LedgerAllocator;
+import org.apache.distributedlog.bk.LedgerAllocatorUtils;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.InvalidStreamNameException;
+import org.apache.distributedlog.impl.federated.FederatedZKLogMetadataStore;
+import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
+import org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore;
+import org.apache.distributedlog.impl.subscription.ZKSubscriptionsStore;
+import org.apache.distributedlog.injector.AsyncFailureInjector;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.metadata.LogMetadataForReader;
+import org.apache.distributedlog.metadata.LogMetadataStore;
+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.util.OrderedScheduler;
+import org.apache.distributedlog.util.Utils;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.data.Stat;
+import org.jboss.netty.channel.socket.ClientSocketChannelFactory;
+import org.jboss.netty.channel.socket.nio.NioClientSocketChannelFactory;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.distributedlog.util.DLUtils.isReservedStreamName;
+import static org.apache.distributedlog.util.DLUtils.validateName;
+
+/**
+ * Manager for ZooKeeper/BookKeeper based namespace
+ */
+public class BKNamespaceDriver implements NamespaceDriver {
+
+    private static Logger LOG = LoggerFactory.getLogger(BKNamespaceDriver.class);
+
+    // register itself
+    static {
+        NamespaceDriverManager.registerDriver(DistributedLogConstants.BACKEND_BK, BKNamespaceDriver.class);
+    }
+
+    /**
+     * Extract zk servers fro dl <i>namespace</i>.
+     *
+     * @param uri dl namespace
+     * @return zk servers
+     */
+    public static String getZKServersFromDLUri(URI uri) {
+        return uri.getAuthority().replace(";", ",");
+    }
+
+    // resources (passed from initialization)
+    private DistributedLogConfiguration conf;
+    private DynamicDistributedLogConfiguration dynConf;
+    private URI namespace;
+    private OrderedScheduler scheduler;
+    private FeatureProvider featureProvider;
+    private AsyncFailureInjector failureInjector;
+    private StatsLogger statsLogger;
+    private StatsLogger perLogStatsLogger;
+    private String clientId;
+    private int regionId;
+
+    //
+    // resources (created internally and initialized at #initialize())
+    //
+
+    // namespace binding
+    private BKDLConfig bkdlConfig;
+
+    // zookeeper clients
+    // NOTE: The actual zookeeper client is initialized lazily when it is referenced by
+    //       {@link org.apache.distributedlog.ZooKeeperClient#get()}. So it is safe to
+    //       keep builders and their client wrappers here, as they will be used when
+    //       instantiating readers or writers.
+    private ZooKeeperClientBuilder sharedWriterZKCBuilder;
+    private ZooKeeperClient writerZKC;
+    private ZooKeeperClientBuilder sharedReaderZKCBuilder;
+    private ZooKeeperClient readerZKC;
+    // NOTE: The actual bookkeeper client is initialized lazily when it is referenced by
+    //       {@link org.apache.distributedlog.BookKeeperClient#get()}. So it is safe to
+    //       keep builders and their client wrappers here, as they will be used when
+    //       instantiating readers or writers.
+    private ClientSocketChannelFactory channelFactory;
+    private HashedWheelTimer requestTimer;
+    private BookKeeperClientBuilder sharedWriterBKCBuilder;
+    private BookKeeperClient writerBKC;
+    private BookKeeperClientBuilder sharedReaderBKCBuilder;
+    private BookKeeperClient readerBKC;
+
+    // log stream metadata store
+    private LogMetadataStore metadataStore;
+    private LogStreamMetadataStore writerStreamMetadataStore;
+    private LogStreamMetadataStore readerStreamMetadataStore;
+
+    //
+    // resources (lazily initialized)
+    //
+
+    // ledger allocator
+    private LedgerAllocator allocator;
+
+    // log segment entry stores
+    private LogSegmentEntryStore writerEntryStore;
+    private LogSegmentEntryStore readerEntryStore;
+
+    // access control manager
+    private AccessControlManager accessControlManager;
+
+    //
+    // states
+    //
+    protected boolean initialized = false;
+    protected AtomicBoolean closed = new AtomicBoolean(false);
+
+    /**
+     * Public constructor for reflection.
+     */
+    public BKNamespaceDriver() {
+    }
+
+    @Override
+    public synchronized NamespaceDriver initialize(DistributedLogConfiguration conf,
+                                                   DynamicDistributedLogConfiguration dynConf,
+                                                   URI namespace,
+                                                   OrderedScheduler scheduler,
+                                                   FeatureProvider featureProvider,
+                                                   AsyncFailureInjector failureInjector,
+                                                   StatsLogger statsLogger,
+                                                   StatsLogger perLogStatsLogger,
+                                                   String clientId,
+                                                   int regionId) throws IOException {
+        if (initialized) {
+            return this;
+        }
+        // validate the namespace
+        if ((null == namespace) || (null == namespace.getAuthority()) || (null == namespace.getPath())) {
+            throw new IOException("Incorrect distributedlog namespace : " + namespace);
+        }
+
+        // initialize the resources
+        this.conf = conf;
+        this.dynConf = dynConf;
+        this.namespace = namespace;
+        this.scheduler = scheduler;
+        this.featureProvider = featureProvider;
+        this.failureInjector = failureInjector;
+        this.statsLogger = statsLogger;
+        this.perLogStatsLogger = perLogStatsLogger;
+        this.clientId = clientId;
+        this.regionId = regionId;
+
+        // initialize the zookeeper clients
+        initializeZooKeeperClients();
+
+        // initialize the bookkeeper clients
+        initializeBookKeeperClients();
+
+        // propagate bkdlConfig to configuration
+        BKDLConfig.propagateConfiguration(bkdlConfig, conf);
+
+        // initialize the log metadata & stream metadata store
+        initializeLogStreamMetadataStores();
+
+        // initialize other resources
+        initializeOtherResources();
+
+        initialized = true;
+
+        LOG.info("Initialized BK namespace driver: clientId = {}, regionId = {}, federated = {}.",
+                new Object[]{clientId, regionId, bkdlConfig.isFederatedNamespace()});
+        return this;
+    }
+
+    private void initializeZooKeeperClients() throws IOException {
+        // Build the namespace zookeeper client
+        this.sharedWriterZKCBuilder = createZKClientBuilder(
+                String.format("dlzk:%s:factory_writer_shared", namespace),
+                conf,
+                getZKServersFromDLUri(namespace),
+                statsLogger.scope("dlzk_factory_writer_shared"));
+        this.writerZKC = sharedWriterZKCBuilder.build();
+
+        // Resolve namespace binding
+        this.bkdlConfig = BKDLConfig.resolveDLConfig(writerZKC, namespace);
+
+        // Build zookeeper client for readers
+        if (bkdlConfig.getDlZkServersForWriter().equals(bkdlConfig.getDlZkServersForReader())) {
+            this.sharedReaderZKCBuilder = this.sharedWriterZKCBuilder;
+        } else {
+            this.sharedReaderZKCBuilder = createZKClientBuilder(
+                    String.format("dlzk:%s:factory_reader_shared", namespace),
+                    conf,
+                    bkdlConfig.getDlZkServersForReader(),
+                    statsLogger.scope("dlzk_factory_reader_shared"));
+        }
+        this.readerZKC = this.sharedReaderZKCBuilder.build();
+    }
+
+    private synchronized BKDLConfig getBkdlConfig() {
+        return bkdlConfig;
+    }
+
+    private void initializeBookKeeperClients() throws IOException {
+        this.channelFactory = new NioClientSocketChannelFactory(
+                Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat("DL-netty-boss-%d").build()),
+                Executors.newCachedThreadPool(new ThreadFactoryBuilder().setNameFormat("DL-netty-worker-%d").build()),
+                conf.getBKClientNumberIOThreads());
+        this.requestTimer = new HashedWheelTimer(
+                new ThreadFactoryBuilder().setNameFormat("DLFactoryTimer-%d").build(),
+                conf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS,
+                conf.getTimeoutTimerNumTicks());
+        // Build bookkeeper client for writers
+        this.sharedWriterBKCBuilder = createBKCBuilder(
+                String.format("bk:%s:factory_writer_shared", namespace),
+                conf,
+                bkdlConfig.getBkZkServersForWriter(),
+                bkdlConfig.getBkLedgersPath(),
+                channelFactory,
+                requestTimer,
+                Optional.of(featureProvider.scope("bkc")),
+                statsLogger);
+        this.writerBKC = this.sharedWriterBKCBuilder.build();
+
+        // Build bookkeeper client for readers
+        if (bkdlConfig.getBkZkServersForWriter().equals(bkdlConfig.getBkZkServersForReader())) {
+            this.sharedReaderBKCBuilder = this.sharedWriterBKCBuilder;
+        } else {
+            this.sharedReaderBKCBuilder = createBKCBuilder(
+                    String.format("bk:%s:factory_reader_shared", namespace),
+                    conf,
+                    bkdlConfig.getBkZkServersForReader(),
+                    bkdlConfig.getBkLedgersPath(),
+                    channelFactory,
+                    requestTimer,
+                    Optional.<FeatureProvider>absent(),
+                    statsLogger);
+        }
+        this.readerBKC = this.sharedReaderBKCBuilder.build();
+    }
+
+    private void initializeLogStreamMetadataStores() throws IOException {
+        // log metadata store
+        if (bkdlConfig.isFederatedNamespace() || conf.isFederatedNamespaceEnabled()) {
+            this.metadataStore = new FederatedZKLogMetadataStore(conf, namespace, readerZKC, scheduler);
+        } else {
+            this.metadataStore = new ZKLogMetadataStore(conf, namespace, readerZKC, scheduler);
+        }
+
+        // create log stream metadata store
+        this.writerStreamMetadataStore =
+                new ZKLogStreamMetadataStore(
+                        clientId,
+                        conf,
+                        writerZKC,
+                        scheduler,
+                        statsLogger);
+        this.readerStreamMetadataStore =
+                new ZKLogStreamMetadataStore(
+                        clientId,
+                        conf,
+                        readerZKC,
+                        scheduler,
+                        statsLogger);
+    }
+
+    @VisibleForTesting
+    public static String validateAndGetFullLedgerAllocatorPoolPath(DistributedLogConfiguration conf, URI uri) throws IOException {
+        String poolPath = conf.getLedgerAllocatorPoolPath();
+        LOG.info("PoolPath is {}", poolPath);
+        if (null == poolPath || !poolPath.startsWith(".") || poolPath.endsWith("/")) {
+            LOG.error("Invalid ledger allocator pool path specified when enabling ledger allocator pool : {}", poolPath);
+            throw new IOException("Invalid ledger allocator pool path specified : " + poolPath);
+        }
+        String poolName = conf.getLedgerAllocatorPoolName();
+        if (null == poolName) {
+            LOG.error("No ledger allocator pool name specified when enabling ledger allocator pool.");
+            throw new IOException("No ledger allocator name specified when enabling ledger allocator pool.");
+        }
+        String rootPath = uri.getPath() + "/" + poolPath + "/" + poolName;
+        try {
+            PathUtils.validatePath(rootPath);
+        } catch (IllegalArgumentException iae) {
+            LOG.error("Invalid ledger allocator pool path specified when enabling ledger allocator pool : {}", poolPath);
+            throw new IOException("Invalid ledger allocator pool path specified : " + poolPath);
+        }
+        return rootPath;
+    }
+
+    private void initializeOtherResources() throws IOException {
+        // Ledger allocator
+        if (conf.getEnableLedgerAllocatorPool()) {
+            String allocatorPoolPath = validateAndGetFullLedgerAllocatorPoolPath(conf, namespace);
+            allocator = LedgerAllocatorUtils.createLedgerAllocatorPool(
+                    allocatorPoolPath,
+                    conf.getLedgerAllocatorPoolCoreSize(),
+                    conf,
+                    writerZKC,
+                    writerBKC,
+                    scheduler);
+            if (null != allocator) {
+                allocator.start();
+            }
+            LOG.info("Created ledger allocator pool under {} with size {}.", allocatorPoolPath, conf.getLedgerAllocatorPoolCoreSize());
+        } else {
+            allocator = null;
+        }
+
+    }
+
+    private void checkState() throws IOException {
+        if (closed.get()) {
+            LOG.error("BK namespace driver {} is already closed", namespace);
+            throw new AlreadyClosedException("BK namespace driver " + namespace + " is already closed");
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        if (!closed.compareAndSet(false, true)) {
+            return;
+        }
+        doClose();
+    }
+
+    private void doClose() {
+        if (null != accessControlManager) {
+            accessControlManager.close();
+            LOG.info("Access Control Manager Stopped.");
+        }
+
+        // Close the allocator
+        if (null != allocator) {
+            Utils.closeQuietly(allocator);
+            LOG.info("Ledger Allocator stopped.");
+        }
+
+        // Shutdown log segment metadata stores
+        Utils.close(writerStreamMetadataStore);
+        Utils.close(readerStreamMetadataStore);
+
+        writerBKC.close();
+        readerBKC.close();
+        writerZKC.close();
+        readerZKC.close();
+        // release bookkeeper resources
+        channelFactory.releaseExternalResources();
+        LOG.info("Release external resources used by channel factory.");
+        requestTimer.stop();
+        LOG.info("Stopped request timer");
+    }
+
+    @Override
+    public URI getUri() {
+        return namespace;
+    }
+
+    @Override
+    public String getScheme() {
+        return DistributedLogConstants.BACKEND_BK;
+    }
+
+    @Override
+    public LogMetadataStore getLogMetadataStore() {
+        return metadataStore;
+    }
+
+    @Override
+    public LogStreamMetadataStore getLogStreamMetadataStore(Role role) {
+        if (Role.WRITER == role) {
+            return writerStreamMetadataStore;
+        } else {
+            return readerStreamMetadataStore;
+        }
+    }
+
+    @Override
+    public LogSegmentEntryStore getLogSegmentEntryStore(Role role) {
+        if (Role.WRITER == role) {
+            return getWriterEntryStore();
+        } else {
+            return getReaderEntryStore();
+        }
+    }
+
+    private LogSegmentEntryStore getWriterEntryStore() {
+        if (null == writerEntryStore) {
+            writerEntryStore = new BKLogSegmentEntryStore(
+                    conf,
+                    dynConf,
+                    writerZKC,
+                    writerBKC,
+                    scheduler,
+                    allocator,
+                    statsLogger,
+                    failureInjector);
+        }
+        return writerEntryStore;
+    }
+
+    private LogSegmentEntryStore getReaderEntryStore() {
+        if (null == readerEntryStore) {
+            readerEntryStore = new BKLogSegmentEntryStore(
+                    conf,
+                    dynConf,
+                    writerZKC,
+                    readerBKC,
+                    scheduler,
+                    allocator,
+                    statsLogger,
+                    failureInjector);
+        }
+        return readerEntryStore;
+    }
+
+    @Override
+    public AccessControlManager getAccessControlManager() throws IOException {
+        if (null == accessControlManager) {
+            String aclRootPath = getBkdlConfig().getACLRootPath();
+            // Build the access control manager
+            if (aclRootPath == null) {
+                accessControlManager = DefaultAccessControlManager.INSTANCE;
+                LOG.info("Created default access control manager for {}", namespace);
+            } else {
+                if (!isReservedStreamName(aclRootPath)) {
+                    throw new IOException("Invalid Access Control List Root Path : " + aclRootPath);
+                }
+                String zkRootPath = namespace.getPath() + "/" + aclRootPath;
+                LOG.info("Creating zk based access control manager @ {} for {}",
+                        zkRootPath, namespace);
+                accessControlManager = new ZKAccessControlManager(conf, readerZKC,
+                        zkRootPath, scheduler);
+                LOG.info("Created zk based access control manager @ {} for {}",
+                        zkRootPath, namespace);
+            }
+        }
+        return accessControlManager;
+    }
+
+    @Override
+    public SubscriptionsStore getSubscriptionsStore(String streamName) {
+        return new ZKSubscriptionsStore(
+                writerZKC,
+                LogMetadataForReader.getSubscribersPath(namespace, streamName, conf.getUnpartitionedStreamName()));
+    }
+
+    //
+    // Legacy Intefaces
+    //
+
+    @Override
+    public MetadataAccessor getMetadataAccessor(String streamName)
+            throws InvalidStreamNameException, IOException {
+        if (getBkdlConfig().isFederatedNamespace()) {
+            throw new UnsupportedOperationException();
+        }
+        checkState();
+        validateName(streamName);
+        return new ZKMetadataAccessor(
+                streamName,
+                conf,
+                namespace,
+                sharedWriterZKCBuilder,
+                sharedReaderZKCBuilder,
+                statsLogger);
+    }
+
+    public Map<String, byte[]> enumerateLogsWithMetadataInNamespace()
+        throws IOException, IllegalArgumentException {
+        String namespaceRootPath = namespace.getPath();
+        HashMap<String, byte[]> result = new HashMap<String, byte[]>();
+        ZooKeeperClient zkc = writerZKC;
+        try {
+            ZooKeeper zk = Utils.sync(zkc, namespaceRootPath);
+            Stat currentStat = zk.exists(namespaceRootPath, false);
+            if (currentStat == null) {
+                return result;
+            }
+            List<String> children = zk.getChildren(namespaceRootPath, false);
+            for(String child: children) {
+                if (isReservedStreamName(child)) {
+                    continue;
+                }
+                String zkPath = String.format("%s/%s", namespaceRootPath, child);
+                currentStat = zk.exists(zkPath, false);
+                if (currentStat == null) {
+                    result.put(child, new byte[0]);
+                } else {
+                    result.put(child, zk.getData(zkPath, false, currentStat));
+                }
+            }
+        } catch (InterruptedException ie) {
+            LOG.error("Interrupted while deleting " + namespaceRootPath, ie);
+            throw new IOException("Interrupted while reading " + namespaceRootPath, ie);
+        } catch (KeeperException ke) {
+            LOG.error("Error reading" + namespaceRootPath + "entry in zookeeper", ke);
+            throw new IOException("Error reading" + namespaceRootPath + "entry in zookeeper", ke);
+        }
+        return result;
+    }
+
+    //
+    // Zk & Bk Utils
+    //
+
+    public static ZooKeeperClientBuilder createZKClientBuilder(String zkcName,
+                                                               DistributedLogConfiguration conf,
+                                                               String zkServers,
+                                                               StatsLogger statsLogger) {
+        RetryPolicy retryPolicy = null;
+        if (conf.getZKNumRetries() > 0) {
+            retryPolicy = new BoundExponentialBackoffRetryPolicy(
+                conf.getZKRetryBackoffStartMillis(),
+                conf.getZKRetryBackoffMaxMillis(), conf.getZKNumRetries());
+        }
+        ZooKeeperClientBuilder builder = ZooKeeperClientBuilder.newBuilder()
+            .name(zkcName)
+            .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+            .retryThreadCount(conf.getZKClientNumberRetryThreads())
+            .requestRateLimit(conf.getZKRequestRateLimit())
+            .zkServers(zkServers)
+            .retryPolicy(retryPolicy)
+            .statsLogger(statsLogger)
+            .zkAclId(conf.getZkAclId());
+        LOG.info("Created shared zooKeeper client builder {}: zkServers = {}, numRetries = {}, sessionTimeout = {}, retryBackoff = {},"
+                + " maxRetryBackoff = {}, zkAclId = {}.", new Object[] { zkcName, zkServers, conf.getZKNumRetries(),
+                conf.getZKSessionTimeoutMilliseconds(), conf.getZKRetryBackoffStartMillis(),
+                conf.getZKRetryBackoffMaxMillis(), conf.getZkAclId() });
+        return builder;
+    }
+
+    private BookKeeperClientBuilder createBKCBuilder(String bkcName,
+                                                     DistributedLogConfiguration conf,
+                                                     String zkServers,
+                                                     String ledgersPath,
+                                                     ClientSocketChannelFactory channelFactory,
+                                                     HashedWheelTimer requestTimer,
+                                                     Optional<FeatureProvider> featureProviderOptional,
+                                                     StatsLogger statsLogger) {
+        BookKeeperClientBuilder builder = BookKeeperClientBuilder.newBuilder()
+                .name(bkcName)
+                .dlConfig(conf)
+                .zkServers(zkServers)
+                .ledgersPath(ledgersPath)
+                .channelFactory(channelFactory)
+                .requestTimer(requestTimer)
+                .featureProvider(featureProviderOptional)
+                .statsLogger(statsLogger);
+        LOG.info("Created shared client builder {} : zkServers = {}, ledgersPath = {}, numIOThreads = {}",
+                new Object[] { bkcName, zkServers, ledgersPath, conf.getBKClientNumberIOThreads() });
+        return builder;
+    }
+
+    //
+    // Test Methods
+    //
+
+    @VisibleForTesting
+    public ZooKeeperClient getWriterZKC() {
+        return writerZKC;
+    }
+
+    @VisibleForTesting
+    public BookKeeperClient getReaderBKC() {
+        return readerBKC;
+    }
+
+    @VisibleForTesting
+    public AsyncFailureInjector getFailureInjector() {
+        return this.failureInjector;
+    }
+
+    @VisibleForTesting
+    public LogStreamMetadataStore getWriterStreamMetadataStore() {
+        return writerStreamMetadataStore;
+    }
+
+    @VisibleForTesting
+    public LedgerAllocator getLedgerAllocator() {
+        return allocator;
+    }
+}
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
new file mode 100644
index 0000000..0761cfc
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogMetadataStore.java
@@ -0,0 +1,124 @@
+/**
+ * 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.impl;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+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.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.*;
+
+/**
+ * ZooKeeper based log metadata store
+ */
+public class ZKLogMetadataStore implements LogMetadataStore {
+
+    final URI namespace;
+    final Optional<URI> nsOptional;
+    final ZooKeeperClient zkc;
+    final ZKNamespaceWatcher nsWatcher;
+
+    public ZKLogMetadataStore(
+            DistributedLogConfiguration conf,
+            URI namespace,
+            ZooKeeperClient zkc,
+            OrderedScheduler scheduler) {
+        this.namespace = namespace;
+        this.nsOptional = Optional.of(this.namespace);
+        this.zkc = zkc;
+        this.nsWatcher = new ZKNamespaceWatcher(conf, namespace, zkc, scheduler);
+    }
+
+    @Override
+    public Future<URI> createLog(String logName) {
+        return Future.value(namespace);
+    }
+
+    @Override
+    public Future<Optional<URI>> getLogLocation(String logName) {
+        return Future.value(nsOptional);
+    }
+
+    @Override
+    public Future<Iterator<String>> getLogs() {
+        final Promise<Iterator<String>> promise = new Promise<Iterator<String>>();
+        final String nsRootPath = namespace.getPath();
+        try {
+            final ZooKeeper zk = zkc.get();
+            zk.sync(nsRootPath, new AsyncCallback.VoidCallback() {
+                @Override
+                public void processResult(int syncRc, String syncPath, Object ctx) {
+                    if (KeeperException.Code.OK.intValue() == syncRc) {
+                        zk.getChildren(nsRootPath, false, new AsyncCallback.Children2Callback() {
+                            @Override
+                            public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+                                if (KeeperException.Code.OK.intValue() == rc) {
+                                    List<String> results = Lists.newArrayListWithExpectedSize(children.size());
+                                    for (String child : children) {
+                                        if (!isReservedStreamName(child)) {
+                                            results.add(child);
+                                        }
+                                    }
+                                    promise.setValue(results.iterator());
+                                } else if (KeeperException.Code.NONODE.intValue() == rc) {
+                                    List<String> streams = Lists.newLinkedList();
+                                    promise.setValue(streams.iterator());
+                                } else {
+                                    promise.setException(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());
+                    } else {
+                        promise.setException(new ZKException("Error reading namespace " + nsRootPath,
+                                KeeperException.Code.get(syncRc)));
+                    }
+                }
+            }, null);
+            zkc.get();
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            promise.setException(e);
+        } catch (InterruptedException e) {
+            promise.setException(e);
+        }
+        return promise;
+    }
+
+    @Override
+    public void registerNamespaceListener(NamespaceListener listener) {
+        this.nsWatcher.registerListener(listener);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentFilters.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentFilters.java
new file mode 100644
index 0000000..ca35c1c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentFilters.java
@@ -0,0 +1,89 @@
+/**
+ * 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.impl;
+
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.logsegment.LogSegmentFilter;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.List;
+
+/**
+ * Filters based on current zookeeper log segments.
+ */
+public class ZKLogSegmentFilters {
+
+    static final Logger LOG = LoggerFactory.getLogger(ZKLogSegmentFilters.class);
+
+    /**
+     * Write handler filter should return all inprogress log segments and the last completed log segment.
+     * Because sequence id & ledger sequence number assignment rely on previous log segments.
+     */
+    public static final LogSegmentFilter WRITE_HANDLE_FILTER = new LogSegmentFilter() {
+        @Override
+        public Collection<String> filter(Collection<String> fullList) {
+            List<String> result = new ArrayList<String>(fullList.size());
+            String lastCompletedLogSegmentName = null;
+            long lastLogSegmentSequenceNumber = -1L;
+            for (String s : fullList) {
+                if (s.startsWith(DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX)) {
+                    result.add(s);
+                } else if (s.startsWith(DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX)) {
+                    String[] parts = s.split("_");
+                    try {
+                        if (2 == parts.length) {
+                            // name: logrecs_<logsegment_sequence_number>
+                            long logSegmentSequenceNumber = Long.parseLong(parts[1]);
+                            if (logSegmentSequenceNumber > lastLogSegmentSequenceNumber) {
+                                lastLogSegmentSequenceNumber = logSegmentSequenceNumber;
+                                lastCompletedLogSegmentName = s;
+                            }
+                        } else if (6 == parts.length) {
+                            // name: logrecs_<start_tx_id>_<end_tx_id>_<logsegment_sequence_number>_<ledger_id>_<region_id>
+                            long logSegmentSequenceNumber = Long.parseLong(parts[3]);
+                            if (logSegmentSequenceNumber > lastLogSegmentSequenceNumber) {
+                                lastLogSegmentSequenceNumber = logSegmentSequenceNumber;
+                                lastCompletedLogSegmentName = s;
+                            }
+                        } else {
+                            // name: logrecs_<start_tx_id>_<end_tx_id> or any unknown names
+                            // we don't know the ledger sequence from the name, so add it to the list
+                            result.add(s);
+                        }
+                    } catch (NumberFormatException nfe) {
+                        LOG.warn("Unexpected sequence number in log segment {} :", s, nfe);
+                        result.add(s);
+                    }
+                } else {
+                    LOG.error("Unknown log segment name : {}", s);
+                }
+            }
+            if (null != lastCompletedLogSegmentName) {
+                result.add(lastCompletedLogSegmentName);
+            }
+            if (LOG.isTraceEnabled()) {
+                LOG.trace("Filtered log segments {} from {}.", result, fullList);
+            }
+            return result;
+        }
+    };
+
+}
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
new file mode 100644
index 0000000..b9cb374
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKLogSegmentMetadataStore.java
@@ -0,0 +1,503 @@
+/**
+ * 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.impl;
+
+import com.google.common.collect.ImmutableList;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.callback.LogSegmentNamesListener;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.exceptions.LogSegmentNotFoundException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.metadata.LogMetadata;
+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.util.OrderedScheduler;
+import org.apache.distributedlog.util.Transaction;
+import org.apache.distributedlog.util.Transaction.OpListener;
+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;
+import org.apache.zookeeper.AsyncCallback.Children2Callback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * ZooKeeper based log segment metadata store.
+ */
+public class ZKLogSegmentMetadataStore implements LogSegmentMetadataStore, Watcher, Children2Callback {
+
+    private static final Logger logger = LoggerFactory.getLogger(ZKLogSegmentMetadataStore.class);
+
+    private static final List<String> EMPTY_LIST = ImmutableList.of();
+
+    private static class ReadLogSegmentsTask implements Runnable, FutureEventListener<Versioned<List<String>>> {
+
+        private final String logSegmentsPath;
+        private final ZKLogSegmentMetadataStore store;
+        private int currentZKBackOffMs;
+
+        ReadLogSegmentsTask(String logSegmentsPath,
+                            ZKLogSegmentMetadataStore metadataStore) {
+            this.logSegmentsPath = logSegmentsPath;
+            this.store = metadataStore;
+            this.currentZKBackOffMs = store.minZKBackoffMs;
+        }
+
+        @Override
+        public void onSuccess(final Versioned<List<String>> segments) {
+            // reset the back off after a successful operation
+            currentZKBackOffMs = store.minZKBackoffMs;
+            store.notifyLogSegmentsUpdated(
+                    logSegmentsPath,
+                    store.listeners.get(logSegmentsPath),
+                    segments);
+        }
+
+        @Override
+        public void onFailure(Throwable cause) {
+            int backoffMs;
+            if (cause instanceof LogNotFoundException) {
+                // the log segment has been deleted, remove all the registered listeners
+                store.notifyLogStreamDeleted(logSegmentsPath,
+                        store.listeners.remove(logSegmentsPath));
+                return;
+            } else {
+                backoffMs = currentZKBackOffMs;
+                currentZKBackOffMs = Math.min(2 * currentZKBackOffMs, store.maxZKBackoffMs);
+            }
+            store.scheduleTask(logSegmentsPath, this, backoffMs);
+        }
+
+        @Override
+        public void run() {
+            if (null != store.listeners.get(logSegmentsPath)) {
+                store.zkGetLogSegmentNames(logSegmentsPath, store).addEventListener(this);
+            } else {
+                logger.debug("Log segments listener for {} has been removed.", logSegmentsPath);
+            }
+        }
+    }
+
+    /**
+     * A log segment names listener that keeps tracking the version of list of log segments that it has been notified.
+     * It only notify the newer log segments.
+     */
+    static class VersionedLogSegmentNamesListener {
+
+        private final LogSegmentNamesListener listener;
+        private Versioned<List<String>> lastNotifiedLogSegments;
+
+        VersionedLogSegmentNamesListener(LogSegmentNamesListener listener) {
+            this.listener = listener;
+            this.lastNotifiedLogSegments = new Versioned<List<String>>(EMPTY_LIST, Version.NEW);
+        }
+
+        synchronized void onSegmentsUpdated(Versioned<List<String>> logSegments) {
+            if (lastNotifiedLogSegments.getVersion() == Version.NEW ||
+                    lastNotifiedLogSegments.getVersion().compare(logSegments.getVersion()) == Version.Occurred.BEFORE) {
+                lastNotifiedLogSegments = logSegments;
+                listener.onSegmentsUpdated(logSegments);
+            }
+        }
+
+        @Override
+        public int hashCode() {
+            return listener.hashCode();
+        }
+
+        @Override
+        public boolean equals(Object obj) {
+            if (!(obj instanceof VersionedLogSegmentNamesListener)) {
+                return false;
+            }
+            VersionedLogSegmentNamesListener other = (VersionedLogSegmentNamesListener) obj;
+            return listener.equals(other.listener);
+        }
+
+        @Override
+        public String toString() {
+            return listener.toString();
+        }
+    }
+
+    final DistributedLogConfiguration conf;
+    // settings
+    final int minZKBackoffMs;
+    final int maxZKBackoffMs;
+    final boolean skipMinVersionCheck;
+
+    final ZooKeeperClient zkc;
+    // log segment listeners
+    final ConcurrentMap<String, Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener>> listeners;
+    // scheduler
+    final OrderedScheduler scheduler;
+    final ReentrantReadWriteLock closeLock;
+    boolean closed = false;
+
+    public ZKLogSegmentMetadataStore(DistributedLogConfiguration conf,
+                                     ZooKeeperClient zkc,
+                                     OrderedScheduler scheduler) {
+        this.conf = conf;
+        this.zkc = zkc;
+        this.listeners =
+                new ConcurrentHashMap<String, Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener>>();
+        this.scheduler = scheduler;
+        this.closeLock = new ReentrantReadWriteLock();
+        // settings
+        this.minZKBackoffMs = conf.getZKRetryBackoffStartMillis();
+        this.maxZKBackoffMs = conf.getZKRetryBackoffMaxMillis();
+        this.skipMinVersionCheck = conf.getDLLedgerMetadataSkipMinVersionCheck();
+    }
+
+    protected void scheduleTask(Object key, Runnable r, long delayMs) {
+        closeLock.readLock().lock();
+        try {
+            if (closed) {
+                return;
+            }
+            scheduler.schedule(key, r, delayMs, TimeUnit.MILLISECONDS);
+        } finally {
+            closeLock.readLock().unlock();
+        }
+    }
+
+    protected void submitTask(Object key, Runnable r) {
+        closeLock.readLock().lock();
+        try {
+            if (closed) {
+                return;
+            }
+            scheduler.submit(key, r);
+        } finally {
+            closeLock.readLock().unlock();
+        }
+    }
+
+    // max sequence number and max transaction id
+
+    @Override
+    public void storeMaxLogSegmentSequenceNumber(Transaction<Object> txn,
+                                                 LogMetadata logMetadata,
+                                                 Versioned<Long> lssn,
+                                                 Transaction.OpListener<Version> listener) {
+        Version version = lssn.getVersion();
+        assert(version instanceof ZkVersion);
+        ZkVersion zkVersion = (ZkVersion) version;
+        byte[] data = DLUtils.serializeLogSegmentSequenceNumber(lssn.getValue());
+        Op setDataOp = Op.setData(logMetadata.getLogSegmentsPath(), data, zkVersion.getZnodeVersion());
+        ZKOp zkOp = new ZKVersionedSetOp(setDataOp, listener);
+        txn.addOp(zkOp);
+    }
+
+    @Override
+    public void storeMaxTxnId(Transaction<Object> txn,
+                              LogMetadataForWriter logMetadata,
+                              Versioned<Long> transactionId,
+                              Transaction.OpListener<Version> listener) {
+        Version version = transactionId.getVersion();
+        assert(version instanceof ZkVersion);
+        ZkVersion zkVersion = (ZkVersion) version;
+        byte[] data = DLUtils.serializeTransactionId(transactionId.getValue());
+        Op setDataOp = Op.setData(logMetadata.getMaxTxIdPath(), data, zkVersion.getZnodeVersion());
+        ZKOp zkOp = new ZKVersionedSetOp(setDataOp, listener);
+        txn.addOp(zkOp);
+    }
+
+    // updates
+
+    @Override
+    public Transaction<Object> transaction() {
+        return new ZKTransaction(zkc);
+    }
+
+    @Override
+    public void createLogSegment(Transaction<Object> txn,
+                                 LogSegmentMetadata segment,
+                                 OpListener<Void> listener) {
+        byte[] finalisedData = segment.getFinalisedData().getBytes(UTF_8);
+        Op createOp = Op.create(
+                segment.getZkPath(),
+                finalisedData,
+                zkc.getDefaultACL(),
+                CreateMode.PERSISTENT);
+        txn.addOp(DefaultZKOp.of(createOp, listener));
+    }
+
+    @Override
+    public void deleteLogSegment(Transaction<Object> txn,
+                                 final LogSegmentMetadata segment,
+                                 final OpListener<Void> listener) {
+        Op deleteOp = Op.delete(
+                segment.getZkPath(),
+                -1);
+        logger.info("Delete segment : {}", segment);
+        txn.addOp(DefaultZKOp.of(deleteOp, new OpListener<Void>() {
+            @Override
+            public void onCommit(Void r) {
+                if (null != listener) {
+                    listener.onCommit(r);
+                }
+            }
+
+            @Override
+            public void onAbort(Throwable t) {
+                logger.info("Aborted transaction on deleting segment {}", segment);
+                KeeperException.Code kc;
+                if (t instanceof KeeperException) {
+                    kc = ((KeeperException) t).code();
+                } else if (t instanceof ZKException) {
+                    kc = ((ZKException) t).getKeeperExceptionCode();
+                } else {
+                    abortListener(t);
+                    return;
+                }
+                if (KeeperException.Code.NONODE == kc) {
+                    abortListener(new LogSegmentNotFoundException(segment.getZkPath()));
+                    return;
+                }
+                abortListener(t);
+            }
+
+            private void abortListener(Throwable t) {
+                if (null != listener) {
+                    listener.onAbort(t);
+                }
+            }
+        }));
+    }
+
+    @Override
+    public void updateLogSegment(Transaction<Object> txn, LogSegmentMetadata segment) {
+        byte[] finalisedData = segment.getFinalisedData().getBytes(UTF_8);
+        Op updateOp = Op.setData(segment.getZkPath(), finalisedData, -1);
+        txn.addOp(DefaultZKOp.of(updateOp, null));
+    }
+
+    // reads
+
+    /**
+     * Process the watched events for registered listeners
+     */
+    @Override
+    public void process(WatchedEvent event) {
+        if (Event.EventType.None == event.getType()
+                && Event.KeeperState.Expired == event.getState()) {
+            Set<String> keySet = new HashSet<String>(listeners.keySet());
+            for (String logSegmentsPath : keySet) {
+                scheduleTask(logSegmentsPath, new ReadLogSegmentsTask(logSegmentsPath, this), 0L);
+            }
+            return;
+        }
+        String path = event.getPath();
+        if (null == path) {
+            return;
+        }
+        switch (event.getType()) {
+            case NodeDeleted:
+                notifyLogStreamDeleted(path, listeners.remove(path));
+                break;
+            case NodeChildrenChanged:
+                new ReadLogSegmentsTask(path, this).run();
+                break;
+            default:
+                break;
+        }
+    }
+
+    @Override
+    public Future<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>>>();
+        try {
+            zkc.get().getChildren(logSegmentsPath, watcher, this, result);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            result.setException(FutureUtils.zkException(e, logSegmentsPath));
+        } catch (InterruptedException e) {
+            result.setException(FutureUtils.zkException(e, logSegmentsPath));
+        }
+        return result;
+    }
+
+    @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);
+        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));
+        } else if (KeeperException.Code.NONODE.intValue() == rc) {
+            result.setException(new LogNotFoundException("Log " + path + " not found"));
+        } else {
+            result.setException(new ZKException("Failed to get log segments from " + path,
+                    KeeperException.Code.get(rc)));
+        }
+    }
+
+    @Override
+    public Future<Versioned<List<String>>> getLogSegmentNames(String logSegmentsPath,
+                                                              LogSegmentNamesListener listener) {
+        Watcher zkWatcher;
+        if (null == listener) {
+            zkWatcher = null;
+        } else {
+            closeLock.readLock().lock();
+            try {
+                if (closed) {
+                    zkWatcher = null;
+                } else {
+                    Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> listenerSet =
+                            listeners.get(logSegmentsPath);
+                    if (null == listenerSet) {
+                        Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> newListenerSet =
+                                new HashMap<LogSegmentNamesListener, VersionedLogSegmentNamesListener>();
+                        Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> oldListenerSet =
+                                listeners.putIfAbsent(logSegmentsPath, newListenerSet);
+                        if (null != oldListenerSet) {
+                            listenerSet = oldListenerSet;
+                        } else {
+                            listenerSet = newListenerSet;
+                        }
+                    }
+                    synchronized (listenerSet) {
+                        listenerSet.put(listener, new VersionedLogSegmentNamesListener(listener));
+                        if (!listeners.containsKey(logSegmentsPath)) {
+                            // listener set has been removed, add it back
+                            if (null != listeners.putIfAbsent(logSegmentsPath, listenerSet)) {
+                                logger.debug("Listener set is already found for log segments path {}", logSegmentsPath);
+                            }
+                        }
+                    }
+                    zkWatcher = ZKLogSegmentMetadataStore.this;
+                }
+            } finally {
+                closeLock.readLock().unlock();
+            }
+        }
+        Future<Versioned<List<String>>> getLogSegmentNamesResult = zkGetLogSegmentNames(logSegmentsPath, zkWatcher);
+        if (null != listener) {
+            getLogSegmentNamesResult.addEventListener(new ReadLogSegmentsTask(logSegmentsPath, this));
+        }
+        return zkGetLogSegmentNames(logSegmentsPath, zkWatcher);
+    }
+
+    @Override
+    public void unregisterLogSegmentListener(String logSegmentsPath,
+                                             LogSegmentNamesListener listener) {
+        closeLock.readLock().lock();
+        try {
+            if (closed) {
+                return;
+            }
+            Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> listenerSet =
+                    listeners.get(logSegmentsPath);
+            if (null == listenerSet) {
+                return;
+            }
+            synchronized (listenerSet) {
+                listenerSet.remove(listener);
+                if (listenerSet.isEmpty()) {
+                    listeners.remove(logSegmentsPath, listenerSet);
+                }
+            }
+        } finally {
+            closeLock.readLock().unlock();
+        }
+    }
+
+    @Override
+    public void close() throws IOException {
+        closeLock.writeLock().lock();
+        try {
+            if (closed) {
+                return;
+            }
+            closed = true;
+        } finally {
+            closeLock.writeLock().unlock();
+        }
+    }
+
+    // Notifications
+
+    void notifyLogStreamDeleted(String logSegmentsPath,
+                                final Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> listeners) {
+        if (null == listeners) {
+            return;
+        }
+        this.submitTask(logSegmentsPath, new Runnable() {
+            @Override
+            public void run() {
+                for (LogSegmentNamesListener listener : listeners.keySet()) {
+                    listener.onLogStreamDeleted();
+                }
+            }
+        });
+
+    }
+
+    void notifyLogSegmentsUpdated(String logSegmentsPath,
+                                  final Map<LogSegmentNamesListener, VersionedLogSegmentNamesListener> listeners,
+                                  final Versioned<List<String>> segments) {
+        if (null == listeners) {
+            return;
+        }
+        this.submitTask(logSegmentsPath, new Runnable() {
+            @Override
+            public void run() {
+                for (VersionedLogSegmentNamesListener listener : listeners.values()) {
+                    listener.onSegmentsUpdated(segments);
+                }
+            }
+        });
+    }
+
+}
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
new file mode 100644
index 0000000..551cc44
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKMetadataAccessor.java
@@ -0,0 +1,264 @@
+/**
+ * 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.impl;
+
+import java.io.IOException;
+import java.net.URI;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.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.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;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.distributedlog.impl.BKNamespaceDriver.getZKServersFromDLUri;
+
+public class ZKMetadataAccessor implements MetadataAccessor {
+    static final Logger LOG = LoggerFactory.getLogger(ZKMetadataAccessor.class);
+    protected final String name;
+    protected Promise<Void> closePromise;
+    protected final URI uri;
+    // zookeeper clients
+    // NOTE: The actual zookeeper client is initialized lazily when it is referenced by
+    //       {@link org.apache.distributedlog.ZooKeeperClient#get()}. So it is safe to
+    //       keep builders and their client wrappers here, as they will be used when
+    //       instantiating readers or writers.
+    protected final ZooKeeperClientBuilder writerZKCBuilder;
+    protected final ZooKeeperClient writerZKC;
+    protected final boolean ownWriterZKC;
+    protected final ZooKeeperClientBuilder readerZKCBuilder;
+    protected final ZooKeeperClient readerZKC;
+    protected final boolean ownReaderZKC;
+
+    ZKMetadataAccessor(String name,
+                       DistributedLogConfiguration conf,
+                       URI uri,
+                       ZooKeeperClientBuilder writerZKCBuilder,
+                       ZooKeeperClientBuilder readerZKCBuilder,
+                       StatsLogger statsLogger) {
+        this.name = name;
+        this.uri = uri;
+
+        if (null == writerZKCBuilder) {
+            RetryPolicy retryPolicy = null;
+            if (conf.getZKNumRetries() > 0) {
+                retryPolicy = new BoundExponentialBackoffRetryPolicy(
+                    conf.getZKRetryBackoffStartMillis(),
+                    conf.getZKRetryBackoffMaxMillis(), conf.getZKNumRetries());
+            }
+            this.writerZKCBuilder = ZooKeeperClientBuilder.newBuilder()
+                    .name(String.format("dlzk:%s:dlm_writer_shared", name))
+                    .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                    .retryThreadCount(conf.getZKClientNumberRetryThreads())
+                    .requestRateLimit(conf.getZKRequestRateLimit())
+                    .zkAclId(conf.getZkAclId())
+                    .uri(uri)
+                    .retryPolicy(retryPolicy)
+                    .statsLogger(statsLogger.scope("dlzk_dlm_writer_shared"));
+            this.ownWriterZKC = true;
+        } else {
+            this.writerZKCBuilder = writerZKCBuilder;
+            this.ownWriterZKC = false;
+        }
+        this.writerZKC = this.writerZKCBuilder.build();
+
+        if (null == readerZKCBuilder) {
+            String zkServersForWriter = getZKServersFromDLUri(uri);
+            String zkServersForReader;
+            try {
+                BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(this.writerZKC, uri);
+                zkServersForReader = bkdlConfig.getDlZkServersForReader();
+            } catch (IOException e) {
+                LOG.warn("Error on resolving dl metadata bindings for {} : ", uri, e);
+                zkServersForReader = zkServersForWriter;
+            }
+            if (zkServersForReader.equals(zkServersForWriter)) {
+                LOG.info("Used same zookeeper servers '{}' for both writers and readers for {}.",
+                         zkServersForWriter, name);
+                this.readerZKCBuilder = this.writerZKCBuilder;
+                this.ownReaderZKC = false;
+            } else {
+                RetryPolicy retryPolicy = null;
+                if (conf.getZKNumRetries() > 0) {
+                    retryPolicy = new BoundExponentialBackoffRetryPolicy(
+                        conf.getZKRetryBackoffStartMillis(),
+                        conf.getZKRetryBackoffMaxMillis(), conf.getZKNumRetries());
+                }
+                this.readerZKCBuilder = ZooKeeperClientBuilder.newBuilder()
+                        .name(String.format("dlzk:%s:dlm_reader_shared", name))
+                        .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                        .retryThreadCount(conf.getZKClientNumberRetryThreads())
+                        .requestRateLimit(conf.getZKRequestRateLimit())
+                        .zkServers(zkServersForReader)
+                        .retryPolicy(retryPolicy)
+                        .zkAclId(conf.getZkAclId())
+                        .statsLogger(statsLogger.scope("dlzk_dlm_reader_shared"));
+                this.ownReaderZKC = true;
+            }
+        } else {
+            this.readerZKCBuilder = readerZKCBuilder;
+            this.ownReaderZKC = false;
+        }
+        this.readerZKC = this.readerZKCBuilder.build();
+    }
+
+    /**
+     * Get the name of the stream managed by this log manager
+     *
+     * @return streamName
+     */
+    @Override
+    public String getStreamName() {
+        return name;
+    }
+
+    /**
+     * Creates or update the metadata stored at the node associated with the
+     * name and URI
+     * @param metadata opaque metadata to be stored for the node
+     * @throws IOException
+     */
+    @Override
+    public void createOrUpdateMetadata(byte[] metadata) throws IOException {
+        checkClosedOrInError("createOrUpdateMetadata");
+
+        String zkPath = getZKPath();
+        LOG.debug("Setting application specific metadata on {}", zkPath);
+        try {
+            Stat currentStat = writerZKC.get().exists(zkPath, false);
+            if (currentStat == null) {
+                if (metadata.length > 0) {
+                    Utils.zkCreateFullPathOptimistic(writerZKC,
+                            zkPath,
+                            metadata,
+                            writerZKC.getDefaultACL(),
+                            CreateMode.PERSISTENT);
+                }
+            } else {
+                writerZKC.get().setData(zkPath, metadata, currentStat.getVersion());
+            }
+        } catch (InterruptedException ie) {
+            throw new DLInterruptedException("Interrupted on creating or updating container metadata", ie);
+        } catch (Exception exc) {
+            throw new IOException("Exception creating or updating container metadata", exc);
+        }
+    }
+
+    /**
+     * Delete the metadata stored at the associated node. This only deletes the metadata
+     * and not the node itself
+     * @throws IOException
+     */
+    @Override
+    public void deleteMetadata() throws IOException {
+        checkClosedOrInError("createOrUpdateMetadata");
+        createOrUpdateMetadata(null);
+    }
+
+    /**
+     * Retrieve the metadata stored at the node
+     * @return byte array containing the metadata
+     * @throws IOException
+     */
+    @Override
+    public byte[] getMetadata() throws IOException {
+        checkClosedOrInError("createOrUpdateMetadata");
+        String zkPath = getZKPath();
+        LOG.debug("Getting application specific metadata from {}", zkPath);
+        try {
+            Stat currentStat = readerZKC.get().exists(zkPath, false);
+            if (currentStat == null) {
+                return null;
+            } else {
+                return readerZKC.get().getData(zkPath, false, currentStat);
+            }
+        } catch (InterruptedException ie) {
+            throw new DLInterruptedException("Error reading the max tx id from zk", ie);
+        } catch (Exception e) {
+            throw new IOException("Error reading the max tx id from zk", e);
+        }
+    }
+
+    /**
+     * Close the metadata accessor, freeing any resources it may hold.
+     * @return future represents the close result.
+     */
+    @Override
+    public Future<Void> asyncClose() {
+        Promise<Void> closeFuture;
+        synchronized (this) {
+            if (null != closePromise) {
+                return closePromise;
+            }
+            closeFuture = closePromise = new Promise<Void>();
+        }
+        // NOTE: ownWriterZKC and ownReaderZKC are mostly used by tests
+        //       the managers created by the namespace - whose zkc will be closed by namespace
+        try {
+            if (ownWriterZKC) {
+                writerZKC.close();
+            }
+            if (ownReaderZKC) {
+                readerZKC.close();
+            }
+        } catch (Exception e) {
+            LOG.warn("Exception while closing distributed log manager", e);
+        }
+        FutureUtils.setValue(closeFuture, null);
+        return closeFuture;
+    }
+
+    @Override
+    public void close() throws IOException {
+        FutureUtils.result(asyncClose());
+    }
+
+    public synchronized void checkClosedOrInError(String operation) throws AlreadyClosedException {
+        if (null != closePromise) {
+            throw new AlreadyClosedException("Executing " + operation + " on already closed ZKMetadataAccessor");
+        }
+    }
+
+    protected String getZKPath() {
+        return String.format("%s/%s", uri.getPath(), name);
+    }
+
+    @VisibleForTesting
+    protected ZooKeeperClient getReaderZKC() {
+        return readerZKC;
+    }
+
+    @VisibleForTesting
+    protected ZooKeeperClient getWriterZKC() {
+        return writerZKC;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java
new file mode 100644
index 0000000..621cb2d
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/ZKNamespaceWatcher.java
@@ -0,0 +1,133 @@
+/**
+ * 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.impl;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.callback.NamespaceListener;
+import org.apache.distributedlog.namespace.NamespaceWatcher;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static org.apache.distributedlog.util.DLUtils.*;
+
+/**
+ * Watcher on watching a given namespace
+ */
+public class ZKNamespaceWatcher extends NamespaceWatcher
+        implements Runnable, Watcher, AsyncCallback.Children2Callback {
+
+    static final Logger logger = LoggerFactory.getLogger(ZKNamespaceWatcher.class);
+
+    private final DistributedLogConfiguration conf;
+    private final URI uri;
+    private final ZooKeeperClient zkc;
+    private final OrderedScheduler scheduler;
+    private final AtomicBoolean namespaceWatcherSet = new AtomicBoolean(false);
+
+    public ZKNamespaceWatcher(DistributedLogConfiguration conf,
+                              URI uri,
+                              ZooKeeperClient zkc,
+                              OrderedScheduler scheduler) {
+        this.conf = conf;
+        this.uri = uri;
+        this.zkc = zkc;
+        this.scheduler = scheduler;
+    }
+
+    private void scheduleTask(Runnable r, long ms) {
+        try {
+            scheduler.schedule(r, ms, TimeUnit.MILLISECONDS);
+        } catch (RejectedExecutionException ree) {
+            logger.error("Task {} scheduled in {} ms is rejected : ", new Object[]{r, ms, ree});
+        }
+    }
+
+    @Override
+    public void run() {
+        try {
+            doWatchNamespaceChanges();
+        } catch (Exception e) {
+            logger.error("Encountered unknown exception on watching namespace {} ", uri, e);
+        }
+    }
+
+    public void watchNamespaceChanges() {
+        if (!namespaceWatcherSet.compareAndSet(false, true)) {
+            return;
+        }
+        doWatchNamespaceChanges();
+    }
+
+    private void doWatchNamespaceChanges() {
+        try {
+            zkc.get().getChildren(uri.getPath(), this, this, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
+        } catch (InterruptedException e) {
+            logger.warn("Interrupted on watching namespace changes for {} : ", uri, e);
+            scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
+        }
+    }
+
+    @Override
+    public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+        if (KeeperException.Code.OK.intValue() == rc) {
+            logger.info("Received updated logs under {} : {}", uri, children);
+            List<String> result = new ArrayList<String>(children.size());
+            for (String s : children) {
+                if (isReservedStreamName(s)) {
+                    continue;
+                }
+                result.add(s);
+            }
+            for (NamespaceListener listener : listeners) {
+                listener.onStreamsChanged(result.iterator());
+            }
+        } else {
+            scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
+        }
+    }
+
+    @Override
+    public void process(WatchedEvent event) {
+        if (event.getType() == Event.EventType.None) {
+            if (event.getState() == Event.KeeperState.Expired) {
+                scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
+            }
+            return;
+        }
+        if (event.getType() == Event.EventType.NodeChildrenChanged) {
+            // watch namespace changes again.
+            doWatchNamespaceChanges();
+        }
+    }
+}
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
new file mode 100644
index 0000000..63a81bd
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControl.java
@@ -0,0 +1,232 @@
+/**
+ * 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.impl.acl;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+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;
+import org.apache.thrift.transport.TMemoryInputTransport;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.data.Stat;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+public class ZKAccessControl {
+
+    private static final int BUFFER_SIZE = 4096;
+
+    public static final AccessControlEntry DEFAULT_ACCESS_CONTROL_ENTRY = new AccessControlEntry();
+
+    public static class CorruptedAccessControlException extends IOException {
+
+        private static final long serialVersionUID = 5391285182476211603L;
+
+        public CorruptedAccessControlException(String zkPath, Throwable t) {
+            super("Access Control @ " + zkPath + " is corrupted.", t);
+        }
+    }
+
+    protected final AccessControlEntry accessControlEntry;
+    protected final String zkPath;
+    private int zkVersion;
+
+    public ZKAccessControl(AccessControlEntry ace, String zkPath) {
+        this(ace, zkPath, -1);
+    }
+
+    private ZKAccessControl(AccessControlEntry ace, String zkPath, int zkVersion) {
+        this.accessControlEntry = ace;
+        this.zkPath = zkPath;
+        this.zkVersion = zkVersion;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hashCode(zkPath, accessControlEntry);
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (!(obj instanceof ZKAccessControl)) {
+            return false;
+        }
+        ZKAccessControl other = (ZKAccessControl) obj;
+        return Objects.equal(zkPath, other.zkPath) &&
+                Objects.equal(accessControlEntry, other.accessControlEntry);
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("entry(path=").append(zkPath).append(", acl=")
+                .append(accessControlEntry).append(")");
+        return sb.toString();
+    }
+
+    @VisibleForTesting
+    public String getZKPath() {
+        return zkPath;
+    }
+
+    @VisibleForTesting
+    public AccessControlEntry getAccessControlEntry() {
+        return accessControlEntry;
+    }
+
+    public Future<ZKAccessControl> create(ZooKeeperClient zkc) {
+        final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
+        try {
+            zkc.get().create(zkPath, serialize(accessControlEntry), zkc.getDefaultACL(), CreateMode.PERSISTENT,
+                    new AsyncCallback.StringCallback() {
+                        @Override
+                        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);
+                            } else {
+                                promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                            }
+                        }
+                    }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            promise.setException(e);
+        } catch (InterruptedException e) {
+            promise.setException(e);
+        } catch (IOException e) {
+            promise.setException(e);
+        }
+        return promise;
+    }
+
+    public Future<ZKAccessControl> update(ZooKeeperClient zkc) {
+        final Promise<ZKAccessControl> promise = new Promise<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);
+                    } else {
+                        promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                    }
+                }
+            }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            promise.setException(e);
+        } catch (InterruptedException e) {
+            promise.setException(e);
+        } catch (IOException e) {
+            promise.setException(e);
+        }
+        return promise;
+    }
+
+    public static Future<ZKAccessControl> read(final ZooKeeperClient zkc, final String zkPath, Watcher watcher) {
+        final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
+
+        try {
+            zkc.get().getData(zkPath, watcher, new AsyncCallback.DataCallback() {
+                @Override
+                public void processResult(int rc, String path, Object ctx, byte[] data, Stat stat) {
+                    if (KeeperException.Code.OK.intValue() == rc) {
+                        try {
+                            AccessControlEntry ace = deserialize(zkPath, data);
+                            promise.setValue(new ZKAccessControl(ace, zkPath, stat.getVersion()));
+                        } catch (IOException ioe) {
+                            promise.setException(ioe);
+                        }
+                    } else {
+                        promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                    }
+                }
+            }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            promise.setException(e);
+        } catch (InterruptedException e) {
+            promise.setException(e);
+        }
+        return promise;
+    }
+
+    public static Future<Void> delete(final ZooKeeperClient zkc, final String zkPath) {
+        final Promise<Void> promise = new Promise<Void>();
+
+        try {
+            zkc.get().delete(zkPath, -1, new AsyncCallback.VoidCallback() {
+                @Override
+                public void processResult(int rc, String path, Object ctx) {
+                    if (KeeperException.Code.OK.intValue() == rc ||
+                            KeeperException.Code.NONODE.intValue() == rc) {
+                        promise.setValue(null);
+                    } else {
+                        promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                    }
+                }
+            }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            promise.setException(e);
+        } catch (InterruptedException e) {
+            promise.setException(e);
+        }
+        return promise;
+    }
+
+    static byte[] serialize(AccessControlEntry ace) throws IOException {
+        TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
+        TJSONProtocol protocol = new TJSONProtocol(transport);
+        try {
+            ace.write(protocol);
+            transport.flush();
+            return transport.toString(UTF_8.name()).getBytes(UTF_8);
+        } catch (TException e) {
+            throw new IOException("Failed to serialize access control entry : ", e);
+        } catch (UnsupportedEncodingException uee) {
+            throw new IOException("Failed to serialize acesss control entry : ", uee);
+        }
+    }
+
+    static AccessControlEntry deserialize(String zkPath, byte[] data) throws IOException {
+        if (data.length == 0) {
+            return DEFAULT_ACCESS_CONTROL_ENTRY;
+        }
+
+        AccessControlEntry ace = new AccessControlEntry();
+        TMemoryInputTransport transport = new TMemoryInputTransport(data);
+        TJSONProtocol protocol = new TJSONProtocol(transport);
+        try {
+            ace.read(protocol);
+        } catch (TException e) {
+            throw new CorruptedAccessControlException(zkPath, e);
+        }
+        return ace;
+    }
+
+}
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
new file mode 100644
index 0000000..be8db64
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/acl/ZKAccessControlManager.java
@@ -0,0 +1,374 @@
+/**
+ * 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.impl.acl;
+
+import com.google.common.collect.Sets;
+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.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * ZooKeeper Based {@link org.apache.distributedlog.acl.AccessControlManager}
+ */
+public class ZKAccessControlManager implements AccessControlManager, Watcher {
+
+    private static final Logger logger = LoggerFactory.getLogger(ZKAccessControlManager.class);
+
+    private static final int ZK_RETRY_BACKOFF_MS = 500;
+
+    protected final DistributedLogConfiguration conf;
+    protected final ZooKeeperClient zkc;
+    protected final String zkRootPath;
+    protected final ScheduledExecutorService scheduledExecutorService;
+
+    protected final ConcurrentMap<String, ZKAccessControl> streamEntries;
+    protected ZKAccessControl defaultAccessControl;
+    protected volatile boolean closed = false;
+
+    public ZKAccessControlManager(DistributedLogConfiguration conf,
+                                  ZooKeeperClient zkc,
+                                  String zkRootPath,
+                                  ScheduledExecutorService scheduledExecutorService) throws IOException {
+        this.conf = conf;
+        this.zkc = zkc;
+        this.zkRootPath = zkRootPath;
+        this.scheduledExecutorService = scheduledExecutorService;
+        this.streamEntries = new ConcurrentHashMap<String, ZKAccessControl>();
+        try {
+            Await.result(fetchDefaultAccessControlEntry());
+        } catch (Throwable t) {
+            if (t instanceof InterruptedException) {
+                throw new DLInterruptedException("Interrupted on getting default access control entry for " + zkRootPath, t);
+            } else if (t instanceof KeeperException) {
+                throw new IOException("Encountered zookeeper exception on getting default access control entry for " + zkRootPath, t);
+            } else if (t instanceof IOException) {
+                throw (IOException) t;
+            } else {
+                throw new IOException("Encountered unknown exception on getting access control entries for " + zkRootPath, t);
+            }
+        }
+
+        try {
+            Await.result(fetchAccessControlEntries());
+        } catch (Throwable t) {
+            if (t instanceof InterruptedException) {
+                throw new DLInterruptedException("Interrupted on getting access control entries for " + zkRootPath, t);
+            } else if (t instanceof KeeperException) {
+                throw new IOException("Encountered zookeeper exception on getting access control entries for " + zkRootPath, t);
+            } else if (t instanceof IOException) {
+                throw (IOException) t;
+            } else {
+                throw new IOException("Encountered unknown exception on getting access control entries for " + zkRootPath, t);
+            }
+        }
+    }
+
+    protected AccessControlEntry getAccessControlEntry(String stream) {
+        ZKAccessControl entry = streamEntries.get(stream);
+        entry = null == entry ? defaultAccessControl : entry;
+        return entry.getAccessControlEntry();
+    }
+
+    @Override
+    public boolean allowWrite(String stream) {
+        return !getAccessControlEntry(stream).isDenyWrite();
+    }
+
+    @Override
+    public boolean allowTruncate(String stream) {
+        return !getAccessControlEntry(stream).isDenyTruncate();
+    }
+
+    @Override
+    public boolean allowDelete(String stream) {
+        return !getAccessControlEntry(stream).isDenyDelete();
+    }
+
+    @Override
+    public boolean allowAcquire(String stream) {
+        return !getAccessControlEntry(stream).isDenyAcquire();
+    }
+
+    @Override
+    public boolean allowRelease(String stream) {
+        return !getAccessControlEntry(stream).isDenyRelease();
+    }
+
+    @Override
+    public void close() {
+        closed = true;
+    }
+
+    private Future<Void> fetchAccessControlEntries() {
+        final Promise<Void> promise = new Promise<Void>();
+        fetchAccessControlEntries(promise);
+        return promise;
+    }
+
+    private void fetchAccessControlEntries(final Promise<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)));
+                        return;
+                    }
+                    Set<String> streamsReceived = new HashSet<String>();
+                    streamsReceived.addAll(children);
+                    Set<String> streamsCached = streamEntries.keySet();
+                    Set<String> streamsRemoved = Sets.difference(streamsCached, streamsReceived).immutableCopy();
+                    for (String s : streamsRemoved) {
+                        ZKAccessControl accessControl = streamEntries.remove(s);
+                        if (null != accessControl) {
+                            logger.info("Removed Access Control Entry for stream {} : {}", s, accessControl.getAccessControlEntry());
+                        }
+                    }
+                    if (streamsReceived.isEmpty()) {
+                        promise.setValue(null);
+                        return;
+                    }
+                    final AtomicInteger numPendings = new AtomicInteger(streamsReceived.size());
+                    final AtomicInteger numFailures = new AtomicInteger(0);
+                    for (String s : streamsReceived) {
+                        final String streamName = s;
+                        ZKAccessControl.read(zkc, zkRootPath + "/" + streamName, null)
+                                .addEventListener(new FutureEventListener<ZKAccessControl>() {
+
+                                    @Override
+                                    public void onSuccess(ZKAccessControl accessControl) {
+                                        streamEntries.put(streamName, accessControl);
+                                        logger.info("Added overrided access control for stream {} : {}", streamName, accessControl.getAccessControlEntry());
+                                        complete();
+                                    }
+
+                                    @Override
+                                    public void onFailure(Throwable cause) {
+                                        if (cause instanceof KeeperException.NoNodeException) {
+                                            streamEntries.remove(streamName);
+                                        } else if (cause instanceof ZKAccessControl.CorruptedAccessControlException) {
+                                            logger.warn("Access control is corrupted for stream {} @ {}, skipped it ...",
+                                                        new Object[] { streamName, zkRootPath, cause });
+                                            streamEntries.remove(streamName);
+                                        } else {
+                                            if (1 == numFailures.incrementAndGet()) {
+                                                promise.setException(cause);
+                                            }
+                                        }
+                                        complete();
+                                    }
+
+                                    private void complete() {
+                                        if (0 == numPendings.decrementAndGet() && numFailures.get() == 0) {
+                                            promise.setValue(null);
+                                        }
+                                    }
+                                });
+                    }
+                }
+            }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            promise.setException(e);
+        } catch (InterruptedException e) {
+            promise.setException(e);
+        }
+    }
+
+    private Future<ZKAccessControl> fetchDefaultAccessControlEntry() {
+        final Promise<ZKAccessControl> promise = new Promise<ZKAccessControl>();
+        fetchDefaultAccessControlEntry(promise);
+        return promise;
+    }
+
+    private void fetchDefaultAccessControlEntry(final Promise<ZKAccessControl> promise) {
+        ZKAccessControl.read(zkc, zkRootPath, this)
+            .addEventListener(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);
+                }
+
+                @Override
+                public void onFailure(Throwable cause) {
+                    if (cause instanceof KeeperException.NoNodeException) {
+                        logger.info("Default Access Control is missing, creating one for {} ...", zkRootPath);
+                        createDefaultAccessControlEntryIfNeeded(promise);
+                    } else {
+                        promise.setException(cause);
+                    }
+                }
+            });
+    }
+
+    private void createDefaultAccessControlEntryIfNeeded(final Promise<ZKAccessControl> promise) {
+        ZooKeeper zk;
+        try {
+            zk = zkc.get();
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            promise.setException(e);
+            return;
+        } catch (InterruptedException e) {
+            promise.setException(e);
+            return;
+        }
+        ZkUtils.asyncCreateFullPathOptimistic(zk, zkRootPath, new byte[0], zkc.getDefaultACL(),
+                CreateMode.PERSISTENT, new AsyncCallback.StringCallback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx, String name) {
+                if (KeeperException.Code.OK.intValue() == rc) {
+                    logger.info("Created zk path {} for default ACL.", zkRootPath);
+                    fetchDefaultAccessControlEntry(promise);
+                } else {
+                    promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                }
+            }
+        }, null);
+    }
+
+    private void refetchDefaultAccessControlEntry(final int delayMs) {
+        if (closed) {
+            return;
+        }
+        scheduledExecutorService.schedule(new Runnable() {
+            @Override
+            public void run() {
+                fetchDefaultAccessControlEntry().addEventListener(new FutureEventListener<ZKAccessControl>() {
+                    @Override
+                    public void onSuccess(ZKAccessControl value) {
+                        // no-op
+                    }
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        if (cause instanceof ZKAccessControl.CorruptedAccessControlException) {
+                            logger.warn("Default access control entry is corrupted, ignore this update : ", cause);
+                            return;
+                        }
+
+                        logger.warn("Encountered an error on refetching default access control entry, retrying in {} ms : ",
+                                    ZK_RETRY_BACKOFF_MS, cause);
+                        refetchDefaultAccessControlEntry(ZK_RETRY_BACKOFF_MS);
+                    }
+                });
+            }
+        }, delayMs, TimeUnit.MILLISECONDS);
+    }
+
+    private void refetchAccessControlEntries(final int delayMs) {
+        if (closed) {
+            return;
+        }
+        scheduledExecutorService.schedule(new Runnable() {
+            @Override
+            public void run() {
+                fetchAccessControlEntries().addEventListener(new FutureEventListener<Void>() {
+                    @Override
+                    public void onSuccess(Void value) {
+                        // no-op
+                    }
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        logger.warn("Encountered an error on refetching access control entries, retrying in {} ms : ",
+                                    ZK_RETRY_BACKOFF_MS, cause);
+                        refetchAccessControlEntries(ZK_RETRY_BACKOFF_MS);
+                    }
+                });
+            }
+        }, delayMs, TimeUnit.MILLISECONDS);
+    }
+
+    private void refetchAllAccessControlEntries(final int delayMs) {
+        if (closed) {
+            return;
+        }
+        scheduledExecutorService.schedule(new Runnable() {
+            @Override
+            public void run() {
+                fetchDefaultAccessControlEntry().addEventListener(new FutureEventListener<ZKAccessControl>() {
+                    @Override
+                    public void onSuccess(ZKAccessControl value) {
+                        fetchAccessControlEntries().addEventListener(new FutureEventListener<Void>() {
+                            @Override
+                            public void onSuccess(Void value) {
+                                // no-op
+                            }
+
+                            @Override
+                            public void onFailure(Throwable cause) {
+                                logger.warn("Encountered an error on fetching all access control entries, retrying in {} ms : ",
+                                            ZK_RETRY_BACKOFF_MS, cause);
+                                refetchAccessControlEntries(ZK_RETRY_BACKOFF_MS);
+                            }
+                        });
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        logger.warn("Encountered an error on refetching all access control entries, retrying in {} ms : ",
+                                    ZK_RETRY_BACKOFF_MS, cause);
+                        refetchAllAccessControlEntries(ZK_RETRY_BACKOFF_MS);
+                    }
+                });
+            }
+        }, delayMs, TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public void process(WatchedEvent event) {
+        if (Event.EventType.None.equals(event.getType())) {
+            if (event.getState() == Event.KeeperState.Expired) {
+                refetchAllAccessControlEntries(0);
+            }
+        } else if (Event.EventType.NodeDataChanged.equals(event.getType())) {
+            logger.info("Default ACL for {} is changed, refetching ...", zkRootPath);
+            refetchDefaultAccessControlEntry(0);
+        } else if (Event.EventType.NodeChildrenChanged.equals(event.getType())) {
+            logger.info("List of ACLs for {} are changed, refetching ...", zkRootPath);
+            refetchAccessControlEntries(0);
+        }
+    }
+}
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
new file mode 100644
index 0000000..5d7af9d
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
@@ -0,0 +1,760 @@
+/**
+ * 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.impl.federated;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import com.google.common.base.Optional;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.callback.NamespaceListener;
+import org.apache.distributedlog.exceptions.LogExistsException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.impl.ZKNamespaceWatcher;
+import org.apache.distributedlog.metadata.LogMetadataStore;
+import org.apache.distributedlog.namespace.NamespaceWatcher;
+import org.apache.distributedlog.util.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;
+import org.apache.zookeeper.KeeperException.Code;
+import org.apache.zookeeper.OpResult;
+import org.apache.zookeeper.Transaction;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+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;
+import java.net.URISyntaxException;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.ConcurrentSkipListMap;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * A Federated ZooKeeper Based Log Metadata Store.
+ *
+ * To Upgrade a simple ZKLogMetadataStore to FederatedZKLogMetadataStore, following steps should be taken in sequence:
+ * a) deploy the new code with disabling createStreamsIfNotExists in all writer.
+ * b) once all proxies disable the flag, update namespace binding to enable federated namespace.
+ * c) restart writers to take federated namespace in place.
+ *
+ * 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>> {
+
+    static final Logger logger = LoggerFactory.getLogger(FederatedZKLogMetadataStore.class);
+
+    private final static String ZNODE_SUB_NAMESPACES = ".subnamespaces";
+    private final static String SUB_NAMESPACE_PREFIX = "NS_";
+
+    /**
+     * Create the federated namespace.
+     *
+     * @param namespace
+     *          namespace to create
+     * @param zkc
+     *          zookeeper client
+     * @throws InterruptedException
+     * @throws ZooKeeperClient.ZooKeeperConnectionException
+     * @throws KeeperException
+     */
+    public static void createFederatedNamespace(URI namespace, ZooKeeperClient zkc)
+            throws InterruptedException, ZooKeeperClient.ZooKeeperConnectionException, KeeperException {
+        String zkSubNamespacesPath = namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES;
+        Utils.zkCreateFullPathOptimistic(zkc, zkSubNamespacesPath, new byte[0],
+                zkc.getDefaultACL(), CreateMode.PERSISTENT);
+    }
+
+    /**
+     * Represent a sub namespace inside the federated namespace.
+     */
+    class SubNamespace implements NamespaceListener {
+        final URI uri;
+        final ZKNamespaceWatcher watcher;
+        Promise<Set<String>> logsFuture = new Promise<Set<String>>();
+
+        SubNamespace(URI uri) {
+            this.uri = uri;
+            this.watcher = new ZKNamespaceWatcher(conf, uri, zkc, scheduler);
+            this.watcher.registerListener(this);
+        }
+
+        void watch() {
+            this.watcher.watchNamespaceChanges();
+        }
+
+        synchronized Future<Set<String>> getLogs() {
+            return logsFuture;
+        }
+
+        @Override
+        public void onStreamsChanged(Iterator<String> newLogsIter) {
+            Set<String> newLogs = Sets.newHashSet(newLogsIter);
+            Set<String> oldLogs = Sets.newHashSet();
+
+            // update the sub namespace cache
+            Promise<Set<String>> newLogsPromise;
+            synchronized (this) {
+                if (logsFuture.isDefined()) { // the promise is already satisfied
+                    try {
+                        oldLogs = FutureUtils.result(logsFuture);
+                    } catch (IOException e) {
+                        logger.error("Unexpected exception when getting logs from a satisified future of {} : ",
+                                uri, e);
+                    }
+                    logsFuture = new Promise<Set<String>>();
+                }
+
+                // update the reverse cache
+                for (String logName : newLogs) {
+                    URI oldURI = log2Locations.putIfAbsent(logName, uri);
+                    if (null != oldURI && !Objects.equal(uri, oldURI)) {
+                        logger.error("Log {} is found duplicated in multiple locations : old location = {}," +
+                                " new location = {}", new Object[] { logName, oldURI, uri });
+                        duplicatedLogFound.set(true);
+                    }
+                }
+
+                // remove the gone streams
+                Set<String> deletedLogs = Sets.difference(oldLogs, newLogs);
+                for (String logName : deletedLogs) {
+                    log2Locations.remove(logName, uri);
+                }
+                newLogsPromise = logsFuture;
+            }
+            newLogsPromise.setValue(newLogs);
+
+            // notify namespace changes
+            notifyOnNamespaceChanges();
+        }
+    }
+
+    final DistributedLogConfiguration conf;
+    final URI namespace;
+    final ZooKeeperClient zkc;
+    final OrderedScheduler scheduler;
+    final String zkSubnamespacesPath;
+    final AtomicBoolean duplicatedLogFound = new AtomicBoolean(false);
+    final AtomicReference<String> duplicatedLogName = new AtomicReference<String>(null);
+    final AtomicReference<Integer> zkSubnamespacesVersion = new AtomicReference<Integer>(null);
+
+    final int maxLogsPerSubnamespace;
+    // sub namespaces
+    final ConcurrentSkipListMap<URI, SubNamespace> subNamespaces;
+    // map between log name and its location
+    final ConcurrentMap<String, URI> log2Locations;
+    // final
+    final boolean forceCheckLogExistence;
+
+    public FederatedZKLogMetadataStore(
+            DistributedLogConfiguration conf,
+            URI namespace,
+            ZooKeeperClient zkc,
+            OrderedScheduler scheduler) throws IOException {
+        this.conf = conf;
+        this.namespace = namespace;
+        this.zkc = zkc;
+        this.scheduler = scheduler;
+        this.forceCheckLogExistence = conf.getFederatedCheckExistenceWhenCacheMiss();
+        this.subNamespaces = new ConcurrentSkipListMap<URI, SubNamespace>();
+        this.log2Locations = new ConcurrentHashMap<String, URI>();
+        this.zkSubnamespacesPath = namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES;
+        this.maxLogsPerSubnamespace = conf.getFederatedMaxLogsPerSubnamespace();
+
+        // fetch the sub namespace
+        Set<URI> uris = FutureUtils.result(fetchSubNamespaces(this));
+        for (URI uri : uris) {
+            SubNamespace subNs = new SubNamespace(uri);
+            if (null == subNamespaces.putIfAbsent(uri, subNs)) {
+                subNs.watch();
+                logger.info("Watched sub namespace {}", uri);
+            }
+        }
+
+        logger.info("Federated ZK LogMetadataStore is initialized for {}", namespace);
+    }
+
+    private void scheduleTask(Runnable r, long ms) {
+        if (duplicatedLogFound.get()) {
+            logger.error("Scheduler is halted for federated namespace {} as duplicated log found",
+                    namespace);
+            return;
+        }
+        try {
+            scheduler.schedule(r, ms, TimeUnit.MILLISECONDS);
+        } catch (RejectedExecutionException ree) {
+            logger.error("Task {} scheduled in {} ms is rejected : ", new Object[]{r, ms, ree});
+        }
+    }
+
+    private <T> Future<T> postStateCheck(Future<T> future) {
+        final Promise<T> postCheckedPromise = new Promise<T>();
+        future.addEventListener(new FutureEventListener<T>() {
+            @Override
+            public void onSuccess(T value) {
+                if (duplicatedLogFound.get()) {
+                    postCheckedPromise.setException(new UnexpectedException("Duplicate log found under " + namespace));
+                } else {
+                    postCheckedPromise.setValue(value);
+                }
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                postCheckedPromise.setException(cause);
+            }
+        });
+        return postCheckedPromise;
+    }
+
+    //
+    // SubNamespace Related Methods
+    //
+
+    @VisibleForTesting
+    Set<URI> getSubnamespaces() {
+        return subNamespaces.keySet();
+    }
+
+    @VisibleForTesting
+    void removeLogFromCache(String logName) {
+        log2Locations.remove(logName);
+    }
+
+    private URI getSubNamespaceURI(String ns) throws URISyntaxException {
+        return new URI(
+                namespace.getScheme(),
+                namespace.getUserInfo(),
+                namespace.getHost(),
+                namespace.getPort(),
+                namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES + "/" + ns,
+                namespace.getQuery(),
+                namespace.getFragment());
+    }
+
+    Future<Set<URI>> getCachedSubNamespaces() {
+        Set<URI> nsSet = subNamespaces.keySet();
+        return Future.value(nsSet);
+    }
+
+    Future<Set<URI>> fetchSubNamespaces(final Watcher watcher) {
+        final Promise<Set<URI>> promise = new Promise<Set<URI>>();
+        try {
+            zkc.get().sync(this.zkSubnamespacesPath, new AsyncCallback.VoidCallback() {
+                @Override
+                public void processResult(int rc, String path, Object ctx) {
+                    if (Code.OK.intValue() == rc) {
+                        fetchSubNamespaces(watcher, promise);
+                    } else {
+                        promise.setException(KeeperException.create(Code.get(rc)));
+                    }
+                }
+            }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            promise.setException(e);
+        } catch (InterruptedException e) {
+            promise.setException(e);
+        }
+        return promise;
+    }
+
+    private void fetchSubNamespaces(Watcher watcher,
+                                    final Promise<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(
+                                        "The subnamespaces don't exist for the federated namespace " + namespace));
+                            } else if (Code.OK.intValue() == rc) {
+                                Set<URI> subnamespaces = Sets.newHashSet();
+                                subnamespaces.add(namespace);
+                                try {
+                                    for (String ns : children) {
+                                        subnamespaces.add(getSubNamespaceURI(ns));
+                                    }
+                                } catch (URISyntaxException use) {
+                                    logger.error("Invalid sub namespace uri found : ", use);
+                                    promise.setException(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);
+                            }
+                        }
+                    }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            promise.setException(e);
+        } catch (InterruptedException e) {
+            promise.setException(e);
+        }
+    }
+
+    @Override
+    public void run() {
+        fetchSubNamespaces(this).addEventListener(this);
+    }
+
+    @Override
+    public void onSuccess(Set<URI> uris) {
+        for (URI uri : uris) {
+            if (subNamespaces.containsKey(uri)) {
+                continue;
+            }
+            SubNamespace subNs = new SubNamespace(uri);
+            if (null == subNamespaces.putIfAbsent(uri, subNs)) {
+                subNs.watch();
+                logger.info("Watched new sub namespace {}.", uri);
+                notifyOnNamespaceChanges();
+            }
+        }
+    }
+
+    @Override
+    public void onFailure(Throwable cause) {
+        // failed to fetch namespaces, retry later
+        scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
+    }
+
+    @Override
+    public void process(WatchedEvent watchedEvent) {
+        if (Event.EventType.None == watchedEvent.getType() &&
+                Event.KeeperState.Expired == watchedEvent.getState()) {
+            scheduleTask(this, conf.getZKSessionTimeoutMilliseconds());
+            return;
+        }
+        if (Event.EventType.NodeChildrenChanged == watchedEvent.getType()) {
+            // fetch the namespace
+            fetchSubNamespaces(this).addEventListener(this);
+        }
+    }
+
+    //
+    // Log Related Methods
+    //
+
+    private <A> Future<A> duplicatedLogException(String logName) {
+        return Future.exception(new UnexpectedException("Duplicated log " + logName
+                + " found in namespace " + namespace));
+    }
+
+    @Override
+    public Future<URI> createLog(final String logName) {
+        if (duplicatedLogFound.get()) {
+            return duplicatedLogException(duplicatedLogName.get());
+        }
+        Promise<URI> createPromise = new Promise<URI>();
+        doCreateLog(logName, createPromise);
+        return postStateCheck(createPromise);
+    }
+
+    void doCreateLog(final String logName, final Promise<URI> createPromise) {
+        getLogLocation(logName).addEventListener(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()));
+                } else {
+                    getCachedSubNamespacesAndCreateLog(logName, createPromise);
+                }
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                createPromise.setException(cause);
+            }
+        });
+    }
+
+    private void getCachedSubNamespacesAndCreateLog(final String logName,
+                                                    final Promise<URI> createPromise) {
+        getCachedSubNamespaces().addEventListener(new FutureEventListener<Set<URI>>() {
+            @Override
+            public void onSuccess(Set<URI> uris) {
+                findSubNamespaceToCreateLog(logName, uris, createPromise);
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                createPromise.setException(cause);
+            }
+        });
+    }
+
+    private void fetchSubNamespacesAndCreateLog(final String logName,
+                                                final Promise<URI> createPromise) {
+        fetchSubNamespaces(null).addEventListener(new FutureEventListener<Set<URI>>() {
+            @Override
+            public void onSuccess(Set<URI> uris) {
+                findSubNamespaceToCreateLog(logName, uris, createPromise);
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                createPromise.setException(cause);
+            }
+        });
+    }
+
+    private void findSubNamespaceToCreateLog(final String logName,
+                                             final Set<URI> uris,
+                                             final Promise<URI> createPromise) {
+        final List<URI> uriList = Lists.newArrayListWithExpectedSize(uris.size());
+        List<Future<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"));
+                return;
+            }
+            futureList.add(subNs.getLogs());
+            uriList.add(uri);
+        }
+        Future.collect(futureList).addEventListener(new FutureEventListener<List<Set<String>>>() {
+            @Override
+            public void onSuccess(List<Set<String>> resultList) {
+                for (int i = resultList.size() - 1; i >= 0; i--) {
+                    Set<String> logs = resultList.get(i);
+                    if (logs.size() < maxLogsPerSubnamespace) {
+                        URI uri = uriList.get(i);
+                        createLogInNamespace(uri, logName, createPromise);
+                        return;
+                    }
+                }
+                // All sub namespaces are full
+                createSubNamespace().addEventListener(new FutureEventListener<URI>() {
+                    @Override
+                    public void onSuccess(URI uri) {
+                        // the new namespace will be propagated to the namespace cache by the namespace listener
+                        // so we don't need to cache it here. we could go ahead to create the stream under this
+                        // namespace, as we are using sequential znode. we are mostly the first guy who create
+                        // the log under this namespace.
+                        createLogInNamespace(uri, logName, createPromise);
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        createPromise.setException(cause);
+                    }
+                });
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                createPromise.setException(cause);
+            }
+        });
+    }
+
+    private String getNamespaceFromZkPath(String zkPath) throws UnexpectedException {
+        String parts[] = zkPath.split(SUB_NAMESPACE_PREFIX);
+        if (parts.length <= 0) {
+            throw new UnexpectedException("Invalid namespace @ " + zkPath);
+        }
+        return SUB_NAMESPACE_PREFIX + parts[parts.length - 1];
+    }
+
+    Future<URI> createSubNamespace() {
+        final Promise<URI> promise = new Promise<URI>();
+
+        final String nsPath = namespace.getPath() + "/" + ZNODE_SUB_NAMESPACES + "/" + SUB_NAMESPACE_PREFIX;
+        try {
+            zkc.get().create(nsPath, new byte[0], zkc.getDefaultACL(), CreateMode.PERSISTENT_SEQUENTIAL,
+                    new AsyncCallback.StringCallback() {
+                        @Override
+                        public void processResult(int rc, String path, Object ctx, String name) {
+                            if (Code.OK.intValue() == rc) {
+                                try {
+                                    URI newUri = getSubNamespaceURI(getNamespaceFromZkPath(name));
+                                    logger.info("Created sub namespace {}", newUri);
+                                    promise.setValue(newUri);
+                                } catch (UnexpectedException ue) {
+                                    promise.setException(ue);
+                                } catch (URISyntaxException e) {
+                                    promise.setException(new UnexpectedException("Invalid namespace " + name + " is created."));
+                                }
+                            } else {
+                                promise.setException(KeeperException.create(Code.get(rc)));
+                            }
+                        }
+                    }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            promise.setException(e);
+        } catch (InterruptedException e) {
+            promise.setException(e);
+        }
+
+        return promise;
+    }
+
+    /**
+     * Create a log under the namespace. To guarantee there is only one creation happens at time
+     * in a federated namespace, we use CAS operation in zookeeper.
+     *
+     * @param uri
+     *          namespace
+     * @param logName
+     *          name of the log
+     * @param createPromise
+     *          the promise representing the creation result.
+     */
+    private void createLogInNamespace(final URI uri,
+                                      final String logName,
+                                      final Promise<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);
+                } catch (InterruptedException e) {
+                    createPromise.setException(e);
+                } catch (IOException e) {
+                    createPromise.setException(e);
+                } catch (KeeperException.BadVersionException bve) {
+                    fetchSubNamespacesAndCreateLog(logName, createPromise);
+                } catch (KeeperException e) {
+                    createPromise.setException(e);
+                }
+            }
+        });
+    }
+
+    void createLogInNamespaceSync(URI uri, String logName)
+            throws InterruptedException, IOException, KeeperException {
+        Transaction txn = zkc.get().transaction();
+        // we don't have the zk version yet. set it to 0 instead of -1, to prevent non CAS operation.
+        int zkVersion = null == zkSubnamespacesVersion.get() ? 0 : zkSubnamespacesVersion.get();
+        txn.setData(zkSubnamespacesPath, uri.getPath().getBytes(UTF_8), zkVersion);
+        String logPath = uri.getPath() + "/" + logName;
+        txn.create(logPath, new byte[0], zkc.getDefaultACL(), CreateMode.PERSISTENT);
+        try {
+            txn.commit();
+            // if the transaction succeed, the zk version is advanced
+            setZkSubnamespacesVersion(zkVersion + 1);
+        } catch (KeeperException ke) {
+            List<OpResult> opResults = ke.getResults();
+            OpResult createResult = opResults.get(1);
+            if (createResult instanceof OpResult.ErrorResult) {
+                OpResult.ErrorResult errorResult = (OpResult.ErrorResult) createResult;
+                if (Code.NODEEXISTS.intValue() == errorResult.getErr()) {
+                    throw new LogExistsException("Log " + logName + " already exists");
+                }
+            }
+            OpResult setResult = opResults.get(0);
+            if (setResult instanceof OpResult.ErrorResult) {
+                OpResult.ErrorResult errorResult = (OpResult.ErrorResult) setResult;
+                if (Code.BADVERSION.intValue() == errorResult.getErr()) {
+                    throw KeeperException.create(Code.BADVERSION);
+                }
+            }
+            throw new ZKException("ZK exception in creating log " + logName + " in " + uri, ke);
+        }
+    }
+
+    void setZkSubnamespacesVersion(int zkVersion) {
+        Integer oldVersion;
+        boolean done = false;
+        while (!done) {
+            oldVersion = zkSubnamespacesVersion.get();
+            if (null == oldVersion) {
+                done = zkSubnamespacesVersion.compareAndSet(null, zkVersion);
+                continue;
+            }
+            if (oldVersion < zkVersion) {
+                done = zkSubnamespacesVersion.compareAndSet(oldVersion, zkVersion);
+                continue;
+            } else {
+                done = true;
+            }
+        }
+    }
+
+    @Override
+    public Future<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)));
+        }
+        if (!forceCheckLogExistence) {
+            Optional<URI> result = Optional.absent();
+            return Future.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;
+                    }
+                }));
+    }
+
+    private Future<Optional<URI>> fetchLogLocation(final String logName) {
+        final Promise<Optional<URI>> fetchPromise = new Promise<Optional<URI>>();
+
+        Set<URI> uris = subNamespaces.keySet();
+        List<Future<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>>>() {
+            @Override
+            public void onSuccess(List<Optional<URI>> fetchResults) {
+                Optional<URI> result = Optional.absent();
+                for (Optional<URI> fetchResult : fetchResults) {
+                    if (result.isPresent()) {
+                        if (fetchResult.isPresent()) {
+                            logger.error("Log {} is found in multiple sub namespaces : {} & {}.",
+                                    new Object[] { logName, result.get(), fetchResult.get() });
+                            duplicatedLogName.compareAndSet(null, logName);
+                            duplicatedLogFound.set(true);
+                            fetchPromise.setException(new UnexpectedException("Log " + logName
+                                    + " is found in multiple sub namespaces : "
+                                    + result.get() + " & " + fetchResult.get()));
+                            return;
+                        }
+                    } else {
+                        result = fetchResult;
+                    }
+                }
+                fetchPromise.setValue(result);
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                fetchPromise.setException(cause);
+            }
+        });
+        return fetchPromise;
+    }
+
+    private Future<Optional<URI>> fetchLogLocation(final URI uri, String logName) {
+        final Promise<Optional<URI>> fetchPromise = new Promise<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));
+                    } else if (Code.NONODE.intValue() == rc) {
+                        fetchPromise.setValue(Optional.<URI>absent());
+                    } else {
+                        fetchPromise.setException(KeeperException.create(Code.get(rc)));
+                    }
+                }
+            }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            fetchPromise.setException(e);
+        } catch (InterruptedException e) {
+            fetchPromise.setException(e);
+        }
+        return fetchPromise;
+    }
+
+    @Override
+    public Future<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);
+                    }
+                }));
+    }
+
+    private Future<List<Set<String>>> retrieveLogs() {
+        Collection<SubNamespace> subNss = subNamespaces.values();
+        List<Future<Set<String>>> logsList = Lists.newArrayListWithExpectedSize(subNss.size());
+        for (SubNamespace subNs : subNss) {
+            logsList.add(subNs.getLogs());
+        }
+        return Future.collect(logsList);
+    }
+
+    private Iterator<String> getIterator(List<Set<String>> resultList) {
+        List<Iterator<String>> iterList = Lists.newArrayListWithExpectedSize(resultList.size());
+        for (Set<String> result : resultList) {
+            iterList.add(result.iterator());
+        }
+        return Iterators.concat(iterList.iterator());
+    }
+
+    @Override
+    public void registerNamespaceListener(NamespaceListener listener) {
+        registerListener(listener);
+    }
+
+    @Override
+    protected void watchNamespaceChanges() {
+        // as the federated namespace already started watching namespace changes,
+        // we don't need to do any actions here
+    }
+
+    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;
+            }
+        });
+    }
+}
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
new file mode 100644
index 0000000..8f9913e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
@@ -0,0 +1,85 @@
+/**
+ * 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.impl.logsegment;
+
+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> {
+
+        static final Function1<LedgerHandle, LogSegmentEntryWriter> INSTANCE =
+                new NewLogSegmentEntryWriterFn();
+
+        private NewLogSegmentEntryWriterFn() {}
+
+        @Override
+        public LogSegmentEntryWriter apply(LedgerHandle lh) {
+            return new BKLogSegmentEntryWriter(lh);
+        }
+    }
+
+    LedgerAllocator allocator;
+
+    BKLogSegmentAllocator(LedgerAllocator allocator) {
+        this.allocator = allocator;
+    }
+
+    @Override
+    public void allocate() throws IOException {
+        allocator.allocate();
+    }
+
+    @Override
+    public Future<LogSegmentEntryWriter> tryObtain(Transaction<Object> txn,
+                                                   final Transaction.OpListener<LogSegmentEntryWriter> listener) {
+        return allocator.tryObtain(txn, new Transaction.OpListener<LedgerHandle>() {
+            @Override
+            public void onCommit(LedgerHandle lh) {
+                listener.onCommit(new BKLogSegmentEntryWriter(lh));
+            }
+
+            @Override
+            public void onAbort(Throwable t) {
+                listener.onAbort(t);
+            }
+        }).map(NewLogSegmentEntryWriterFn.INSTANCE);
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        return allocator.asyncClose();
+    }
+
+    @Override
+    public Future<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
new file mode 100644
index 0000000..034b23e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
@@ -0,0 +1,837 @@
+/**
+ * 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.impl.logsegment;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Lists;
+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.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.EndOfLogSegmentException;
+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.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;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Enumeration;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CopyOnWriteArraySet;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * BookKeeper ledger based log segment entry reader.
+ */
+public class BKLogSegmentEntryReader implements Runnable, LogSegmentEntryReader, AsyncCallback.OpenCallback {
+
+    private static final Logger logger = LoggerFactory.getLogger(BKLogSegmentEntryReader.class);
+
+    private class CacheEntry implements Runnable, AsyncCallback.ReadCallback,
+            AsyncCallback.ReadLastConfirmedAndEntryCallback {
+
+        protected final long entryId;
+        private boolean done;
+        private LedgerEntry entry;
+        private int rc;
+
+        private CacheEntry(long entryId) {
+            this.entryId = entryId;
+            this.entry = null;
+            this.rc = BKException.Code.UnexpectedConditionException;
+            this.done = false;
+        }
+
+        long getEntryId() {
+            return entryId;
+        }
+
+        synchronized boolean isDone() {
+            return done;
+        }
+
+        void setValue(LedgerEntry entry) {
+            synchronized (this) {
+                if (done) {
+                    return;
+                }
+                this.rc = BKException.Code.OK;
+                this.entry = entry;
+            }
+            setDone(true);
+        }
+
+        void setException(int rc) {
+            synchronized (this) {
+                if (done) {
+                    return;
+                }
+                this.rc = rc;
+            }
+            setDone(false);
+        }
+
+        void setDone(boolean success) {
+            synchronized (this) {
+                this.done = true;
+            }
+            onReadEntryDone(success);
+        }
+
+        synchronized boolean isSuccess() {
+            return BKException.Code.OK == rc;
+        }
+
+        synchronized LedgerEntry getEntry() {
+            return this.entry;
+        }
+
+        synchronized int getRc() {
+            return rc;
+        }
+
+        @Override
+        public void readComplete(int rc,
+                                 LedgerHandle lh,
+                                 Enumeration<LedgerEntry> entries,
+                                 Object ctx) {
+            if (failureInjector.shouldInjectCorruption(entryId, entryId)) {
+                rc = BKException.Code.DigestMatchException;
+            }
+            processReadEntries(rc, lh, entries, ctx);
+        }
+
+        void processReadEntries(int rc,
+                                LedgerHandle lh,
+                                Enumeration<LedgerEntry> entries,
+                                Object ctx) {
+            if (isDone()) {
+                return;
+            }
+            if (!checkReturnCodeAndHandleFailure(rc, false)) {
+                return;
+            }
+            LedgerEntry entry = null;
+            while (entries.hasMoreElements()) {
+                // more entries are returned
+                if (null != entry) {
+                    setException(BKException.Code.UnexpectedConditionException);
+                    return;
+                }
+                entry = entries.nextElement();
+            }
+            if (null == entry || entry.getEntryId() != entryId) {
+                setException(BKException.Code.UnexpectedConditionException);
+                return;
+            }
+            setValue(entry);
+        }
+
+        @Override
+        public void readLastConfirmedAndEntryComplete(int rc,
+                                                      long entryId,
+                                                      LedgerEntry entry,
+                                                      Object ctx) {
+            if (failureInjector.shouldInjectCorruption(this.entryId, this.entryId)) {
+                rc = BKException.Code.DigestMatchException;
+            }
+            processReadEntry(rc, entryId, entry, ctx);
+        }
+
+        void processReadEntry(int rc,
+                              long entryId,
+                              LedgerEntry entry,
+                              Object ctx) {
+            if (isDone()) {
+                return;
+            }
+            if (!checkReturnCodeAndHandleFailure(rc, true)) {
+                return;
+            }
+            if (null != entry && this.entryId == entryId) {
+                setValue(entry);
+                return;
+            }
+            // the long poll is timeout or interrupted; we will retry it again.
+            issueRead(this);
+        }
+
+        /**
+         * Check return code and retry if needed.
+         *
+         * @param rc the return code
+         * @param isLongPoll is it a long poll request
+         * @return is the request successful or not
+         */
+        boolean checkReturnCodeAndHandleFailure(int rc, boolean isLongPoll) {
+            if (BKException.Code.OK == rc) {
+                numReadErrors.set(0);
+                return true;
+            }
+            if (BKException.Code.BookieHandleNotAvailableException == rc ||
+                    (isLongPoll && BKException.Code.NoSuchLedgerExistsException == rc)) {
+                int numErrors = Math.max(1, numReadErrors.incrementAndGet());
+                int nextReadBackoffTime = Math.min(numErrors * readAheadWaitTime, maxReadBackoffTime);
+                scheduler.schedule(
+                        getSegment().getLogSegmentId(),
+                        this,
+                        nextReadBackoffTime,
+                        TimeUnit.MILLISECONDS);
+            } else {
+                setException(rc);
+            }
+            return false;
+        }
+
+        @Override
+        public void run() {
+            issueRead(this);
+        }
+    }
+
+    private class PendingReadRequest {
+        private final int numEntries;
+        private final List<Entry.Reader> entries;
+        private final Promise<List<Entry.Reader>> promise;
+
+        PendingReadRequest(int numEntries) {
+            this.numEntries = numEntries;
+            if (numEntries == 1) {
+                this.entries = new ArrayList<Entry.Reader>(1);
+            } else {
+                this.entries = new ArrayList<Entry.Reader>();
+            }
+            this.promise = new Promise<List<Entry.Reader>>();
+        }
+
+        Promise<List<Entry.Reader>> getPromise() {
+            return promise;
+        }
+
+        void setException(Throwable throwable) {
+            FutureUtils.setException(promise, throwable);
+        }
+
+        void addEntry(Entry.Reader entry) {
+            entries.add(entry);
+        }
+
+        void complete() {
+            FutureUtils.setValue(promise, entries);
+            onEntriesConsumed(entries.size());
+        }
+
+        boolean hasReadEntries() {
+            return entries.size() > 0;
+        }
+
+        boolean hasReadEnoughEntries() {
+            return entries.size() >= numEntries;
+        }
+    }
+
+    private final BookKeeper bk;
+    private final DistributedLogConfiguration conf;
+    private final OrderedScheduler scheduler;
+    private final long lssn;
+    private final long startSequenceId;
+    private final boolean envelopeEntries;
+    private final boolean deserializeRecordSet;
+    private final int numPrefetchEntries;
+    private final int maxPrefetchEntries;
+    // state
+    private Promise<Void> closePromise = null;
+    private LogSegmentMetadata metadata;
+    private LedgerHandle lh;
+    private final List<LedgerHandle> openLedgerHandles;
+    private CacheEntry outstandingLongPoll;
+    private long nextEntryId;
+    private final AtomicReference<Throwable> lastException = new AtomicReference<Throwable>(null);
+    private final AtomicLong scheduleCount = new AtomicLong(0);
+    private volatile boolean hasCaughtupOnInprogress = false;
+    private final CopyOnWriteArraySet<StateChangeListener> stateChangeListeners =
+            new CopyOnWriteArraySet<StateChangeListener>();
+    // read retries
+    private int readAheadWaitTime;
+    private final int maxReadBackoffTime;
+    private final AtomicInteger numReadErrors = new AtomicInteger(0);
+    private final boolean skipBrokenEntries;
+    // readahead cache
+    int cachedEntries = 0;
+    int numOutstandingEntries = 0;
+    final LinkedBlockingQueue<CacheEntry> readAheadEntries;
+    // request queue
+    final LinkedList<PendingReadRequest> readQueue;
+
+    // failure injector
+    private final AsyncFailureInjector failureInjector;
+    // Stats
+    private final Counter skippedBrokenEntriesCounter;
+
+    BKLogSegmentEntryReader(LogSegmentMetadata metadata,
+                            LedgerHandle lh,
+                            long startEntryId,
+                            BookKeeper bk,
+                            OrderedScheduler scheduler,
+                            DistributedLogConfiguration conf,
+                            StatsLogger statsLogger,
+                            AsyncFailureInjector failureInjector) {
+        this.metadata = metadata;
+        this.lssn = metadata.getLogSegmentSequenceNumber();
+        this.startSequenceId = metadata.getStartSequenceId();
+        this.envelopeEntries = metadata.getEnvelopeEntries();
+        this.deserializeRecordSet = conf.getDeserializeRecordSetOnReads();
+        this.lh = lh;
+        this.nextEntryId = Math.max(startEntryId, 0);
+        this.bk = bk;
+        this.conf = conf;
+        this.numPrefetchEntries = conf.getNumPrefetchEntriesPerLogSegment();
+        this.maxPrefetchEntries = conf.getMaxPrefetchEntriesPerLogSegment();
+        this.scheduler = scheduler;
+        this.openLedgerHandles = Lists.newArrayList();
+        this.openLedgerHandles.add(lh);
+        this.outstandingLongPoll = null;
+        // create the readahead queue
+        this.readAheadEntries = new LinkedBlockingQueue<CacheEntry>();
+        // create the read request queue
+        this.readQueue = new LinkedList<PendingReadRequest>();
+        // read backoff settings
+        this.readAheadWaitTime = conf.getReadAheadWaitTime();
+        this.maxReadBackoffTime = 4 * conf.getReadAheadWaitTime();
+        // other read settings
+        this.skipBrokenEntries = conf.getReadAheadSkipBrokenEntries();
+
+        // Failure Injection
+        this.failureInjector = failureInjector;
+        // Stats
+        this.skippedBrokenEntriesCounter = statsLogger.getCounter("skipped_broken_entries");
+    }
+
+    @VisibleForTesting
+    public synchronized CacheEntry getOutstandingLongPoll() {
+        return outstandingLongPoll;
+    }
+
+    @VisibleForTesting
+    LinkedBlockingQueue<CacheEntry> getReadAheadEntries() {
+        return this.readAheadEntries;
+    }
+
+    synchronized LedgerHandle getLh() {
+        return lh;
+    }
+
+    @Override
+    public synchronized LogSegmentMetadata getSegment() {
+        return metadata;
+    }
+
+    @VisibleForTesting
+    synchronized long getNextEntryId() {
+        return nextEntryId;
+    }
+
+    @Override
+    public void start() {
+        prefetchIfNecessary();
+    }
+
+    @Override
+    public boolean hasCaughtUpOnInprogress() {
+        return hasCaughtupOnInprogress;
+    }
+
+    @Override
+    public LogSegmentEntryReader registerListener(StateChangeListener listener) {
+        stateChangeListeners.add(listener);
+        return this;
+    }
+
+    @Override
+    public LogSegmentEntryReader unregisterListener(StateChangeListener listener) {
+        stateChangeListeners.remove(listener);
+        return this;
+    }
+
+    private void notifyCaughtupOnInprogress() {
+        for (StateChangeListener listener : stateChangeListeners) {
+            listener.onCaughtupOnInprogress();
+        }
+    }
+
+    //
+    // Process on Log Segment Metadata Updates
+    //
+
+    @Override
+    public synchronized void onLogSegmentMetadataUpdated(LogSegmentMetadata segment) {
+        if (metadata == segment ||
+                LogSegmentMetadata.COMPARATOR.compare(metadata, segment) == 0 ||
+                !(metadata.isInProgress() && !segment.isInProgress())) {
+            return;
+        }
+        // segment is closed from inprogress, then re-open the log segment
+        bk.asyncOpenLedger(
+                segment.getLogSegmentId(),
+                BookKeeper.DigestType.CRC32,
+                conf.getBKDigestPW().getBytes(UTF_8),
+                this,
+                segment);
+    }
+
+    @Override
+    public void openComplete(int rc, LedgerHandle lh, Object ctx) {
+        LogSegmentMetadata segment = (LogSegmentMetadata) ctx;
+        if (BKException.Code.OK != rc) {
+            // fail current reader or retry opening the reader
+            failOrRetryOpenLedger(rc, segment);
+            return;
+        }
+        // switch to new ledger handle if the log segment is moved to completed.
+        CacheEntry longPollRead = null;
+        synchronized (this) {
+            if (isClosed()) {
+                lh.asyncClose(new AsyncCallback.CloseCallback() {
+                    @Override
+                    public void closeComplete(int rc, LedgerHandle lh, Object ctx) {
+                        logger.debug("Close the open ledger {} since the log segment reader is already closed",
+                                lh.getId());
+                    }
+                }, null);
+                return;
+            }
+            this.metadata = segment;
+            this.lh = lh;
+            this.openLedgerHandles.add(lh);
+            longPollRead = outstandingLongPoll;
+        }
+        if (null != longPollRead) {
+            // reissue the long poll read when the log segment state is changed
+            issueRead(longPollRead);
+        }
+        // notify readers
+        notifyReaders();
+    }
+
+    private void failOrRetryOpenLedger(int rc, final LogSegmentMetadata segment) {
+        if (isClosed()) {
+            return;
+        }
+        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),
+                    true);
+            return;
+        }
+        // the reader is still catching up, retry opening the log segment later
+        scheduler.schedule(segment.getLogSegmentId(), new Runnable() {
+            @Override
+            public void run() {
+                onLogSegmentMetadataUpdated(segment);
+            }
+        }, conf.getZKRetryBackoffStartMillis(), TimeUnit.MILLISECONDS);
+    }
+
+    //
+    // Change the state of this reader
+    //
+
+    private boolean checkClosedOrInError() {
+        if (null != lastException.get()) {
+            cancelAllPendingReads(lastException.get());
+            return true;
+        }
+        return false;
+    }
+
+    /**
+     * Set the reader into error state with return code <i>rc</i>.
+     *
+     * @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) {
+        lastException.compareAndSet(null, throwable);
+        if (isBackground) {
+            notifyReaders();
+        }
+    }
+
+    /**
+     * Notify the readers with the state change.
+     */
+    private void notifyReaders() {
+        processReadRequests();
+    }
+
+    private void cancelAllPendingReads(Throwable throwExc) {
+        List<PendingReadRequest> requestsToCancel;
+        synchronized (readQueue) {
+            requestsToCancel = Lists.newArrayListWithExpectedSize(readQueue.size());
+            requestsToCancel.addAll(readQueue);
+            readQueue.clear();
+        }
+        for (PendingReadRequest request : requestsToCancel) {
+            request.setException(throwExc);
+        }
+    }
+
+    //
+    // Background Read Operations
+    //
+
+    private void onReadEntryDone(boolean success) {
+        // we successfully read an entry
+        synchronized (this) {
+            --numOutstandingEntries;
+        }
+        // notify reader that there is entry ready
+        notifyReaders();
+        // stop prefetch if we already encountered exceptions
+        if (success) {
+            prefetchIfNecessary();
+        }
+    }
+
+    private void onEntriesConsumed(int numEntries) {
+        synchronized (this) {
+            cachedEntries -= numEntries;
+        }
+        prefetchIfNecessary();
+    }
+
+    private void prefetchIfNecessary() {
+        List<CacheEntry> entriesToFetch;
+        synchronized (this) {
+            if (cachedEntries >= maxPrefetchEntries) {
+                return;
+            }
+            // we don't have enough entries, do prefetch
+            int numEntriesToFetch = numPrefetchEntries - numOutstandingEntries;
+            if (numEntriesToFetch <= 0) {
+                return;
+            }
+            entriesToFetch = new ArrayList<CacheEntry>(numEntriesToFetch);
+            for (int i = 0; i < numEntriesToFetch; i++) {
+                if (cachedEntries >= maxPrefetchEntries) {
+                    break;
+                }
+                if ((isLedgerClosed() && nextEntryId > getLastAddConfirmed()) ||
+                        (!isLedgerClosed() && nextEntryId > getLastAddConfirmed() + 1)) {
+                    break;
+                }
+                CacheEntry entry = new CacheEntry(nextEntryId);
+                entriesToFetch.add(entry);
+                readAheadEntries.add(entry);
+                ++numOutstandingEntries;
+                ++cachedEntries;
+                ++nextEntryId;
+            }
+        }
+        for (CacheEntry entry : entriesToFetch) {
+            issueRead(entry);
+        }
+    }
+
+
+    private void issueRead(CacheEntry cacheEntry) {
+        if (isClosed()) {
+            return;
+        }
+        if (isLedgerClosed()) {
+            if (isNotBeyondLastAddConfirmed(cacheEntry.getEntryId())) {
+                issueSimpleRead(cacheEntry);
+                return;
+            } else {
+                // Reach the end of stream
+                notifyReaders();
+            }
+        } else { // the ledger is still in progress
+            if (isNotBeyondLastAddConfirmed(cacheEntry.getEntryId())) {
+                issueSimpleRead(cacheEntry);
+            } else {
+                issueLongPollRead(cacheEntry);
+            }
+        }
+    }
+
+    private void issueSimpleRead(CacheEntry cacheEntry) {
+        getLh().asyncReadEntries(cacheEntry.entryId, cacheEntry.entryId, cacheEntry, null);
+    }
+
+    private void issueLongPollRead(CacheEntry cacheEntry) {
+        // register the read as outstanding reads
+        synchronized (this) {
+            this.outstandingLongPoll = cacheEntry;
+        }
+
+        if (!hasCaughtupOnInprogress) {
+            hasCaughtupOnInprogress = true;
+            notifyCaughtupOnInprogress();
+        }
+        getLh().asyncReadLastConfirmedAndEntry(
+                cacheEntry.entryId,
+                conf.getReadLACLongPollTimeout(),
+                false,
+                cacheEntry,
+                null);
+    }
+
+    //
+    // Foreground Read Operations
+    //
+
+    Entry.Reader processReadEntry(LedgerEntry entry) throws IOException {
+        return Entry.newBuilder()
+                .setLogSegmentInfo(lssn, startSequenceId)
+                .setEntryId(entry.getEntryId())
+                .setEnvelopeEntry(envelopeEntries)
+                .deserializeRecordSet(deserializeRecordSet)
+                .setInputStream(entry.getEntryInputStream())
+                .buildReader();
+    }
+
+    @Override
+    public Future<List<Entry.Reader>> readNext(int numEntries) {
+        final PendingReadRequest readRequest = new PendingReadRequest(numEntries);
+
+        if (checkClosedOrInError()) {
+            readRequest.setException(lastException.get());
+        } else {
+            boolean wasQueueEmpty;
+            synchronized (readQueue) {
+                wasQueueEmpty = readQueue.isEmpty();
+                readQueue.add(readRequest);
+            }
+            if (wasQueueEmpty) {
+                processReadRequests();
+            }
+        }
+        return readRequest.getPromise();
+    }
+
+    private void processReadRequests() {
+        if (isClosed()) {
+            // the reader is already closed.
+            return;
+        }
+
+        long prevCount = scheduleCount.getAndIncrement();
+        if (0 == prevCount) {
+            scheduler.submit(getSegment().getLogSegmentId(), this);
+        }
+    }
+
+    /**
+     * The core function to propagate fetched entries to read requests
+     */
+    @Override
+    public void run() {
+        long scheduleCountLocal = scheduleCount.get();
+        while (true) {
+            PendingReadRequest nextRequest = null;
+            synchronized (readQueue) {
+                nextRequest = readQueue.peek();
+            }
+
+            // if read queue is empty, nothing to read, return
+            if (null == nextRequest) {
+                scheduleCount.set(0L);
+                return;
+            }
+
+            // if the oldest pending promise is interrupted then we must
+            // 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 the reader is in error state, stop read
+            if (checkClosedOrInError()) {
+                return;
+            }
+
+            // read entries from readahead cache to satisfy next read request
+            readEntriesFromReadAheadCache(nextRequest);
+
+            // check if we can satisfy the read request
+            if (nextRequest.hasReadEntries()) {
+                PendingReadRequest request;
+                synchronized (readQueue) {
+                    request = readQueue.poll();
+                }
+                if (null != request && nextRequest == request) {
+                    request.complete();
+                } else {
+                    DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at reading from "
+                            + getSegment());
+                    nextRequest.setException(ise);
+                    if (null != request) {
+                        request.setException(ise);
+                    }
+                    setException(ise, false);
+                }
+            } else {
+                if (0 == scheduleCountLocal) {
+                    return;
+                }
+                scheduleCountLocal = scheduleCount.decrementAndGet();
+            }
+        }
+    }
+
+    private void readEntriesFromReadAheadCache(PendingReadRequest nextRequest) {
+        while (!nextRequest.hasReadEnoughEntries()) {
+            CacheEntry entry;
+            boolean hitEndOfLogSegment;
+            synchronized (this) {
+                entry = readAheadEntries.peek();
+                hitEndOfLogSegment = (null == entry) && isEndOfLogSegment();
+            }
+            // reach end of log segment
+            if (hitEndOfLogSegment) {
+                setException(new EndOfLogSegmentException(getSegment().getZNodeName()), false);
+                return;
+            }
+            if (null == entry) {
+                return;
+            }
+            // entry is not complete yet.
+            if (!entry.isDone()) {
+                // we already reached end of the log segment
+                if (isEndOfLogSegment(entry.getEntryId())) {
+                    setException(new EndOfLogSegmentException(getSegment().getZNodeName()), false);
+                }
+                return;
+            }
+            if (entry.isSuccess()) {
+                CacheEntry removedEntry = readAheadEntries.poll();
+                if (entry != removedEntry) {
+                    DLIllegalStateException ise = new DLIllegalStateException("Unexpected condition at reading from "
+                            + getSegment());
+                    setException(ise, false);
+                    return;
+                }
+                try {
+                    nextRequest.addEntry(processReadEntry(entry.getEntry()));
+                } catch (IOException e) {
+                    setException(e, false);
+                    return;
+                }
+            } else if (skipBrokenEntries && BKException.Code.DigestMatchException == entry.getRc()) {
+                // skip this entry and move forward
+                skippedBrokenEntriesCounter.inc();
+                readAheadEntries.poll();
+                continue;
+            } else {
+                setException(new BKTransmitException("Encountered issue on reading entry " + entry.getEntryId()
+                        + " @ log segment " + getSegment(), entry.getRc()), false);
+                return;
+            }
+        }
+    }
+
+    //
+    // State Management
+    //
+
+    private synchronized boolean isEndOfLogSegment() {
+        return isEndOfLogSegment(nextEntryId);
+    }
+
+    private boolean isEndOfLogSegment(long entryId) {
+        return isLedgerClosed() && entryId > getLastAddConfirmed();
+    }
+
+    @Override
+    public synchronized boolean isBeyondLastAddConfirmed() {
+        return isBeyondLastAddConfirmed(nextEntryId);
+    }
+
+    private boolean isBeyondLastAddConfirmed(long entryId) {
+        return entryId > getLastAddConfirmed();
+    }
+
+    private boolean isNotBeyondLastAddConfirmed(long entryId) {
+        return entryId <= getLastAddConfirmed();
+    }
+
+    private boolean isLedgerClosed() {
+        return getLh().isClosed();
+    }
+
+    @Override
+    public long getLastAddConfirmed() {
+        return getLh().getLastAddConfirmed();
+    }
+
+    synchronized boolean isClosed() {
+        return null != closePromise;
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        final Promise<Void> closeFuture;
+        ReadCancelledException exception;
+        LedgerHandle[] lhsToClose;
+        synchronized (this) {
+            if (null != closePromise) {
+                return closePromise;
+            }
+            closeFuture = closePromise = new Promise<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);
+        }
+
+        // cancel all pending reads
+        cancelAllPendingReads(exception);
+
+        // close all the open ledger
+        BKUtils.closeLedgers(lhsToClose).proxyTo(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
new file mode 100644
index 0000000..4dd4c12
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
@@ -0,0 +1,286 @@
+/**
+ * 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.impl.logsegment;
+
+import org.apache.distributedlog.BookKeeperClient;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.bk.DynamicQuorumConfigProvider;
+import org.apache.distributedlog.bk.LedgerAllocator;
+import org.apache.distributedlog.bk.LedgerAllocatorDelegator;
+import org.apache.distributedlog.bk.QuorumConfigProvider;
+import org.apache.distributedlog.bk.SimpleLedgerAllocator;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.BKTransmitException;
+import org.apache.distributedlog.injector.AsyncFailureInjector;
+import org.apache.distributedlog.logsegment.LogSegmentEntryReader;
+import org.apache.distributedlog.logsegment.LogSegmentEntryStore;
+import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
+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.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;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * BookKeeper Based Entry Store
+ */
+public class BKLogSegmentEntryStore implements
+        LogSegmentEntryStore,
+        AsyncCallback.OpenCallback,
+        AsyncCallback.DeleteCallback {
+
+    private static final Logger logger = LoggerFactory.getLogger(BKLogSegmentEntryReader.class);
+
+    private static class OpenReaderRequest {
+
+        private final LogSegmentMetadata segment;
+        private final long startEntryId;
+        private final Promise<LogSegmentEntryReader> openPromise;
+
+        OpenReaderRequest(LogSegmentMetadata segment,
+                          long startEntryId) {
+            this.segment = segment;
+            this.startEntryId = startEntryId;
+            this.openPromise = new Promise<LogSegmentEntryReader>();
+        }
+
+    }
+
+    private static class DeleteLogSegmentRequest {
+
+        private final LogSegmentMetadata segment;
+        private final Promise<LogSegmentMetadata> deletePromise;
+
+        DeleteLogSegmentRequest(LogSegmentMetadata segment) {
+            this.segment = segment;
+            this.deletePromise = new Promise<LogSegmentMetadata>();
+        }
+
+    }
+
+    private final byte[] passwd;
+    private final ZooKeeperClient zkc;
+    private final BookKeeperClient bkc;
+    private final OrderedScheduler scheduler;
+    private final DistributedLogConfiguration conf;
+    private final DynamicDistributedLogConfiguration dynConf;
+    private final StatsLogger statsLogger;
+    private final AsyncFailureInjector failureInjector;
+    // ledger allocator
+    private final LedgerAllocator allocator;
+
+    public BKLogSegmentEntryStore(DistributedLogConfiguration conf,
+                                  DynamicDistributedLogConfiguration dynConf,
+                                  ZooKeeperClient zkc,
+                                  BookKeeperClient bkc,
+                                  OrderedScheduler scheduler,
+                                  LedgerAllocator allocator,
+                                  StatsLogger statsLogger,
+                                  AsyncFailureInjector failureInjector) {
+        this.conf = conf;
+        this.dynConf = dynConf;
+        this.zkc = zkc;
+        this.bkc = bkc;
+        this.passwd = conf.getBKDigestPW().getBytes(UTF_8);
+        this.scheduler = scheduler;
+        this.allocator = allocator;
+        this.statsLogger = statsLogger;
+        this.failureInjector = failureInjector;
+    }
+
+    @Override
+    public Future<LogSegmentMetadata> deleteLogSegment(LogSegmentMetadata segment) {
+        DeleteLogSegmentRequest request = new DeleteLogSegmentRequest(segment);
+        BookKeeper bk;
+        try {
+            bk = this.bkc.get();
+        } catch (IOException e) {
+            return Future.exception(e);
+        }
+        bk.asyncDeleteLedger(segment.getLogSegmentId(), this, request);
+        return request.deletePromise;
+    }
+
+    @Override
+    public void deleteComplete(int rc, Object ctx) {
+        DeleteLogSegmentRequest deleteRequest = (DeleteLogSegmentRequest) ctx;
+        if (BKException.Code.NoSuchLedgerExistsException == rc) {
+            logger.warn("No ledger {} found to delete for {}.",
+                    deleteRequest.segment.getLogSegmentId(), deleteRequest.segment);
+        } else if (BKException.Code.OK != rc) {
+            logger.error("Couldn't delete ledger {} from bookkeeper for {} : {}",
+                    new Object[]{ deleteRequest.segment.getLogSegmentId(), deleteRequest.segment,
+                            BKException.getMessage(rc) });
+            FutureUtils.setException(deleteRequest.deletePromise,
+                    new BKTransmitException("Couldn't delete log segment " + deleteRequest.segment, rc));
+            return;
+        }
+        FutureUtils.setValue(deleteRequest.deletePromise, deleteRequest.segment);
+    }
+
+    //
+    // Writers
+    //
+
+    LedgerAllocator createLedgerAllocator(LogMetadataForWriter logMetadata,
+                                          DynamicDistributedLogConfiguration dynConf)
+            throws IOException {
+        LedgerAllocator ledgerAllocatorDelegator;
+        if (null == allocator || !dynConf.getEnableLedgerAllocatorPool()) {
+            QuorumConfigProvider quorumConfigProvider =
+                    new DynamicQuorumConfigProvider(dynConf);
+            LedgerAllocator allocator = new SimpleLedgerAllocator(
+                    logMetadata.getAllocationPath(),
+                    logMetadata.getAllocationData(),
+                    quorumConfigProvider,
+                    zkc,
+                    bkc);
+            ledgerAllocatorDelegator = new LedgerAllocatorDelegator(allocator, true);
+        } else {
+            ledgerAllocatorDelegator = allocator;
+        }
+        return ledgerAllocatorDelegator;
+    }
+
+    @Override
+    public Allocator<LogSegmentEntryWriter, Object> newLogSegmentAllocator(
+            LogMetadataForWriter logMetadata,
+            DynamicDistributedLogConfiguration dynConf) throws IOException {
+        // Build the ledger allocator
+        LedgerAllocator allocator = createLedgerAllocator(logMetadata, dynConf);
+        return new BKLogSegmentAllocator(allocator);
+    }
+
+    //
+    // Readers
+    //
+
+    @Override
+    public Future<LogSegmentEntryReader> openReader(LogSegmentMetadata segment,
+                                                    long startEntryId) {
+        BookKeeper bk;
+        try {
+            bk = this.bkc.get();
+        } catch (IOException e) {
+            return Future.exception(e);
+        }
+        OpenReaderRequest request = new OpenReaderRequest(segment, startEntryId);
+        if (segment.isInProgress()) {
+            bk.asyncOpenLedgerNoRecovery(
+                    segment.getLogSegmentId(),
+                    BookKeeper.DigestType.CRC32,
+                    passwd,
+                    this,
+                    request);
+        } else {
+            bk.asyncOpenLedger(
+                    segment.getLogSegmentId(),
+                    BookKeeper.DigestType.CRC32,
+                    passwd,
+                    this,
+                    request);
+        }
+        return request.openPromise;
+    }
+
+    @Override
+    public void openComplete(int rc, LedgerHandle lh, Object ctx) {
+        OpenReaderRequest request = (OpenReaderRequest) ctx;
+        if (BKException.Code.OK != rc) {
+            FutureUtils.setException(
+                    request.openPromise,
+                    new BKTransmitException("Failed to open ledger handle for log segment " + request.segment, rc));
+            return;
+        }
+        // successfully open a ledger
+        try {
+            LogSegmentEntryReader reader = new BKLogSegmentEntryReader(
+                    request.segment,
+                    lh,
+                    request.startEntryId,
+                    bkc.get(),
+                    scheduler,
+                    conf,
+                    statsLogger,
+                    failureInjector);
+            FutureUtils.setValue(request.openPromise, reader);
+        } catch (IOException e) {
+            FutureUtils.setException(request.openPromise, e);
+        }
+
+    }
+
+    @Override
+    public Future<LogSegmentRandomAccessEntryReader> openRandomAccessReader(final LogSegmentMetadata segment,
+                                                                            final boolean fence) {
+        final BookKeeper bk;
+        try {
+            bk = this.bkc.get();
+        } catch (IOException e) {
+            return Future.exception(e);
+        }
+        final Promise<LogSegmentRandomAccessEntryReader> openPromise = new Promise<LogSegmentRandomAccessEntryReader>();
+        AsyncCallback.OpenCallback openCallback = new AsyncCallback.OpenCallback() {
+            @Override
+            public void openComplete(int rc, LedgerHandle lh, Object ctx) {
+                if (BKException.Code.OK != rc) {
+                    FutureUtils.setException(
+                            openPromise,
+                            new BKTransmitException("Failed to open ledger handle for log segment " + segment, rc));
+                    return;
+                }
+                LogSegmentRandomAccessEntryReader reader = new BKLogSegmentRandomAccessEntryReader(
+                        segment,
+                        lh,
+                        conf);
+                FutureUtils.setValue(openPromise, reader);
+            }
+        };
+        if (segment.isInProgress() && !fence) {
+            bk.asyncOpenLedgerNoRecovery(
+                    segment.getLogSegmentId(),
+                    BookKeeper.DigestType.CRC32,
+                    passwd,
+                    openCallback,
+                    null);
+        } else {
+            bk.asyncOpenLedger(
+                    segment.getLogSegmentId(),
+                    BookKeeper.DigestType.CRC32,
+                    passwd,
+                    openCallback,
+                    null);
+        }
+        return openPromise;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java
new file mode 100644
index 0000000..b333e96
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java
@@ -0,0 +1,61 @@
+/**
+ * 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.impl.logsegment;
+
+import com.google.common.annotations.VisibleForTesting;
+import org.apache.distributedlog.logsegment.LogSegmentEntryWriter;
+import org.apache.bookkeeper.client.AsyncCallback;
+import org.apache.bookkeeper.client.LedgerHandle;
+
+/**
+ * Ledger based log segment entry writer.
+ */
+public class BKLogSegmentEntryWriter implements LogSegmentEntryWriter {
+
+    private final LedgerHandle lh;
+
+    public BKLogSegmentEntryWriter(LedgerHandle lh) {
+        this.lh = lh;
+    }
+
+    @VisibleForTesting
+    public LedgerHandle getLedgerHandle() {
+        return this.lh;
+    }
+
+    @Override
+    public long getLogSegmentId() {
+        return lh.getId();
+    }
+
+    @Override
+    public void asyncClose(AsyncCallback.CloseCallback callback, Object ctx) {
+        lh.asyncClose(callback, ctx);
+    }
+
+    @Override
+    public void asyncAddEntry(byte[] data, int offset, int length,
+                              AsyncCallback.AddCallback callback, Object ctx) {
+        lh.asyncAddEntry(data, offset, length, callback, ctx);
+    }
+
+    @Override
+    public long size() {
+        return lh.getLength();
+    }
+}
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
new file mode 100644
index 0000000..d7b331b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.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.impl.logsegment;
+
+import com.google.common.collect.Lists;
+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.bookkeeper.client.AsyncCallback.ReadCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+
+import java.io.IOException;
+import java.util.Enumeration;
+import java.util.List;
+
+/**
+ * BookKeeper ledger based random access entry reader.
+ */
+class BKLogSegmentRandomAccessEntryReader implements
+        LogSegmentRandomAccessEntryReader,
+        ReadCallback {
+
+    private final long lssn;
+    private final long startSequenceId;
+    private final boolean envelopeEntries;
+    private final boolean deserializeRecordSet;
+    // state
+    private final LogSegmentMetadata metadata;
+    private final LedgerHandle lh;
+    private Promise<Void> closePromise = null;
+
+    BKLogSegmentRandomAccessEntryReader(LogSegmentMetadata metadata,
+                                        LedgerHandle lh,
+                                        DistributedLogConfiguration conf) {
+        this.metadata = metadata;
+        this.lssn = metadata.getLogSegmentSequenceNumber();
+        this.startSequenceId = metadata.getStartSequenceId();
+        this.envelopeEntries = metadata.getEnvelopeEntries();
+        this.deserializeRecordSet = conf.getDeserializeRecordSetOnReads();
+        this.lh = lh;
+    }
+
+    @Override
+    public long getLastAddConfirmed() {
+        return lh.getLastAddConfirmed();
+    }
+
+    @Override
+    public Future<List<Entry.Reader>> readEntries(long startEntryId, long endEntryId) {
+        Promise<List<Entry.Reader>> promise = new Promise<List<Entry.Reader>>();
+        lh.asyncReadEntries(startEntryId, endEntryId, this, promise);
+        return promise;
+    }
+
+    Entry.Reader processReadEntry(LedgerEntry entry) throws IOException {
+        return Entry.newBuilder()
+                .setLogSegmentInfo(lssn, startSequenceId)
+                .setEntryId(entry.getEntryId())
+                .setEnvelopeEntry(envelopeEntries)
+                .deserializeRecordSet(deserializeRecordSet)
+                .setInputStream(entry.getEntryInputStream())
+                .buildReader();
+    }
+
+    @Override
+    public void readComplete(int rc, LedgerHandle lh, Enumeration<LedgerEntry> entries, Object ctx) {
+        Promise<List<Entry.Reader>> promise = (Promise<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);
+                    return;
+                }
+            }
+            FutureUtils.setValue(promise, entryList);
+        } else {
+            FutureUtils.setException(promise,
+                    new BKTransmitException("Failed to read entries :", rc));
+        }
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        final Promise<Void> closeFuture;
+        synchronized (this) {
+            if (null != closePromise) {
+                return closePromise;
+            }
+            closeFuture = closePromise = new Promise<Void>();
+        }
+        BKUtils.closeLedgers(lh).proxyTo(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
new file mode 100644
index 0000000..3c02740
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/logsegment/BKUtils.java
@@ -0,0 +1,72 @@
+/**
+ * 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.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 org.apache.bookkeeper.client.AsyncCallback;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.LedgerHandle;
+
+import java.util.List;
+
+/**
+ * BookKeeper Util Functions
+ */
+public class BKUtils {
+
+    /**
+     * Close a ledger <i>lh</i>.
+     *
+     * @param lh ledger handle
+     * @return future represents close result.
+     */
+    public static Future<Void> closeLedger(LedgerHandle lh) {
+        final Promise<Void> closePromise = new Promise<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));
+                } else {
+                    FutureUtils.setValue(closePromise, null);
+                }
+            }
+        }, null);
+        return closePromise;
+    }
+
+    /**
+     * Close a list of ledgers <i>lhs</i>.
+     *
+     * @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);
+        for (LedgerHandle lh : lhs) {
+            closeResults.add(closeLedger(lh));
+        }
+        return Futures.collect(closeResults).map(VoidFunctions.LIST_TO_VOID_FUNC);
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java
new file mode 100644
index 0000000..1f3128d
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/BKDLConfig.java
@@ -0,0 +1,400 @@
+/**
+ * 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.impl.metadata;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Objects;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.distributedlog.metadata.DLConfig;
+import org.apache.distributedlog.thrift.BKDLConfigFormat;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TJSONProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.apache.thrift.transport.TMemoryInputTransport;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.net.URI;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Configurations for BookKeeper based DL.
+ */
+public class BKDLConfig implements DLConfig {
+
+    private static final Logger LOG = LoggerFactory.getLogger(BKDLConfig.class);
+
+    private static final int BUFFER_SIZE = 4096;
+    private static final ConcurrentMap<URI, DLConfig> cachedDLConfigs =
+            new ConcurrentHashMap<URI, DLConfig>();
+
+    public static void propagateConfiguration(BKDLConfig bkdlConfig, DistributedLogConfiguration dlConf) {
+        dlConf.setEncodeRegionIDInLogSegmentMetadata(bkdlConfig.getEncodeRegionID());
+        dlConf.setFirstLogSegmentSequenceNumber(bkdlConfig.getFirstLogSegmentSeqNo());
+        if (bkdlConfig.isFederatedNamespace()) {
+            dlConf.setCreateStreamIfNotExists(false);
+            LOG.info("Disabled createIfNotExists for federated namespace.");
+        }
+        LOG.info("Propagate BKDLConfig to DLConfig : encodeRegionID = {}," +
+                        " firstLogSegmentSequenceNumber = {}, createStreamIfNotExists = {}, isFederated = {}.",
+                new Object[] { dlConf.getEncodeRegionIDInLogSegmentMetadata(),
+                        dlConf.getFirstLogSegmentSequenceNumber(), dlConf.getCreateStreamIfNotExists(),
+                        bkdlConfig.isFederatedNamespace() });
+    }
+
+    public static BKDLConfig resolveDLConfig(ZooKeeperClient zkc, URI uri) throws IOException {
+        DLConfig dlConfig = cachedDLConfigs.get(uri);
+        if (dlConfig == null) {
+            dlConfig = (new ZkMetadataResolver(zkc).resolve(uri)).getDLConfig();
+            DLConfig oldDLConfig = cachedDLConfigs.putIfAbsent(uri, dlConfig);
+            if (null != oldDLConfig) {
+                dlConfig = oldDLConfig;
+            }
+        }
+        assert (dlConfig instanceof BKDLConfig);
+        return (BKDLConfig)dlConfig;
+    }
+
+    @VisibleForTesting
+    public static void clearCachedDLConfigs() {
+        cachedDLConfigs.clear();
+    }
+
+    private String bkZkServersForWriter;
+    private String bkZkServersForReader;
+    private String bkLedgersPath;
+    private boolean sanityCheckTxnID = true;
+    private boolean encodeRegionID = false;
+    private String dlZkServersForWriter;
+    private String dlZkServersForReader;
+    private String aclRootPath;
+    private Long firstLogSegmentSeqNo;
+    private boolean isFederatedNamespace = false;
+
+    /**
+     * Construct a empty config with given <i>uri</i>.
+     */
+    public BKDLConfig(URI uri) {
+        this(BKNamespaceDriver.getZKServersFromDLUri(uri),
+             BKNamespaceDriver.getZKServersFromDLUri(uri),
+             null, null, null);
+    }
+
+    /**
+     * The caller should make sure both dl and bk use same zookeeper server.
+     *
+     * @param zkServers
+     *          zk servers used for both dl and bk.
+     * @param ledgersPath
+     *          ledgers path.
+     */
+    @VisibleForTesting
+    public BKDLConfig(String zkServers, String ledgersPath) {
+        this(zkServers, zkServers, zkServers, zkServers, ledgersPath);
+    }
+
+    public BKDLConfig(String dlZkServersForWriter,
+                      String dlZkServersForReader,
+                      String bkZkServersForWriter,
+                      String bkZkServersForReader,
+                      String bkLedgersPath) {
+        this.dlZkServersForWriter = dlZkServersForWriter;
+        this.dlZkServersForReader = dlZkServersForReader;
+        this.bkZkServersForWriter = bkZkServersForWriter;
+        this.bkZkServersForReader = bkZkServersForReader;
+        this.bkLedgersPath = bkLedgersPath;
+    }
+
+    /**
+     * @return zk servers used for bk for writers
+     */
+    public String getBkZkServersForWriter() {
+        return bkZkServersForWriter;
+    }
+
+    /**
+     * @return zk servers used for bk for readers
+     */
+    public String getBkZkServersForReader() {
+        return bkZkServersForReader;
+    }
+
+    /**
+     * @return zk servers used for dl for writers
+     */
+    public String getDlZkServersForWriter() {
+        return dlZkServersForWriter;
+    }
+
+    /**
+     * @return zk servers used for dl for readers
+     */
+    public String getDlZkServersForReader() {
+        return dlZkServersForReader;
+    }
+
+    /**
+     * @return ledgers path for bk
+     */
+    public String getBkLedgersPath() {
+        return bkLedgersPath;
+    }
+
+    /**
+     * Enable/Disable sanity check txn id.
+     *
+     * @param enabled
+     *          flag to enable/disable sanity check txn id.
+     * @return bk dl config.
+     */
+    public BKDLConfig setSanityCheckTxnID(boolean enabled) {
+        this.sanityCheckTxnID = enabled;
+        return this;
+    }
+
+    /**
+     * @return flag to sanity check highest txn id.
+     */
+    public boolean getSanityCheckTxnID() {
+        return sanityCheckTxnID;
+    }
+
+    /**
+     * Enable/Disable encode region id.
+     *
+     * @param enabled
+     *          flag to enable/disable encoding region id.
+     * @return bk dl config
+     */
+    public BKDLConfig setEncodeRegionID(boolean enabled) {
+        this.encodeRegionID = enabled;
+        return this;
+    }
+
+    /**
+     * @return flag to encode region id.
+     */
+    public boolean getEncodeRegionID() {
+        return encodeRegionID;
+    }
+
+    /**
+     * Set the root path of zk based ACL manager.
+     *
+     * @param aclRootPath
+     *          root path of zk based ACL manager.
+     * @return bk dl config
+     */
+    public BKDLConfig setACLRootPath(String aclRootPath) {
+        this.aclRootPath = aclRootPath;
+        return this;
+    }
+
+    /**
+     * Get the root path of zk based ACL manager.
+     *
+     * @return root path of zk based ACL manager.
+     */
+    public String getACLRootPath() {
+        return aclRootPath;
+    }
+
+    /**
+     * Set the value at which ledger sequence number should start for streams that are being
+     * upgraded and did not have ledger sequence number to start with or for newly created
+     * streams
+     *
+     * @param firstLogSegmentSeqNo first ledger sequence number
+     * @return bk dl config
+     */
+    public BKDLConfig setFirstLogSegmentSeqNo(long firstLogSegmentSeqNo) {
+        this.firstLogSegmentSeqNo = firstLogSegmentSeqNo;
+        return this;
+    }
+
+    /**
+     * Get the value at which ledger sequence number should start for streams that are being
+     * upgraded and did not have ledger sequence number to start with or for newly created
+     * streams
+     *
+     * @return first ledger sequence number
+     */
+    public Long getFirstLogSegmentSeqNo() {
+        if (null == firstLogSegmentSeqNo) {
+            return DistributedLogConstants.FIRST_LOGSEGMENT_SEQNO;
+        }
+        return firstLogSegmentSeqNo;
+    }
+
+    /**
+     * Set the namespace to federated <i>isFederatedNamespace</i>.
+     *
+     * @param isFederatedNamespace
+     *          is the namespace federated?
+     * @return bk dl config
+     */
+    public BKDLConfig setFederatedNamespace(boolean isFederatedNamespace) {
+        this.isFederatedNamespace = isFederatedNamespace;
+        return this;
+    }
+
+    /**
+     * Whether the namespace is federated namespace
+     *
+     * @return true if the namespace is a federated namespace. otherwise false.
+     */
+    public boolean isFederatedNamespace() {
+        return this.isFederatedNamespace;
+    }
+
+    @Override
+    public int hashCode() {
+        return Objects.hashCode(bkZkServersForWriter, bkZkServersForReader,
+                                dlZkServersForWriter, dlZkServersForReader,
+                                bkLedgersPath);
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof BKDLConfig)) {
+            return false;
+        }
+        BKDLConfig another = (BKDLConfig) o;
+        return Objects.equal(bkZkServersForWriter, another.bkZkServersForWriter) &&
+               Objects.equal(bkZkServersForReader, another.bkZkServersForReader) &&
+               Objects.equal(dlZkServersForWriter, another.dlZkServersForWriter) &&
+               Objects.equal(dlZkServersForReader, another.dlZkServersForReader) &&
+               Objects.equal(bkLedgersPath, another.bkLedgersPath) &&
+               sanityCheckTxnID == another.sanityCheckTxnID &&
+               encodeRegionID == another.encodeRegionID &&
+               Objects.equal(aclRootPath, another.aclRootPath) &&
+               Objects.equal(firstLogSegmentSeqNo, another.firstLogSegmentSeqNo) &&
+               Objects.equal(isFederatedNamespace, another.isFederatedNamespace);
+
+    }
+
+    @Override
+    public String toString() {
+        return serialize();
+    }
+
+    @Override
+    public String serialize() {
+        BKDLConfigFormat configFormat = new BKDLConfigFormat();
+        if (null != bkZkServersForWriter) {
+            configFormat.setBkZkServers(bkZkServersForWriter);
+        }
+        if (null != bkZkServersForReader) {
+            configFormat.setBkZkServersForReader(bkZkServersForReader);
+        }
+        if (null != dlZkServersForWriter) {
+            configFormat.setDlZkServersForWriter(dlZkServersForWriter);
+        }
+        if (null != dlZkServersForReader) {
+            configFormat.setDlZkServersForReader(dlZkServersForReader);
+        }
+        if (null != bkLedgersPath) {
+            configFormat.setBkLedgersPath(bkLedgersPath);
+        }
+        configFormat.setSanityCheckTxnID(sanityCheckTxnID);
+        configFormat.setEncodeRegionID(encodeRegionID);
+        if (null != aclRootPath) {
+            configFormat.setAclRootPath(aclRootPath);
+        }
+        if (null != firstLogSegmentSeqNo) {
+            configFormat.setFirstLogSegmentSeqNo(firstLogSegmentSeqNo);
+        }
+        if (isFederatedNamespace) {
+            configFormat.setFederatedNamespace(true);
+        }
+        return serialize(configFormat);
+    }
+
+    String serialize(BKDLConfigFormat configFormat) {
+        TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
+        TJSONProtocol protocol = new TJSONProtocol(transport);
+        try {
+            configFormat.write(protocol);
+            transport.flush();
+            return transport.toString("UTF-8");
+        } catch (TException e) {
+            throw new RuntimeException("Failed to serialize BKDLConfig : ", e);
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException("Failed to serialize BKDLConfig : ", e);
+        }
+    }
+
+    @Override
+    public void deserialize(byte[] data) throws IOException {
+        BKDLConfigFormat configFormat = new BKDLConfigFormat();
+        TMemoryInputTransport transport = new TMemoryInputTransport(data);
+        TJSONProtocol protocol = new TJSONProtocol(transport);
+        try {
+            configFormat.read(protocol);
+        } catch (TException e) {
+            throw new IOException("Failed to deserialize data '" +
+                    new String(data, UTF_8) + "' : ", e);
+        }
+        // bookkeeper cluster settings
+        if (configFormat.isSetBkZkServers()) {
+            bkZkServersForWriter = configFormat.getBkZkServers();
+        }
+        if (configFormat.isSetBkZkServersForReader()) {
+            bkZkServersForReader = configFormat.getBkZkServersForReader();
+        } else {
+            bkZkServersForReader = bkZkServersForWriter;
+        }
+        if (configFormat.isSetBkLedgersPath()) {
+            bkLedgersPath = configFormat.getBkLedgersPath();
+        }
+        // dl zookeeper cluster settings
+        if (configFormat.isSetDlZkServersForWriter()) {
+            dlZkServersForWriter = configFormat.getDlZkServersForWriter();
+        }
+        if (configFormat.isSetDlZkServersForReader()) {
+            dlZkServersForReader = configFormat.getDlZkServersForReader();
+        } else {
+            dlZkServersForReader = dlZkServersForWriter;
+        }
+        // dl settings
+        sanityCheckTxnID = !configFormat.isSetSanityCheckTxnID() || configFormat.isSanityCheckTxnID();
+        encodeRegionID = configFormat.isSetEncodeRegionID() && configFormat.isEncodeRegionID();
+        if (configFormat.isSetAclRootPath()) {
+            aclRootPath = configFormat.getAclRootPath();
+        }
+
+        if (configFormat.isSetFirstLogSegmentSeqNo()) {
+            firstLogSegmentSeqNo = configFormat.getFirstLogSegmentSeqNo();
+        }
+        isFederatedNamespace = configFormat.isSetFederatedNamespace() && configFormat.isFederatedNamespace();
+
+        // Validate the settings
+        if (null == bkZkServersForWriter || null == bkZkServersForReader || null == bkLedgersPath ||
+                null == dlZkServersForWriter || null == dlZkServersForReader) {
+            throw new IOException("Missing zk/bk settings in BKDL Config : " + new String(data, UTF_8));
+        }
+    }
+}
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
new file mode 100644
index 0000000..e6891c7
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
@@ -0,0 +1,633 @@
+/**
+ * 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.impl.metadata;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+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;
+import org.apache.distributedlog.exceptions.LogExistsException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.impl.ZKLogSegmentMetadataStore;
+import org.apache.distributedlog.lock.DistributedLock;
+import org.apache.distributedlog.lock.SessionLockFactory;
+import org.apache.distributedlog.lock.ZKDistributedLock;
+import org.apache.distributedlog.lock.ZKSessionLockFactory;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
+import org.apache.distributedlog.metadata.LogStreamMetadataStore;
+import org.apache.distributedlog.metadata.LogMetadata;
+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.zk.LimitedPermitManager;
+import org.apache.distributedlog.util.OrderedScheduler;
+import org.apache.distributedlog.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;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
+import org.apache.zookeeper.ZKUtil;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.data.ACL;
+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;
+import java.util.concurrent.TimeUnit;
+
+import static org.apache.distributedlog.metadata.LogMetadata.*;
+
+/**
+ * zookeeper based {@link LogStreamMetadataStore}
+ */
+public class ZKLogStreamMetadataStore implements LogStreamMetadataStore {
+
+    private final static Logger LOG = LoggerFactory.getLogger(ZKLogStreamMetadataStore.class);
+
+    private final String clientId;
+    private final DistributedLogConfiguration conf;
+    private final ZooKeeperClient zooKeeperClient;
+    private final OrderedScheduler scheduler;
+    private final StatsLogger statsLogger;
+    private final LogSegmentMetadataStore logSegmentStore;
+    private final LimitedPermitManager permitManager;
+    // lock
+    private SessionLockFactory lockFactory;
+    private OrderedScheduler lockStateExecutor;
+
+    public ZKLogStreamMetadataStore(String clientId,
+                                    DistributedLogConfiguration conf,
+                                    ZooKeeperClient zkc,
+                                    OrderedScheduler scheduler,
+                                    StatsLogger statsLogger) {
+        this.clientId = clientId;
+        this.conf = conf;
+        this.zooKeeperClient = zkc;
+        this.scheduler = scheduler;
+        this.statsLogger = statsLogger;
+        // create the log segment metadata store and the permit manager (used for log segment rolling)
+        this.logSegmentStore = new ZKLogSegmentMetadataStore(conf, zooKeeperClient, scheduler);
+        this.permitManager = new LimitedPermitManager(
+                conf.getLogSegmentRollingConcurrency(),
+                1,
+                TimeUnit.MINUTES,
+                scheduler);
+        this.zooKeeperClient.register(permitManager);
+    }
+
+    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;
+    }
+
+    private synchronized SessionLockFactory getLockFactory(boolean createIfNull) {
+        if (createIfNull && null == lockFactory) {
+            lockFactory = new ZKSessionLockFactory(
+                    zooKeeperClient,
+                    clientId,
+                    getLockStateExecutor(createIfNull),
+                    conf.getZKNumRetries(),
+                    conf.getLockTimeoutMilliSeconds(),
+                    conf.getZKRetryBackoffStartMillis(),
+                    statsLogger);
+        }
+        return lockFactory;
+    }
+
+    @Override
+    public void close() throws IOException {
+        this.zooKeeperClient.unregister(permitManager);
+        this.permitManager.close();
+        this.logSegmentStore.close();
+        SchedulerUtils.shutdownScheduler(
+                getLockStateExecutor(false),
+                conf.getSchedulerShutdownTimeoutMs(),
+                TimeUnit.MILLISECONDS);
+    }
+
+    @Override
+    public LogSegmentMetadataStore getLogSegmentMetadataStore() {
+        return logSegmentStore;
+    }
+
+    @Override
+    public PermitManager getPermitManager() {
+        return this.permitManager;
+    }
+
+    @Override
+    public Transaction<Object> newTransaction() {
+        return new ZKTransaction(zooKeeperClient);
+    }
+
+    @Override
+    public Future<Void> logExists(URI uri, final String logName) {
+        final String logSegmentsPath = LogMetadata.getLogSegmentsPath(
+                uri, logName, conf.getUnpartitionedStreamName());
+        final Promise<Void> promise = new Promise<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(
+                                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,
+                                KeeperException.create(KeeperException.Code.get(syncRc))));
+                        return;
+                    }
+                    zk.exists(logSegmentsPath, false, new AsyncCallback.StatCallback() {
+                        @Override
+                        public void processResult(int rc, String path, Object ctx, Stat stat) {
+                            if (KeeperException.Code.OK.intValue() == rc) {
+                                promise.setValue(null);
+                            } else if (KeeperException.Code.NONODE.intValue() == rc) {
+                                promise.setException(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,
+                                        KeeperException.create(KeeperException.Code.get(rc))));
+                            }
+                        }
+                    }, null);
+                }
+            }, null);
+
+        } catch (InterruptedException ie) {
+            LOG.error("Interrupted while reading {}", logSegmentsPath, ie);
+            promise.setException(new DLInterruptedException("Interrupted while checking "
+                    + logSegmentsPath, ie));
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            promise.setException(e);
+        }
+        return promise;
+    }
+
+    //
+    // Create Write Lock
+    //
+
+    @Override
+    public DistributedLock createWriteLock(LogMetadataForWriter metadata) {
+        return new ZKDistributedLock(
+                getLockStateExecutor(true),
+                getLockFactory(true),
+                metadata.getLockPath(),
+                conf.getLockTimeoutMilliSeconds(),
+                statsLogger);
+    }
+
+    //
+    // Create Read Lock
+    //
+
+    private Future<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;
+            }
+        });
+        Optional<String> parentPathShouldNotCreate = Optional.of(logMetadata.getLogRootPath());
+        Utils.zkAsyncCreateFullPathOptimisticRecursive(zooKeeperClient, readLockPath, parentPathShouldNotCreate,
+                new byte[0], zooKeeperClient.getDefaultACL(), CreateMode.PERSISTENT,
+                new org.apache.zookeeper.AsyncCallback.StringCallback() {
+                    @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(
+                                    String.format("Log %s does not exist or has been deleted",
+                                            logMetadata.getFullyQualifiedName())));
+                        } else if (KeeperException.Code.OK.intValue() == rc) {
+                            FutureUtils.setValue(promise, null);
+                            LOG.trace("Created path {}.", path);
+                        } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
+                            FutureUtils.setValue(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));
+                        } else if (DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE == rc) {
+                            FutureUtils.setException(promise, new DLInterruptedException(path));
+                        } else {
+                            FutureUtils.setException(promise, KeeperException.create(KeeperException.Code.get(rc)));
+                        }
+                    }
+                }, null);
+        return promise;
+    }
+
+    @Override
+    public Future<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"));
+                    }
+                });
+            }
+        });
+    }
+
+    //
+    // Create Log
+    //
+
+    static class MetadataIndex {
+        static final int LOG_ROOT_PARENT = 0;
+        static final int LOG_ROOT = 1;
+        static final int MAX_TXID = 2;
+        static final int VERSION = 3;
+        static final int LOCK = 4;
+        static final int READ_LOCK = 5;
+        static final int LOGSEGMENTS = 6;
+        static final int ALLOCATION = 7;
+    }
+
+    static int bytesToInt(byte[] b) {
+        assert b.length >= 4;
+        return b[0] << 24 | b[1] << 16 | b[2] << 8 | b[3];
+    }
+
+    static byte[] intToBytes(int i) {
+        return new byte[]{
+            (byte) (i >> 24),
+            (byte) (i >> 16),
+            (byte) (i >> 8),
+            (byte) (i)};
+    }
+
+    static Future<List<Versioned<byte[]>>> checkLogMetadataPaths(ZooKeeper zk,
+                                                                 String logRootPath,
+                                                                 boolean ownAllocator) {
+        // Note re. persistent lock state initialization: the read lock persistent state (path) is
+        // initialized here but only used in the read handler. The reason is its more convenient and
+        // less error prone to manage all stream structure in one place.
+        final String logRootParentPath = new File(logRootPath).getParent();
+        final String logSegmentsPath = logRootPath + LOGSEGMENTS_PATH;
+        final String maxTxIdPath = logRootPath + MAX_TXID_PATH;
+        final String lockPath = logRootPath + LOCK_PATH;
+        final String readLockPath = logRootPath + READ_LOCK_PATH;
+        final String versionPath = logRootPath + VERSION_PATH;
+        final String allocationPath = logRootPath + ALLOCATION_PATH;
+
+        int numPaths = ownAllocator ? MetadataIndex.ALLOCATION + 1 : MetadataIndex.LOGSEGMENTS + 1;
+        List<Future<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));
+        checkFutures.add(Utils.zkGetData(zk, versionPath, false));
+        checkFutures.add(Utils.zkGetData(zk, lockPath, false));
+        checkFutures.add(Utils.zkGetData(zk, readLockPath, false));
+        checkFutures.add(Utils.zkGetData(zk, logSegmentsPath, false));
+        if (ownAllocator) {
+            checkFutures.add(Utils.zkGetData(zk, allocationPath, false));
+        }
+
+        return Future.collect(checkFutures);
+    }
+
+    static boolean pathExists(Versioned<byte[]> metadata) {
+        return null != metadata.getValue() && null != metadata.getVersion();
+    }
+
+    static void ensureMetadataExist(Versioned<byte[]> metadata) {
+        Preconditions.checkNotNull(metadata.getValue());
+        Preconditions.checkNotNull(metadata.getVersion());
+    }
+
+    static void createMissingMetadata(final ZooKeeper zk,
+                                      final String logRootPath,
+                                      final List<Versioned<byte[]>> metadatas,
+                                      final List<ACL> acl,
+                                      final boolean ownAllocator,
+                                      final boolean createIfNotExists,
+                                      final Promise<List<Versioned<byte[]>>> promise) {
+        final List<byte[]> pathsToCreate = Lists.newArrayListWithExpectedSize(metadatas.size());
+        final List<Op> zkOps = Lists.newArrayListWithExpectedSize(metadatas.size());
+        CreateMode createMode = CreateMode.PERSISTENT;
+
+        // log root parent path
+        if (pathExists(metadatas.get(MetadataIndex.LOG_ROOT_PARENT))) {
+            pathsToCreate.add(null);
+        } else {
+            String logRootParentPath = new File(logRootPath).getParent();
+            pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
+            zkOps.add(Op.create(logRootParentPath, DistributedLogConstants.EMPTY_BYTES, acl, createMode));
+        }
+
+        // log root path
+        if (pathExists(metadatas.get(MetadataIndex.LOG_ROOT))) {
+            pathsToCreate.add(null);
+        } else {
+            pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
+            zkOps.add(Op.create(logRootPath, DistributedLogConstants.EMPTY_BYTES, acl, createMode));
+        }
+
+        // max id
+        if (pathExists(metadatas.get(MetadataIndex.MAX_TXID))) {
+            pathsToCreate.add(null);
+        } else {
+            byte[] zeroTxnIdData = DLUtils.serializeTransactionId(0L);
+            pathsToCreate.add(zeroTxnIdData);
+            zkOps.add(Op.create(logRootPath + MAX_TXID_PATH, zeroTxnIdData, acl, createMode));
+        }
+        // version
+        if (pathExists(metadatas.get(MetadataIndex.VERSION))) {
+            pathsToCreate.add(null);
+        } else {
+            byte[] versionData = intToBytes(LAYOUT_VERSION);
+            pathsToCreate.add(versionData);
+            zkOps.add(Op.create(logRootPath + VERSION_PATH, versionData, acl, createMode));
+        }
+        // lock path
+        if (pathExists(metadatas.get(MetadataIndex.LOCK))) {
+            pathsToCreate.add(null);
+        } else {
+            pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
+            zkOps.add(Op.create(logRootPath + LOCK_PATH, DistributedLogConstants.EMPTY_BYTES, acl, createMode));
+        }
+        // read lock path
+        if (pathExists(metadatas.get(MetadataIndex.READ_LOCK))) {
+            pathsToCreate.add(null);
+        } else {
+            pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
+            zkOps.add(Op.create(logRootPath + READ_LOCK_PATH, DistributedLogConstants.EMPTY_BYTES, acl, createMode));
+        }
+        // log segments path
+        if (pathExists(metadatas.get(MetadataIndex.LOGSEGMENTS))) {
+            pathsToCreate.add(null);
+        } else {
+            byte[] logSegmentsData = DLUtils.serializeLogSegmentSequenceNumber(
+                    DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO);
+            pathsToCreate.add(logSegmentsData);
+            zkOps.add(Op.create(logRootPath + LOGSEGMENTS_PATH, logSegmentsData, acl, createMode));
+        }
+        // allocation path
+        if (ownAllocator) {
+            if (pathExists(metadatas.get(MetadataIndex.ALLOCATION))) {
+                pathsToCreate.add(null);
+            } else {
+                pathsToCreate.add(DistributedLogConstants.EMPTY_BYTES);
+                zkOps.add(Op.create(logRootPath + ALLOCATION_PATH,
+                        DistributedLogConstants.EMPTY_BYTES, acl, createMode));
+            }
+        }
+        if (zkOps.isEmpty()) {
+            // nothing missed
+            promise.setValue(metadatas);
+            return;
+        }
+        if (!createIfNotExists) {
+            promise.setException(new LogNotFoundException("Log " + logRootPath + " not found"));
+            return;
+        }
+
+        zk.multi(zkOps, new AsyncCallback.MultiCallback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx, List<OpResult> resultList) {
+                if (KeeperException.Code.OK.intValue() == rc) {
+                    List<Versioned<byte[]>> finalMetadatas =
+                            Lists.newArrayListWithExpectedSize(metadatas.size());
+                    for (int i = 0; i < pathsToCreate.size(); i++) {
+                        byte[] dataCreated = pathsToCreate.get(i);
+                        if (null == dataCreated) {
+                            finalMetadatas.add(metadatas.get(i));
+                        } else {
+                            finalMetadatas.add(new Versioned<byte[]>(dataCreated, new ZkVersion(0)));
+                        }
+                    }
+                    promise.setValue(finalMetadatas);
+                } else if (KeeperException.Code.NODEEXISTS.intValue() == rc) {
+                    promise.setException(new LogExistsException("Someone just created log "
+                            + logRootPath));
+                } else {
+                    if (LOG.isDebugEnabled()) {
+                        StringBuilder builder = new StringBuilder();
+                        for (OpResult result : resultList) {
+                            if (result instanceof OpResult.ErrorResult) {
+                                OpResult.ErrorResult errorResult = (OpResult.ErrorResult) result;
+                                builder.append(errorResult.getErr()).append(",");
+                            } else {
+                                builder.append(0).append(",");
+                            }
+                        }
+                        String resultCodeList = builder.substring(0, builder.length() - 1);
+                        LOG.debug("Failed to create log, full rc list = {}", resultCodeList);
+                    }
+
+                    promise.setException(new ZKException("Failed to create log " + logRootPath,
+                            KeeperException.Code.get(rc)));
+                }
+            }
+        }, null);
+    }
+
+    static LogMetadataForWriter processLogMetadatas(URI uri,
+                                                    String logName,
+                                                    String logIdentifier,
+                                                    List<Versioned<byte[]>> metadatas,
+                                                    boolean ownAllocator)
+            throws UnexpectedException {
+        try {
+            // max id
+            Versioned<byte[]> maxTxnIdData = metadatas.get(MetadataIndex.MAX_TXID);
+            ensureMetadataExist(maxTxnIdData);
+            // version
+            Versioned<byte[]> versionData = metadatas.get(MetadataIndex.VERSION);
+            ensureMetadataExist(maxTxnIdData);
+            Preconditions.checkArgument(LAYOUT_VERSION == bytesToInt(versionData.getValue()));
+            // lock path
+            ensureMetadataExist(metadatas.get(MetadataIndex.LOCK));
+            // read lock path
+            ensureMetadataExist(metadatas.get(MetadataIndex.READ_LOCK));
+            // max lssn
+            Versioned<byte[]> maxLSSNData = metadatas.get(MetadataIndex.LOGSEGMENTS);
+            ensureMetadataExist(maxLSSNData);
+            try {
+                DLUtils.deserializeLogSegmentSequenceNumber(maxLSSNData.getValue());
+            } catch (NumberFormatException nfe) {
+                throw new UnexpectedException("Invalid max sequence number found in log " + logName, nfe);
+            }
+            // allocation path
+            Versioned<byte[]>  allocationData;
+            if (ownAllocator) {
+                allocationData = metadatas.get(MetadataIndex.ALLOCATION);
+                ensureMetadataExist(allocationData);
+            } else {
+                allocationData = new Versioned<byte[]>(null, null);
+            }
+            return new LogMetadataForWriter(uri, logName, logIdentifier,
+                    maxLSSNData, maxTxnIdData, allocationData);
+        } catch (IllegalArgumentException iae) {
+            throw new UnexpectedException("Invalid log " + logName, iae);
+        } catch (NullPointerException npe) {
+            throw new UnexpectedException("Invalid log " + logName, npe);
+        }
+    }
+
+    static Future<LogMetadataForWriter> getLog(final URI uri,
+                                               final String logName,
+                                               final String logIdentifier,
+                                               final ZooKeeperClient zooKeeperClient,
+                                               final boolean ownAllocator,
+                                               final boolean createIfNotExists) {
+        final String logRootPath = LogMetadata.getLogRootPath(uri, logName, logIdentifier);
+        try {
+            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"));
+        }
+
+        try {
+            final ZooKeeper zk = zooKeeperClient.get();
+            return checkLogMetadataPaths(zk, logRootPath, ownAllocator)
+                    .flatMap(new AbstractFunction1<List<Versioned<byte[]>>, Future<List<Versioned<byte[]>>>>() {
+                        @Override
+                        public Future<List<Versioned<byte[]>>> apply(List<Versioned<byte[]>> metadatas) {
+                            Promise<List<Versioned<byte[]>>> promise =
+                                    new Promise<List<Versioned<byte[]>>>();
+                            createMissingMetadata(zk, logRootPath, metadatas, zooKeeperClient.getDefaultACL(),
+                                    ownAllocator, createIfNotExists, promise);
+                            return promise;
+                        }
+                    }).map(new ExceptionalFunction<List<Versioned<byte[]>>, LogMetadataForWriter>() {
+                        @Override
+                        public LogMetadataForWriter applyE(List<Versioned<byte[]>> metadatas) throws DLException {
+                            return processLogMetadatas(
+                                    uri,
+                                    logName,
+                                    logIdentifier,
+                                    metadatas,
+                                    ownAllocator);
+                        }
+                    });
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            return Future.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));
+        }
+    }
+
+    @Override
+    public Future<LogMetadataForWriter> getLog(final URI uri,
+                                               final String logName,
+                                               final boolean ownAllocator,
+                                               final boolean createIfNotExists) {
+        return getLog(
+                uri,
+                logName,
+                conf.getUnpartitionedStreamName(),
+                zooKeeperClient,
+                ownAllocator,
+                createIfNotExists);
+    }
+
+    //
+    // Delete Log
+    //
+
+    @Override
+    public Future<Void> deleteLog(URI uri, final String logName) {
+        final Promise<Void> promise = new Promise<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,
+                                new ZKException("Encountered zookeeper issue on deleting log stream "
+                                        + logName, KeeperException.Code.get(rc)));
+                        return;
+                    }
+                    FutureUtils.setValue(promise, null);
+                }
+            }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+            FutureUtils.setException(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 "
+                    + logName));
+        } catch (KeeperException e) {
+            FutureUtils.setException(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/metadata/ZkMetadataResolver.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZkMetadataResolver.java
new file mode 100644
index 0000000..1dee147
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/metadata/ZkMetadataResolver.java
@@ -0,0 +1,72 @@
+/**
+ * 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.impl.metadata;
+
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.metadata.DLMetadata;
+import org.apache.distributedlog.metadata.MetadataResolver;
+import org.apache.commons.lang.StringUtils;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.common.PathUtils;
+import org.apache.zookeeper.data.Stat;
+
+import java.io.IOException;
+import java.net.URI;
+
+public class ZkMetadataResolver implements MetadataResolver {
+
+    private final ZooKeeperClient zkc;
+
+    public ZkMetadataResolver(ZooKeeperClient zkc) {
+        this.zkc = zkc;
+    }
+
+    @Override
+    public DLMetadata resolve(URI uri) throws IOException {
+        String dlPath = uri.getPath();
+        PathUtils.validatePath(dlPath);
+        // Normal case the dl metadata is stored in the last segment
+        // so lookup last segment first.
+        String[] parts = StringUtils.split(dlPath, '/');
+        if (null == parts || 0 == parts.length) {
+            throw new IOException("Invalid dlPath to resolve dl metadata : " + dlPath);
+        }
+        for (int i = parts.length; i >= 0; i--) {
+            String pathToResolve = String.format("/%s", StringUtils.join(parts, '/', 0, i));
+            byte[] data;
+            try {
+                data = zkc.get().getData(pathToResolve, false, new Stat());
+            } catch (KeeperException.NoNodeException nne) {
+                continue;
+            } catch (KeeperException ke) {
+                throw new IOException("Fail to resolve dl path : " + pathToResolve);
+            } catch (InterruptedException ie) {
+                throw new IOException("Interrupted when resolving dl path : " + pathToResolve);
+            }
+            if (null == data || data.length == 0) {
+                continue;
+            }
+            try {
+                return DLMetadata.deserialize(uri, data);
+            } catch (IOException ie) {
+                throw new IOException("Failed to deserialize uri : " + uri);
+            }
+        }
+        throw new IOException("No bkdl config bound under dl path : " + dlPath);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/impl/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/package-info.java
new file mode 100644
index 0000000..ecdd194
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * The BookKeeper Based DistributedLog Implementation.
+ */
+package org.apache.distributedlog.impl;
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
new file mode 100644
index 0000000..64abb77
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
@@ -0,0 +1,121 @@
+/**
+ * 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.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 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;
+
+public class ZKSubscriptionStateStore implements SubscriptionStateStore {
+
+    static final Logger logger = LoggerFactory.getLogger(ZKSubscriptionStateStore.class);
+
+    private final ZooKeeperClient zooKeeperClient;
+    private final String zkPath;
+    private AtomicReference<DLSN> lastCommittedPosition = new AtomicReference<DLSN>(null);
+
+    public ZKSubscriptionStateStore(ZooKeeperClient zooKeeperClient, String zkPath) {
+        this.zooKeeperClient = zooKeeperClient;
+        this.zkPath = zkPath;
+    }
+
+    @Override
+    public void close() throws IOException {
+    }
+
+    /**
+     * Get the last committed position stored for this subscription
+     */
+    @Override
+    public Future<DLSN> getLastCommitPosition() {
+        if (null != lastCommittedPosition.get()) {
+            return Future.value(lastCommittedPosition.get());
+        } else {
+            return getLastCommitPositionFromZK();
+        }
+    }
+
+    Future<DLSN> getLastCommitPositionFromZK() {
+        final Promise<DLSN> result = new Promise<DLSN>();
+        try {
+            logger.debug("Reading last commit position from path {}", zkPath);
+            zooKeeperClient.get().getData(zkPath, false, new AsyncCallback.DataCallback() {
+                @Override
+                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);
+                    } else if (KeeperException.Code.OK.intValue() != rc) {
+                        result.setException(KeeperException.create(KeeperException.Code.get(rc), path));
+                    } else {
+                        try {
+                            DLSN dlsn = DLSN.deserialize(new String(data, Charsets.UTF_8));
+                            result.setValue(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);
+                        }
+                    }
+                }
+            }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
+            result.setException(zkce);
+        } catch (InterruptedException ie) {
+            result.setException(new DLInterruptedException("getLastCommitPosition was interrupted", ie));
+        }
+        return result;
+    }
+
+    /**
+     * Advances the position associated with the subscriber
+     *
+     * @param newPosition - new commit position
+     */
+    @Override
+    public Future<BoxedUnit> advanceCommitPosition(DLSN newPosition) {
+        if (null == lastCommittedPosition.get() ||
+            (newPosition.compareTo(lastCommittedPosition.get()) > 0)) {
+            lastCommittedPosition.set(newPosition);
+            return Utils.zkAsyncCreateFullPathOptimisticAndSetData(zooKeeperClient,
+                zkPath, newPosition.serialize().getBytes(Charsets.UTF_8),
+                zooKeeperClient.getDefaultACL(),
+                CreateMode.PERSISTENT);
+        } else {
+            return Future.Done();
+        }
+    }
+}
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
new file mode 100644
index 0000000..d75f5fc
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/impl/subscription/ZKSubscriptionsStore.java
@@ -0,0 +1,164 @@
+/**
+ * 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.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.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * ZooKeeper Based Subscriptions Store.
+ */
+public class ZKSubscriptionsStore implements SubscriptionsStore {
+
+    private final ZooKeeperClient zkc;
+    private final String zkPath;
+    private final ConcurrentMap<String, ZKSubscriptionStateStore> subscribers =
+            new ConcurrentHashMap<String, ZKSubscriptionStateStore>();
+
+    public ZKSubscriptionsStore(ZooKeeperClient zkc, String zkPath) {
+        this.zkc = zkc;
+        this.zkPath = zkPath;
+    }
+
+    private ZKSubscriptionStateStore getSubscriber(String subscriberId) {
+        ZKSubscriptionStateStore ss = subscribers.get(subscriberId);
+        if (ss == null) {
+            ZKSubscriptionStateStore newSS = new ZKSubscriptionStateStore(zkc,
+                getSubscriberZKPath(subscriberId));
+            ZKSubscriptionStateStore oldSS = subscribers.putIfAbsent(subscriberId, newSS);
+            if (oldSS == null) {
+                ss = newSS;
+            } else {
+                try {
+                    newSS.close();
+                } catch (IOException e) {
+                    // ignore the exception
+                }
+                ss = oldSS;
+            }
+        }
+        return ss;
+    }
+
+    private String getSubscriberZKPath(String subscriberId) {
+        return String.format("%s/%s", zkPath, subscriberId);
+    }
+
+    @Override
+    public Future<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>>();
+        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>());
+                    } else if (KeeperException.Code.OK.intValue() != rc) {
+                        result.setException(KeeperException.create(KeeperException.Code.get(rc), path));
+                    } else {
+                        getLastCommitPositions(result, children);
+                    }
+                }
+            }, null);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
+            result.setException(zkce);
+        } catch (InterruptedException ie) {
+            result.setException(new DLInterruptedException("getLastCommitPositions was interrupted", ie));
+        }
+        return result;
+    }
+
+    private void getLastCommitPositions(final Promise<Map<String, DLSN>> result,
+                                        List<String> subscribers) {
+        List<Future<Pair<String, DLSN>>> futures =
+                new ArrayList<Future<Pair<String, DLSN>>>(subscribers.size());
+        for (String s : subscribers) {
+            final String subscriber = s;
+            Future<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);
+                            }
+                        });
+            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;
+                }
+            });
+    }
+
+    @Override
+    public Future<BoxedUnit> advanceCommitPosition(String subscriberId, DLSN newPosition) {
+        return getSubscriber(subscriberId).advanceCommitPosition(newPosition);
+    }
+
+    @Override
+    public Future<Boolean> deleteSubscriber(String subscriberId) {
+        subscribers.remove(subscriberId);
+        String path = getSubscriberZKPath(subscriberId);
+        return Utils.zkDeleteIfNotExist(zkc, path, new ZkVersion(-1));
+    }
+
+    @Override
+    public void close() throws IOException {
+        // no-op
+        for (SubscriptionStateStore store : subscribers.values()) {
+            store.close();
+        }
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncFailureInjector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncFailureInjector.java
new file mode 100644
index 0000000..6590bec
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncFailureInjector.java
@@ -0,0 +1,131 @@
+/**
+ * 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.injector;
+
+/**
+ * Failure Injector that works in asynchronous way
+ */
+public interface AsyncFailureInjector {
+
+    AsyncFailureInjector NULL = new AsyncFailureInjector() {
+        @Override
+        public void injectErrors(boolean enabled) {
+            // no-op
+        }
+
+        @Override
+        public boolean shouldInjectErrors() {
+            return false;
+        }
+
+        @Override
+        public void injectDelays(boolean enabled) {
+            // no-op
+        }
+
+        @Override
+        public boolean shouldInjectDelays() {
+            return false;
+        }
+
+        @Override
+        public int getInjectedDelayMs() {
+            return 0;
+        }
+
+        @Override
+        public void injectStops(boolean enabled) {
+            // no-op
+        }
+
+        @Override
+        public boolean shouldInjectStops() {
+            return false;
+        }
+
+        @Override
+        public boolean shouldInjectCorruption(long startEntryId, long endEntryId) {
+            return false;
+        }
+
+        @Override
+        public String toString() {
+            return "NULL";
+        }
+    };
+
+    /**
+     * Enable or disable error injection.
+     *
+     * @param enabled
+     *          flag to enable or disable error injection.
+     */
+    void injectErrors(boolean enabled);
+
+    /**
+     * Return the flag indicating if should inject errors.
+     *
+     * @return true to inject errors otherwise false.
+     */
+    boolean shouldInjectErrors();
+
+    /**
+     * Enable or disable delay injection.
+     *
+     * @param enabled
+     *          flag to enable or disable delay injection.
+     */
+    void injectDelays(boolean enabled);
+
+    /**
+     * Return the flag indicating if should inject delays.
+     *
+     * @return true to inject delays otherwise false.
+     */
+    boolean shouldInjectDelays();
+
+    /**
+     * Return the injected delay in milliseconds.
+     *
+     * @return the injected delay in milliseconds.
+     */
+    int getInjectedDelayMs();
+
+    /**
+     * Enable or disable injecting stops. This could be used
+     * for simulating stopping an action.
+     */
+    void injectStops(boolean enabled);
+
+    /**
+     * Return the flag indicating if should inject stops.
+     *
+     * @return true to inject stops otherwise false.
+     */
+    boolean shouldInjectStops();
+
+    /**
+     * Return the flag indicating if should inject corruption.
+     *
+     * @param startEntryId the start entry id
+     * @param endEntryId the end entry id
+     * @return true to inject corruption otherwise false.
+     */
+    boolean shouldInjectCorruption(long startEntryId, long endEntryId);
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncRandomFailureInjector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncRandomFailureInjector.java
new file mode 100644
index 0000000..c6c675d
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/AsyncRandomFailureInjector.java
@@ -0,0 +1,186 @@
+/**
+ * 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.injector;
+
+import org.apache.distributedlog.util.Utils;
+
+import java.util.Random;
+
+/**
+ * Failure injector based on {@link java.util.Random}
+ */
+public class AsyncRandomFailureInjector implements AsyncFailureInjector {
+
+    private static final Random random = new Random(System.currentTimeMillis());
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    public static class Builder {
+
+        private boolean _simulateDelays = false;
+        private boolean _simulateErrors = false;
+        private boolean _simulateStops = false;
+        private boolean _simulateCorruption = false;
+        private int _injectedDelayPercent = 0;
+        private int _injectedErrorPercent = 0;
+        private int _injectedStopPercent = 0;
+        private int _maxInjectedDelayMs = Integer.MAX_VALUE;
+
+        private Builder() {}
+
+        public Builder injectDelays(boolean simulateDelays,
+                                    int injectedDelayPercent,
+                                    int maxInjectedDelayMs) {
+            this._simulateDelays = simulateDelays;
+            this._injectedDelayPercent = injectedDelayPercent;
+            this._maxInjectedDelayMs = maxInjectedDelayMs;
+            return this;
+        }
+
+        public Builder injectErrors(boolean simulateErrors,
+                                    int injectedErrorPercent) {
+            this._simulateErrors = simulateErrors;
+            this._injectedErrorPercent = injectedErrorPercent;
+            return this;
+        }
+
+        public Builder injectCorruption(boolean simulateCorruption) {
+            this._simulateCorruption = simulateCorruption;
+            return this;
+        }
+
+        public Builder injectStops(boolean simulateStops,
+                                   int injectedStopPercent) {
+            this._simulateStops = simulateStops;
+            this._injectedStopPercent = injectedStopPercent;
+            return this;
+        }
+
+        public AsyncFailureInjector build() {
+            return new AsyncRandomFailureInjector(
+                    _simulateDelays,
+                    _injectedDelayPercent,
+                    _maxInjectedDelayMs,
+                    _simulateErrors,
+                    _injectedErrorPercent,
+                    _simulateStops,
+                    _injectedStopPercent,
+                    _simulateCorruption);
+        }
+
+    }
+
+    private boolean simulateDelays;
+    private boolean simulateErrors;
+    private boolean simulateStops;
+    private boolean simulateCorruption;
+    private final int injectedDelayPercent;
+    private final int injectedErrorPercent;
+    private final int injectedStopPercent;
+    private final int maxInjectedDelayMs;
+
+    private AsyncRandomFailureInjector(boolean simulateDelays,
+                                       int injectedDelayPercent,
+                                       int maxInjectedDelayMs,
+                                       boolean simulateErrors,
+                                       int injectedErrorPercent,
+                                       boolean simulateStops,
+                                       int injectedStopPercent,
+                                       boolean simulateCorruption) {
+        this.simulateDelays = simulateDelays;
+        this.injectedDelayPercent = injectedDelayPercent;
+        this.maxInjectedDelayMs = maxInjectedDelayMs;
+        this.simulateErrors = simulateErrors;
+        this.injectedErrorPercent = injectedErrorPercent;
+        this.simulateStops = simulateStops;
+        this.injectedStopPercent = injectedStopPercent;
+        this.simulateCorruption = simulateCorruption;
+    }
+
+    @Override
+    public void injectErrors(boolean enabled) {
+        this.simulateErrors = enabled;
+    }
+
+    @Override
+    public boolean shouldInjectErrors() {
+        return simulateErrors && Utils.randomPercent(injectedErrorPercent);
+    }
+
+    @Override
+    public void injectDelays(boolean enabled) {
+        this.simulateDelays = enabled;
+    }
+
+    @Override
+    public boolean shouldInjectDelays() {
+        return simulateDelays && Utils.randomPercent(injectedDelayPercent);
+    }
+
+    @Override
+    public int getInjectedDelayMs() {
+        if (maxInjectedDelayMs > 0) {
+            return random.nextInt(maxInjectedDelayMs);
+        }
+        return 0;
+    }
+
+    @Override
+    public void injectStops(boolean enabled) {
+        this.simulateStops = enabled;
+    }
+
+    @Override
+    public boolean shouldInjectStops() {
+        return simulateStops && Utils.randomPercent(injectedStopPercent);
+    }
+
+    @Override
+    public boolean shouldInjectCorruption(long startEntryId, long endEntryId) {
+        if (!simulateCorruption) {
+            return false;
+        }
+        if (startEntryId == endEntryId) {
+            return startEntryId % 10 == 0;
+        }
+        for (long i = startEntryId; i <= endEntryId; i++) {
+            if (i % 10 == 0) {
+                return true;
+            }
+        }
+        return false;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("FailureInjector[");
+        sb.append("errors=(").append(simulateErrors).append(", pct=")
+                .append(injectedErrorPercent).append("), ");
+        sb.append("delays=(").append(simulateDelays).append(", pct=")
+                .append(injectedDelayPercent).append(", max=")
+                .append(maxInjectedDelayMs).append("), ");
+        sb.append("stops=(").append(simulateStops).append(", pct=")
+                .append(injectedStopPercent).append(")");
+        sb.append("corruption=(").append(simulateCorruption).append(")");
+        sb.append("]");
+        return sb.toString();
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/FailureInjector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/FailureInjector.java
new file mode 100644
index 0000000..9544a6c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/FailureInjector.java
@@ -0,0 +1,37 @@
+/**
+ * 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.injector;
+
+/**
+ * Failure injector.
+ */
+public interface FailureInjector {
+
+    /**
+     * No-op failure injector, which does nothing.
+     */
+    public static FailureInjector NULL = new FailureInjector() {
+        @Override
+        public void inject() {
+            // no-op;
+        }
+    };
+
+    // inject failures
+    void inject();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/RandomDelayFailureInjector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/RandomDelayFailureInjector.java
new file mode 100644
index 0000000..2350b36
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/RandomDelayFailureInjector.java
@@ -0,0 +1,60 @@
+/**
+ * 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.injector;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Injector that injects random delays
+ */
+public class RandomDelayFailureInjector implements FailureInjector {
+
+    private static final Logger LOG = LoggerFactory.getLogger(RandomDelayFailureInjector.class);
+
+    private final DynamicDistributedLogConfiguration dynConf;
+
+    public RandomDelayFailureInjector(DynamicDistributedLogConfiguration dynConf) {
+        this.dynConf = dynConf;
+    }
+
+    private int delayMs() {
+        return dynConf.getEIInjectedWriteDelayMs();
+    }
+
+    private double delayPct() {
+        return dynConf.getEIInjectedWriteDelayPercent();
+    }
+
+    private boolean enabled() {
+        return delayMs() > 0 && delayPct() > 0;
+    }
+
+    @Override
+    public void inject() {
+        try {
+            if (enabled() && Utils.randomPercent(delayPct())) {
+                Thread.sleep(delayMs());
+            }
+        } catch (InterruptedException ex) {
+            LOG.warn("delay was interrupted ", ex);
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/injector/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/package-info.java
new file mode 100644
index 0000000..d76dbc6
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/injector/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * DistributedLog Failure Injection
+ */
+package org.apache.distributedlog.injector;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/Abortable.java b/distributedlog-core/src/main/java/org/apache/distributedlog/io/Abortable.java
new file mode 100644
index 0000000..b2b430d
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/io/Abortable.java
@@ -0,0 +1,41 @@
+/**
+ * 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.io;
+
+import java.io.IOException;
+
+/**
+ * An {@code Abortable} is a source or destination of data that can be aborted.
+ * The abort method is invoked to release resources that the object is holding
+ * (such as open files). The abort happens when the object is in an error state,
+ * which it couldn't be closed gracefully.
+ *
+ * @see java.io.Closeable
+ * @since 0.3.32
+ */
+public interface Abortable {
+
+    /**
+     * Aborts the object and releases any resources associated with it.
+     * If the object is already aborted then invoking this method has no
+     * effect.
+     *
+     * @throws IOException if an I/O error occurs.
+     */
+    public void abort() throws IOException;
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/Abortables.java b/distributedlog-core/src/main/java/org/apache/distributedlog/io/Abortables.java
new file mode 100644
index 0000000..a4838b1
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/io/Abortables.java
@@ -0,0 +1,183 @@
+/**
+ * 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.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.ExecutorService;
+
+/**
+ * Utility methods for working with {@link Abortable} objects.
+ *
+ * @since 0.3.32
+ */
+public final class Abortables {
+
+    static final Logger logger = LoggerFactory.getLogger(Abortables.class);
+
+    private Abortables() {}
+
+    public static Future<Void> asyncAbort(@Nullable AsyncAbortable abortable,
+                                          boolean swallowIOException) {
+        if (null == abortable) {
+            return Future.Void();
+        } else if (swallowIOException) {
+            return FutureUtils.ignore(abortable.asyncAbort());
+        } else {
+            return abortable.asyncAbort();
+        }
+    }
+
+    /**
+     * Aborts a {@link Abortable}, with control over whether an {@link IOException} may be thrown.
+     * This is primarily useful in a finally block, where a thrown exception needs to be logged but
+     * not propagated (otherwise the original exception will be lost).
+     *
+     * <p>If {@code swallowIOException} is true then we never throw {@code IOException} but merely log it.
+     *
+     * <p>Example: <pre>   {@code
+     *
+     *   public void abortStreamNicely() throws IOException {
+     *      SomeStream stream = new SomeStream("foo");
+     *      try {
+     *          // ... code which does something with the stream ...
+     *      } catch (IOException ioe) {
+     *          // If an exception occurs, we might abort the stream.
+     *          Abortables.abort(stream, true);
+     *      }
+     *   }}</pre>
+     *
+     * @param abortable the {@code Abortable} 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 Abortable abortable,
+                             boolean swallowIOException)
+        throws IOException {
+        if (null == abortable) {
+            return;
+        }
+        try {
+            abortable.abort();
+        } catch (IOException ioe) {
+            if (swallowIOException) {
+                logger.warn("IOException thrown while aborting Abortable {} : ", abortable, ioe);
+            } else {
+                throw ioe;
+            }
+        }
+    }
+
+    /**
+     * 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}
+     * @see #abort(Abortable, boolean)
+     */
+    public static void abort(@Nullable AsyncAbortable abortable,
+                             boolean swallowIOException)
+            throws IOException {
+        if (null == abortable) {
+            return;
+        }
+        try {
+            FutureUtils.result(abortable.asyncAbort());
+        } catch (IOException ioe) {
+            if (swallowIOException) {
+                logger.warn("IOException thrown while aborting Abortable {} : ", abortable, ioe);
+            } else {
+                throw ioe;
+            }
+        }
+    }
+
+    /**
+     * 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
+     * I/O resource, it should generally be safe in the case of a resource that's being used only for
+     * reading.
+     *
+     * @param abortable the {@code Abortable} to be closed, or {@code null} in which case this method
+     *                  does nothing.
+     */
+    public static void abortQuietly(@Nullable Abortable abortable) {
+        try {
+            abort(abortable, true);
+        } catch (IOException e) {
+            logger.error("Unexpected IOException thrown while aborting Abortable {} quietly : ", abortable, e);
+        }
+    }
+
+    /**
+     * 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
+     * I/O resource, it should generally be safe in the case of a resource that's being used only for
+     * reading.
+     *
+     * @param abortable the {@code AsyncAbortable} to be closed, or {@code null} in which case this method
+     *                  does nothing.
+     */
+    public static void abortQuietly(@Nullable AsyncAbortable abortable) {
+        try {
+            abort(abortable, true);
+        } catch (IOException e) {
+            logger.error("Unexpected IOException thrown while aborting Abortable {} quietly : ", abortable, e);
+        }
+    }
+
+    /**
+     * Abort the abortables in sequence.
+     *
+     * @param executorService
+     *          executor service to execute
+     * @param abortables
+     *          abortables to abort
+     * @return future represents the abort future
+     */
+    public static Future<Void> abortSequence(ExecutorService executorService,
+                                             AsyncAbortable... abortables) {
+        List<AsyncAbortable> abortableList = Lists.newArrayListWithExpectedSize(abortables.length);
+        for (AsyncAbortable abortable : abortables) {
+            if (null == abortable) {
+                abortableList.add(AsyncAbortable.NULL);
+            } else {
+                abortableList.add(abortable);
+            }
+        }
+        return FutureUtils.processList(
+                abortableList,
+                AsyncAbortable.ABORT_FUNC,
+                executorService).map(VoidFunctions.LIST_TO_VOID_FUNC);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java b/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java
new file mode 100644
index 0000000..7ec26a1
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncAbortable.java
@@ -0,0 +1,57 @@
+/**
+ * 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.io;
+
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+
+/**
+ * An {@code Abortable} is a source or destination of data that can be aborted.
+ * The abort method is invoked to release resources that the object is holding
+ * (such as open files). The abort happens when the object is in an error state,
+ * which it couldn't be closed gracefully.
+ *
+ * @see AsyncCloseable
+ * @see Abortable
+ * @since 0.3.43
+ */
+public interface AsyncAbortable {
+
+    Function<AsyncAbortable, Future<Void>> ABORT_FUNC = new Function<AsyncAbortable, Future<Void>>() {
+        @Override
+        public Future<Void> apply(AsyncAbortable abortable) {
+            return abortable.asyncAbort();
+        }
+    };
+
+    AsyncAbortable NULL = new AsyncAbortable() {
+        @Override
+        public Future<Void> asyncAbort() {
+            return Future.Void();
+        }
+    };
+
+    /**
+     * 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/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java b/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java
new file mode 100644
index 0000000..2bf0119
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncCloseable.java
@@ -0,0 +1,60 @@
+/**
+ * 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.io;
+
+import org.apache.distributedlog.util.FutureUtils;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+
+/**
+ * A {@code AsyncCloseable} is a source or destination of data that can be closed asynchronously.
+ * The close method is invoked to release resources that the object is
+ * holding (such as open files).
+ */
+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, Future<Void>> CLOSE_FUNC_IGNORE_ERRORS = new Function<AsyncCloseable, Future<Void>>() {
+        @Override
+        public Future<Void> apply(AsyncCloseable closeable) {
+            return FutureUtils.ignore(closeable.asyncClose());
+        }
+    };
+
+    AsyncCloseable NULL = new AsyncCloseable() {
+        @Override
+        public Future<Void> asyncClose() {
+            return Future.Void();
+        }
+    };
+
+    /**
+     * 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/src/main/java/org/apache/distributedlog/io/AsyncDeleteable.java b/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncDeleteable.java
new file mode 100644
index 0000000..046c731
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/io/AsyncDeleteable.java
@@ -0,0 +1,34 @@
+/**
+ * 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.io;
+
+import com.twitter.util.Future;
+
+/**
+ * A {@code AsyncDeleteable} is a source or destination of data that can be deleted asynchronously.
+ * This delete method is invoked to delete the source.
+ */
+public interface AsyncDeleteable {
+    /**
+     * Releases any system resources associated with this and delete the source. If the source is
+     * already deleted then invoking this method has no effect.
+     *
+     * @return future representing the deletion result.
+     */
+    Future<Void> delete();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
new file mode 100644
index 0000000..eb81cfe
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/io/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * IO Utils for distributedlog
+ */
+package org.apache.distributedlog.io;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ChainedRequestLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ChainedRequestLimiter.java
new file mode 100644
index 0000000..fdad69b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ChainedRequestLimiter.java
@@ -0,0 +1,76 @@
+/**
+ * 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.limiter;
+
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.ImmutableList;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+
+import java.util.concurrent.TimeUnit;
+
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * Chain request limiters for easier management of multi limiter policy.
+ */
+public class ChainedRequestLimiter<Request> implements RequestLimiter<Request> {
+    private final ImmutableList<RequestLimiter<Request>> limiters;
+    private final OpStatsLogger applyTime;
+
+    public static class Builder<Request> {
+        private final ImmutableList.Builder<RequestLimiter<Request>> limitersBuilder;
+        private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
+
+        public Builder() {
+            this.limitersBuilder = new ImmutableList.Builder<RequestLimiter<Request>>();
+        }
+
+        public Builder<Request> addLimiter(RequestLimiter<Request> limiter) {
+            this.limitersBuilder.add(limiter);
+            return this;
+        }
+
+        public Builder<Request> statsLogger(StatsLogger statsLogger) {
+            this.statsLogger = statsLogger;
+            return this;
+        }
+
+        public ChainedRequestLimiter<Request> build() {
+            return new ChainedRequestLimiter<Request>(limitersBuilder.build(), statsLogger);
+        }
+    }
+
+    private ChainedRequestLimiter(ImmutableList<RequestLimiter<Request>> limiters,
+                                  StatsLogger statsLogger) {
+        this.limiters = limiters;
+        this.applyTime = statsLogger.getOpStatsLogger("apply");
+    }
+
+    public void apply(Request request) throws OverCapacityException {
+        Stopwatch stopwatch = Stopwatch.createStarted();
+        try {
+            for (RequestLimiter<Request> limiter : limiters) {
+                limiter.apply(request);
+            }
+        } finally {
+            applyTime.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+        }
+    }
+}
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
new file mode 100644
index 0000000..95165ef
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/ComposableRequestLimiter.java
@@ -0,0 +1,73 @@
+/**
+ * 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.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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Collect rate limiter implementation, cost(Request), overlimit, etc. behavior.
+ */
+public class ComposableRequestLimiter<Request> implements RequestLimiter<Request> {
+    protected static final Logger LOG = LoggerFactory.getLogger(ComposableRequestLimiter.class);
+
+    private final RateLimiter limiter;
+    private final OverlimitFunction<Request> overlimitFunction;
+    private final CostFunction<Request> costFunction;
+    private final Counter overlimitCounter;
+
+    static public interface OverlimitFunction<Request> {
+        void apply(Request request) throws OverCapacityException;
+    }
+    static public interface CostFunction<Request> {
+        int apply(Request request);
+    }
+
+    public ComposableRequestLimiter(
+            RateLimiter limiter,
+            OverlimitFunction<Request> overlimitFunction,
+            CostFunction<Request> costFunction,
+            StatsLogger statsLogger) {
+        Preconditions.checkNotNull(limiter);
+        Preconditions.checkNotNull(overlimitFunction);
+        Preconditions.checkNotNull(costFunction);
+        this.limiter = limiter;
+        this.overlimitFunction = overlimitFunction;
+        this.costFunction = costFunction;
+        this.overlimitCounter = statsLogger.getCounter("overlimit");
+    }
+
+    @Override
+    public void apply(Request request) throws OverCapacityException {
+        int permits = costFunction.apply(request);
+        if (!limiter.acquire(permits)) {
+            overlimitCounter.inc();
+            overlimitFunction.apply(request);
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/GuavaRateLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/GuavaRateLimiter.java
new file mode 100644
index 0000000..f18e5a3
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/GuavaRateLimiter.java
@@ -0,0 +1,58 @@
+/**
+ * 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.limiter;
+
+import com.google.common.base.Preconditions;
+
+/**
+ * Wrap a guava limiter in a simple interface to make testing easier.
+ * Notes:
+ * 1. Negative limit translates into (virtually) unlimited.
+ * 2. Calling acquire with permits == 0 translates into no acquire.
+ */
+public class GuavaRateLimiter implements RateLimiter {
+    com.google.common.util.concurrent.RateLimiter limiter;
+
+    public static RateLimiter of(int limit) {
+        if (limit == 0) {
+            return RateLimiter.REJECT;
+        } else if (limit < 0) {
+            return RateLimiter.ACCEPT;
+        } else {
+            return new GuavaRateLimiter(limit);
+        }
+    }
+
+    public GuavaRateLimiter(int limit) {
+        double effectiveLimit = limit;
+        if (limit < 0) {
+            effectiveLimit = Double.POSITIVE_INFINITY;
+        }
+        this.limiter = com.google.common.util.concurrent.RateLimiter.create(effectiveLimit);
+    }
+
+    @Override
+    public boolean acquire(int permits) {
+        Preconditions.checkState(permits >= 0);
+        if (permits > 0) {
+            return limiter.tryAcquire(permits);
+        } else {
+            return true;
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RateLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RateLimiter.java
new file mode 100644
index 0000000..d1ddf0d
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RateLimiter.java
@@ -0,0 +1,49 @@
+/**
+ * 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.limiter;
+
+/**
+ * Simple interface for a rate limiter used by RequestLimiter.
+ */
+public interface RateLimiter {
+
+    public static final RateLimiter REJECT = new RateLimiter() {
+        @Override
+        public boolean acquire(int permits) {
+            return false;
+        }
+    };
+
+    public static final RateLimiter ACCEPT = new RateLimiter() {
+        @Override
+        public boolean acquire(int permits) {
+            return true;
+        }
+    };
+
+    public static abstract class Builder {
+        public abstract RateLimiter build();
+    }
+
+    /**
+     * Try to acquire a certain number of permits.
+     *
+     * @param permits number of permits to acquire
+     */
+    boolean acquire(int permits);
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RequestLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RequestLimiter.java
new file mode 100644
index 0000000..73bc953
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/RequestLimiter.java
@@ -0,0 +1,24 @@
+/**
+ * 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.limiter;
+
+import org.apache.distributedlog.exceptions.OverCapacityException;
+
+public interface RequestLimiter<Request> {
+    public void apply(Request request) throws OverCapacityException;
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/package-info.java
new file mode 100644
index 0000000..b41ad6d
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/limiter/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Rate limiting for distributedlog
+ */
+package org.apache.distributedlog.limiter;
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
new file mode 100644
index 0000000..986678c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLock.java
@@ -0,0 +1,54 @@
+/**
+ * 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.lock;
+
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.io.AsyncCloseable;
+import com.twitter.util.Future;
+
+/**
+ * Interface for distributed locking
+ */
+public interface DistributedLock extends AsyncCloseable {
+
+    /**
+     * Asynchronously acquire the lock.
+     *
+     * @return future represents the acquire result.
+     */
+    Future<? extends DistributedLock> asyncAcquire();
+
+    /**
+     * Check if hold lock. If it doesn't, then re-acquire the lock.
+     *
+     * @throws LockingException if the lock attempt fails
+     * @see #checkOwnership()
+     */
+    void checkOwnershipAndReacquire() throws LockingException;
+
+    /**
+     * Check if the lock is held. If not, error out and do not re-acquire.
+     * Use this in cases where there are many waiters by default and re-acquire
+     * is unlikely to succeed.
+     *
+     * @throws LockingException if we lost the ownership
+     * @see #checkOwnershipAndReacquire()
+     */
+    void checkOwnership() throws LockingException;
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLockContext.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLockContext.java
new file mode 100644
index 0000000..4f84bb4
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/DistributedLockContext.java
@@ -0,0 +1,43 @@
+/**
+ * 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.lock;
+
+import org.apache.commons.lang3.tuple.Pair;
+
+import java.util.HashSet;
+import java.util.Set;
+
+class DistributedLockContext {
+    private final Set<Pair<String, Long>> lockIds;
+
+    DistributedLockContext() {
+        this.lockIds = new HashSet<Pair<String, Long>>();
+    }
+
+    synchronized void addLockId(Pair<String, Long> lockId) {
+        this.lockIds.add(lockId);
+    }
+
+    synchronized void clearLockIds() {
+        this.lockIds.clear();
+    }
+
+    synchronized boolean hasLockId(Pair<String, Long> lockId) {
+        return this.lockIds.contains(lockId);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/EpochChangedException.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/EpochChangedException.java
new file mode 100644
index 0000000..2d989e2
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/EpochChangedException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.lock;
+
+import org.apache.distributedlog.exceptions.LockingException;
+
+/**
+ * Exception indicates that epoch already changed when executing a given
+ * {@link LockAction}.
+ */
+public class EpochChangedException extends LockingException {
+
+    private static final long serialVersionUID = 8775257025963870331L;
+
+    public EpochChangedException(String lockPath, int expectedEpoch, int currentEpoch) {
+        super(lockPath, "lock " + lockPath + " already moved to epoch " + currentEpoch + ", expected " + expectedEpoch);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockAction.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockAction.java
new file mode 100644
index 0000000..767088f
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockAction.java
@@ -0,0 +1,36 @@
+/**
+ * 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.lock;
+
+/**
+ * Lock Action
+ */
+interface LockAction {
+
+    /**
+     * Execute a lock action
+     */
+    void execute();
+
+    /**
+     * Get lock action name.
+     *
+     * @return lock action name
+     */
+    String getActionName();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockClosedException.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockClosedException.java
new file mode 100644
index 0000000..fe6b254
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockClosedException.java
@@ -0,0 +1,38 @@
+/**
+ * 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.lock;
+
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.lock.ZKSessionLock.State;
+import org.apache.commons.lang3.tuple.Pair;
+
+/**
+ * Exception indicates that the lock was closed (unlocked) before the lock request could complete.
+ */
+public class LockClosedException extends LockingException {
+
+    private static final long serialVersionUID = 8775257025963470331L;
+
+    public LockClosedException(String lockPath, String msg) {
+        super(lockPath, msg);
+    }
+
+    public LockClosedException(String lockPath, Pair<String, Long> lockId, State currentState) {
+        super(lockPath, "lock at path " + lockPath + " with id " + lockId + " closed early in state : " + currentState);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockListener.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockListener.java
new file mode 100644
index 0000000..2a528e9
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockListener.java
@@ -0,0 +1,28 @@
+/**
+ * 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.lock;
+
+/**
+ * Listener on lock state changes
+ */
+interface LockListener {
+    /**
+     * Triggered when a lock is changed from CLAIMED to EXPIRED.
+     */
+    void onExpired();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockSessionExpiredException.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockSessionExpiredException.java
new file mode 100644
index 0000000..62fef20
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockSessionExpiredException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.lock;
+
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.lock.ZKSessionLock.State;
+import org.apache.commons.lang3.tuple.Pair;
+
+/**
+ * Exception indicates that the lock's zookeeper session was expired before the lock request could complete.
+ */
+public class LockSessionExpiredException extends LockingException {
+
+    private static final long serialVersionUID = 8775253025963470331L;
+
+    public LockSessionExpiredException(String lockPath, Pair<String, Long> lockId, State currentState) {
+        super(lockPath, "lock at path " + lockPath + " with id " + lockId + " expired early in state : " + currentState);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockStateChangedException.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockStateChangedException.java
new file mode 100644
index 0000000..362f602
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockStateChangedException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.lock;
+
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.lock.ZKSessionLock.State;
+import org.apache.commons.lang3.tuple.Pair;
+
+/**
+ * Exception thrown when lock state changed
+ */
+public class LockStateChangedException extends LockingException {
+
+    private static final long serialVersionUID = -3770866789942102262L;
+
+    LockStateChangedException(String lockPath, Pair<String, Long> lockId,
+                              State expectedState, State currentState) {
+        super(lockPath, "Lock state of " + lockId + " for " + lockPath + " has changed : expected "
+                + expectedState + ", but " + currentState);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockTimeoutException.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockTimeoutException.java
new file mode 100644
index 0000000..33f3582
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockTimeoutException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.lock;
+
+import org.apache.distributedlog.exceptions.LockingException;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Exception thrown when acquiring lock timeout
+ */
+public class LockTimeoutException extends LockingException {
+
+    private static final long serialVersionUID = -3837638877423323820L;
+
+    LockTimeoutException(String lockPath, long timeout, TimeUnit unit) {
+        super(lockPath, "Locking " + lockPath + " timeout in " + timeout + " " + unit);
+    }
+}
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
new file mode 100644
index 0000000..b70098e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/LockWaiter.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.lock;
+
+import com.twitter.util.Await;
+import com.twitter.util.Duration;
+import com.twitter.util.Future;
+import com.twitter.util.Timer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Lock waiter represents the attempt that application tries to lock.
+ */
+public class LockWaiter {
+
+    private static final Logger logger = LoggerFactory.getLogger(LockWaiter.class);
+
+    private final String lockId;
+    private final String currentOwner;
+    private final Future<Boolean> acquireFuture;
+
+    public LockWaiter(String lockId,
+                      String currentOwner,
+                      Future<Boolean> acquireFuture) {
+        this.lockId = lockId;
+        this.currentOwner = currentOwner;
+        this.acquireFuture = acquireFuture;
+    }
+
+    /**
+     * Return the lock id of the waiter.
+     *
+     * @return lock id of the waiter
+     */
+    public String getId() {
+        return lockId;
+    }
+
+    /**
+     * Return the owner that observed when waiter is waiting.
+     *
+     * @return the owner that observed when waiter is waiting
+     */
+    public String getCurrentOwner() {
+        return currentOwner;
+    }
+
+    /**
+     * Return the future representing the waiting result.
+     *
+     * <p>If the future is interrupted (e.g. {@link Future#within(Duration, Timer)}),
+     * the waiter will automatically clean up its waiting state.
+     *
+     * @return the future representing the acquire result.
+     */
+    public Future<Boolean> getAcquireFuture() {
+        return acquireFuture;
+    }
+
+    /**
+     * Wait for the acquire result.
+     *
+     * @return true if acquired successfully, otherwise false.
+     */
+    public boolean waitForAcquireQuietly() {
+        boolean success = false;
+        try {
+            success = Await.result(acquireFuture);
+        } catch (InterruptedException ie) {
+            Thread.currentThread().interrupt();
+        } catch (LockTimeoutException lte) {
+            logger.debug("Timeout on lock acquiring", lte);
+        } catch (Exception 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
new file mode 100644
index 0000000..88abffa
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/NopDistributedLock.java
@@ -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.
+ */
+package org.apache.distributedlog.lock;
+
+import org.apache.distributedlog.exceptions.LockingException;
+import com.twitter.util.Future;
+
+/**
+ * An implementation of {@link DistributedLock} which does nothing.
+ */
+public class NopDistributedLock implements DistributedLock {
+
+    public static final DistributedLock INSTANCE = new NopDistributedLock();
+
+    private NopDistributedLock() {}
+
+    @Override
+    public Future<? extends DistributedLock> asyncAcquire() {
+        return Future.value(this);
+    }
+
+    @Override
+    public void checkOwnershipAndReacquire() throws LockingException {
+        // no-op
+    }
+
+    @Override
+    public void checkOwnership() throws LockingException {
+        // no-op
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        return Future.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
new file mode 100644
index 0000000..8aec2c0
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLock.java
@@ -0,0 +1,126 @@
+/**
+ * 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.lock;
+
+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.
+ * <p>The lock is only alive during a given period. It should
+ * be not usable if the lock is expired.
+ * <p>Listener could be registered by {@link #setLockListener(LockListener)}
+ * to receive state changes of the lock.
+ */
+public interface SessionLock {
+
+    /**
+     * Set lock listener for lock state changes.
+     * <p>Typically a listener should be set before try locking.
+     *
+     * @param lockListener
+     *          lock listener for state changes.
+     */
+    SessionLock setLockListener(LockListener lockListener);
+
+    /**
+     * Whether the lock is held or not?
+     *
+     * @return true if the lock is held, otherwise false.
+     */
+    boolean isLockHeld();
+
+    /**
+     * Whether the lock is expired or not?
+     * <p>If a lock is expired, it will not be reusable any more. Because it is an one-time lock.
+     *
+     * @return true if the lock is expired, otherwise false.
+     */
+    boolean isLockExpired();
+
+    /**
+     * Acquire the lock if it is free within given waiting time.
+     * <p>
+     * Calling this method will attempt to acquire the lock. If the lock
+     * is already acquired by others, the caller will wait for <i>timeout</i>
+     * period. If the caller could claim the lock within <i>timeout</i> period,
+     * the caller acquire the lock. Otherwise, it would fail with {@link OwnershipAcquireFailedException}.
+     * <p>
+     * {@link #unlock()} should be called to unlock a claimed lock. The caller
+     * doesn't need to unlock to clean up resources if <i>tryLock</i> fails.
+     * <p>
+     * <i>tryLock</i> here is effectively the combination of following asynchronous calls.
+     * <pre>
+     *     ZKDistributedLock lock = ...;
+     *     Future<LockWaiter> attemptFuture = lock.asyncTryLock(...);
+     *
+     *     boolean acquired = waiter.waitForAcquireQuietly();
+     *     if (acquired) {
+     *         // ...
+     *     } else {
+     *         // ...
+     *     }
+     * </pre>
+     *
+     * @param timeout
+     *          timeout period to wait for claiming ownership
+     * @param unit
+     *          unit of timeout period
+     * @throws OwnershipAcquireFailedException if the lock is already acquired by others
+     * @throws LockingException when encountered other lock related issues.
+     */
+    void tryLock(long timeout, TimeUnit unit)
+            throws OwnershipAcquireFailedException, LockingException;
+
+    /**
+     * Acquire the lock in asynchronous way.
+     * <p>
+     * Calling this method will attempt to place a lock waiter to acquire this lock.
+     * The future returned by this method represents the result of this attempt. It doesn't mean
+     * the caller acquired the lock or not. The application should check {@link LockWaiter#getAcquireFuture()}
+     * to see if it acquired the lock or not.
+     *
+     * @param timeout
+     *          timeout period to wait for claiming ownership
+     * @param unit
+     *          unit of timeout period
+     * @return lock waiter representing this attempt of acquiring lock.
+     * @see #tryLock(long, TimeUnit)
+     */
+    Future<LockWaiter> asyncTryLock(long timeout, TimeUnit unit);
+
+    /**
+     * Release a claimed lock.
+     *
+     * @see #tryLock(long, TimeUnit)
+     */
+    void unlock();
+
+    /**
+     * Release a claimed lock in the asynchronous way.
+     *
+     * @return future representing the result of unlock operation.
+     * @see #unlock()
+     */
+    Future<BoxedUnit> 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
new file mode 100644
index 0000000..a68f2d8
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/SessionLockFactory.java
@@ -0,0 +1,38 @@
+/**
+ * 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.lock;
+
+import com.twitter.util.Future;
+
+/**
+ * Factory to create {@link SessionLock}
+ */
+public interface SessionLockFactory {
+
+    /**
+     * Create a lock with lock path.
+     *
+     * @param lockPath
+     *          lock path
+     * @param context
+     *          lock context
+     * @return future represents the creation result.
+     */
+    Future<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
new file mode 100644
index 0000000..a8eb482
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKDistributedLock.java
@@ -0,0 +1,537 @@
+/**
+ * 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.lock;
+
+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 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 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.
+ * <p/>
+ * The lock is vulnerable to timing issues. For example, the process could
+ * encounter a really long GC cycle between acquiring the lock, and writing to
+ * a ledger. This could have timed out the lock, and another process could have
+ * acquired the lock and started writing to bookkeeper. Therefore other
+ * mechanisms are required to ensure correctness (i.e. Fencing).
+ * <p/>
+ * The lock is only allowed to acquire once. If the lock is acquired successfully,
+ * the caller holds the ownership until it loses the ownership either because of
+ * others already acquired the lock when session expired or explicitly close it.
+ * <p>
+ * The caller could use {@link #checkOwnership()} or {@link #checkOwnershipAndReacquire()}
+ * to check if it still holds the lock. If it doesn't hold the lock, the caller should
+ * give up the ownership and close the lock.
+ * <h3>Metrics</h3>
+ * All the lock related stats are exposed under `lock`.
+ * <ul>
+ * <li>lock/acquire: opstats. latency spent on acquiring a lock.
+ * <li>lock/reacquire: opstats. latency spent on re-acquiring a lock.
+ * <li>lock/internalTryRetries: counter. the number of retries on re-creating internal locks.
+ * </ul>
+ * Other internal lock related stats are also exposed under `lock`. See {@link SessionLock}
+ * for details.
+ */
+public class ZKDistributedLock implements LockListener, DistributedLock {
+
+    static final Logger LOG = LoggerFactory.getLogger(ZKDistributedLock.class);
+
+    private final SessionLockFactory lockFactory;
+    private final OrderedScheduler lockStateExecutor;
+    private final String lockPath;
+    private final long lockTimeout;
+    private final DistributedLockContext lockContext = new DistributedLockContext();
+
+    private final AsyncSemaphore lockSemaphore = new AsyncSemaphore(1);
+    // 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;
+    // following variable tracking the status of acquire process
+    //   => create (internalLock) => tryLock (tryLockFuture) => waitForAcquire (lockWaiter)
+    private SessionLock internalLock = null;
+    private Future<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;
+
+    // A counter to track how many re-acquires happened during a lock's life cycle.
+    private final AtomicInteger reacquireCount = new AtomicInteger(0);
+    private final StatsLogger lockStatsLogger;
+    private final OpStatsLogger acquireStats;
+    private final OpStatsLogger reacquireStats;
+    private final Counter internalTryRetries;
+
+    public ZKDistributedLock(
+            OrderedScheduler lockStateExecutor,
+            SessionLockFactory lockFactory,
+            String lockPath,
+            long lockTimeout,
+            StatsLogger statsLogger) {
+        this.lockStateExecutor = lockStateExecutor;
+        this.lockPath = lockPath;
+        this.lockTimeout = lockTimeout;
+        this.lockFactory = lockFactory;
+
+        lockStatsLogger = statsLogger.scope("lock");
+        acquireStats = lockStatsLogger.getOpStatsLogger("acquire");
+        reacquireStats = lockStatsLogger.getOpStatsLogger("reacquire");
+        internalTryRetries = lockStatsLogger.getCounter("internalTryRetries");
+    }
+
+    private LockClosedException newLockClosedException() {
+        return new LockClosedException(lockPath, "Lock is already closed");
+    }
+
+    private synchronized void checkLockState() throws LockingException {
+        if (closed) {
+            throw newLockClosedException();
+        }
+        if (null != lockReacquireException) {
+            throw lockReacquireException;
+        }
+    }
+
+    /**
+     * 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() {
+        if (null != lockAcquireFuture) {
+            return Future.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 Stopwatch stopwatch = Stopwatch.createStarted();
+        promise.addEventListener(new FutureEventListener<ZKDistributedLock>() {
+            @Override
+            public void onSuccess(ZKDistributedLock lock) {
+                acquireStats.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            }
+            @Override
+            public void onFailure(Throwable cause) {
+                acquireStats.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+                // release the lock if fail to acquire
+                asyncClose();
+            }
+        });
+        this.lockAcquireFuture = promise;
+        lockStateExecutor.submit(lockPath, new Runnable() {
+            @Override
+            public void run() {
+                doAsyncAcquireWithSemaphore(promise, lockTimeout);
+            }
+        });
+        return promise;
+    }
+
+    void doAsyncAcquireWithSemaphore(final Promise<ZKDistributedLock> acquirePromise,
+                                     final long lockTimeout) {
+        lockSemaphore.acquireAndRun(new AbstractFunction0<Future<ZKDistributedLock>>() {
+            @Override
+            public Future<ZKDistributedLock> apply() {
+                doAsyncAcquire(acquirePromise, lockTimeout);
+                return acquirePromise;
+            }
+        });
+    }
+
+    void doAsyncAcquire(final Promise<ZKDistributedLock> acquirePromise,
+                        final long lockTimeout) {
+        LOG.trace("Async Lock Acquire {}", lockPath);
+        try {
+            checkLockState();
+        } catch (IOException ioe) {
+            FutureUtils.setException(acquirePromise, ioe);
+            return;
+        }
+
+        if (haveLock()) {
+            // it already hold the lock
+            FutureUtils.setValue(acquirePromise, this);
+            return;
+        }
+
+        lockFactory.createLock(lockPath, lockContext).addEventListener(OrderedFutureEventListener.of(
+                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());
+                        return;
+                    }
+                }
+                synchronized (ZKDistributedLock.this) {
+                    internalLock = lock;
+                    internalLock.setLockListener(ZKDistributedLock.this);
+                }
+                asyncTryLock(lock, acquirePromise, lockTimeout);
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                FutureUtils.setException(acquirePromise, cause);
+            }
+        }, lockStateExecutor, lockPath));
+    }
+
+    void asyncTryLock(SessionLock lock,
+                      final Promise<ZKDistributedLock> acquirePromise,
+                      final long lockTimeout) {
+        if (null != tryLockFuture) {
+            tryLockFuture.cancel();
+        }
+        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 = null;
+                        lockWaiter = waiter;
+                        waitForAcquire(waiter, acquirePromise);
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        FutureUtils.setException(acquirePromise, cause);
+                    }
+                }, lockStateExecutor, 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()));
+                        }
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        FutureUtils.setException(acquirePromise, cause);
+                    }
+                }, lockStateExecutor, lockPath));
+    }
+
+    /**
+     * NOTE: The {@link LockListener#onExpired()} is already executed in lock executor.
+     */
+    @Override
+    public void onExpired() {
+        try {
+            reacquireLock(false);
+        } catch (LockingException le) {
+            // should not happen
+            LOG.error("Locking exception on re-acquiring lock {} : ", lockPath, le);
+        }
+    }
+
+    /**
+     * Check if hold lock, if it doesn't, then re-acquire the lock.
+     *
+     * @throws LockingException     if the lock attempt fails
+     */
+    public synchronized void checkOwnershipAndReacquire() throws LockingException {
+        if (null == lockAcquireFuture || !lockAcquireFuture.isDefined()) {
+            throw new LockingException(lockPath, "check ownership before acquiring");
+        }
+
+        if (haveLock()) {
+            return;
+        }
+
+        // We may have just lost the lock because of a ZK session timeout
+        // not necessarily because someone else acquired the lock.
+        // In such cases just try to reacquire. If that fails, it will throw
+        reacquireLock(true);
+    }
+
+    /**
+     * Check if lock is held.
+     * If not, error out and do not reacquire. Use this in cases where there are many waiters by default
+     * and reacquire is unlikley to succeed.
+     *
+     * @throws LockingException     if the lock attempt fails
+     */
+    public synchronized void checkOwnership() throws LockingException {
+        if (null == lockAcquireFuture || !lockAcquireFuture.isDefined()) {
+            throw new LockingException(lockPath, "check ownership before acquiring");
+        }
+        if (!haveLock()) {
+            throw new LockingException(lockPath, "Lost lock ownership");
+        }
+    }
+
+    @VisibleForTesting
+    int getReacquireCount() {
+        return reacquireCount.get();
+    }
+
+    @VisibleForTesting
+    synchronized Future<ZKDistributedLock> getLockReacquireFuture() {
+        return lockReacquireFuture;
+    }
+
+    @VisibleForTesting
+    synchronized Future<ZKDistributedLock> getLockAcquireFuture() {
+        return lockAcquireFuture;
+    }
+
+    @VisibleForTesting
+    synchronized SessionLock getInternalLock() {
+        return internalLock;
+    }
+
+    @VisibleForTesting
+    LockWaiter getLockWaiter() {
+        return lockWaiter;
+    }
+
+    synchronized boolean haveLock() {
+        return !closed && internalLock != null && internalLock.isLockHeld();
+    }
+
+    void closeWaiter(final LockWaiter waiter,
+                     final Promise<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());
+        }
+    }
+
+    void interruptTryLock(final Future<LockWaiter> tryLockFuture,
+                          final Promise<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);
+        }
+    }
+
+    synchronized void unlockInternalLock(final Promise<Void> closePromise) {
+        if (internalLock == null) {
+            FutureUtils.setValue(closePromise, null);
+        } else {
+            internalLock.asyncUnlock().ensure(new AbstractFunction0<BoxedUnit>() {
+                @Override
+                public BoxedUnit apply() {
+                    FutureUtils.setValue(closePromise, null);
+                    return BoxedUnit.UNIT;
+                }
+            });
+        }
+    }
+
+    @Override
+    public Future<Void> asyncClose() {
+        final Promise<Void> closePromise;
+        synchronized (this) {
+            if (closed) {
+                return closeFuture;
+            }
+            closed = true;
+            closeFuture = closePromise = new Promise<Void>();
+        }
+        final Promise<Void> closeWaiterFuture = new Promise<Void>();
+        closeWaiterFuture.addEventListener(OrderedFutureEventListener.of(new FutureEventListener<Void>() {
+            @Override
+            public void onSuccess(Void value) {
+                complete();
+            }
+            @Override
+            public void onFailure(Throwable cause) {
+                complete();
+            }
+
+            private void complete() {
+                FutureUtils.setValue(closePromise, null);
+            }
+        }, lockStateExecutor, lockPath));
+        lockStateExecutor.submit(lockPath, new Runnable() {
+            @Override
+            public void run() {
+                closeWaiter(lockWaiter, closeWaiterFuture);
+            }
+        });
+        return closePromise;
+    }
+
+    void internalReacquireLock(final AtomicInteger numRetries,
+                               final long lockTimeout,
+                               final Promise<ZKDistributedLock> reacquirePromise) {
+        lockStateExecutor.submit(lockPath, new Runnable() {
+            @Override
+            public void run() {
+                doInternalReacquireLock(numRetries, lockTimeout, reacquirePromise);
+            }
+        });
+    }
+
+    void doInternalReacquireLock(final AtomicInteger numRetries,
+                                 final long lockTimeout,
+                                 final Promise<ZKDistributedLock> reacquirePromise) {
+        internalTryRetries.inc();
+        Promise<ZKDistributedLock> tryPromise = new Promise<ZKDistributedLock>();
+        tryPromise.addEventListener(new FutureEventListener<ZKDistributedLock>() {
+            @Override
+            public void onSuccess(ZKDistributedLock lock) {
+                FutureUtils.setValue(reacquirePromise, lock);
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                if (cause instanceof OwnershipAcquireFailedException) {
+                    // the lock has been acquired by others
+                    FutureUtils.setException(reacquirePromise, cause);
+                } else {
+                    if (numRetries.getAndDecrement() > 0 && !closed) {
+                        internalReacquireLock(numRetries, lockTimeout, reacquirePromise);
+                    } else {
+                        FutureUtils.setException(reacquirePromise, cause);
+                    }
+                }
+            }
+        });
+        doAsyncAcquireWithSemaphore(tryPromise, 0);
+    }
+
+    private Future<ZKDistributedLock> reacquireLock(boolean throwLockAcquireException) throws LockingException {
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+        Promise<ZKDistributedLock> lockPromise;
+        synchronized (this) {
+            if (closed) {
+                throw newLockClosedException();
+            }
+            if (null != lockReacquireException) {
+                if (throwLockAcquireException) {
+                    throw lockReacquireException;
+                } else {
+                    return null;
+                }
+            }
+            if (null != lockReacquireFuture) {
+                return lockReacquireFuture;
+            }
+            LOG.info("reacquiring lock at {}", lockPath);
+            lockReacquireFuture = lockPromise = new Promise<ZKDistributedLock>();
+            lockReacquireFuture.addEventListener(new FutureEventListener<ZKDistributedLock>() {
+                @Override
+                public void onSuccess(ZKDistributedLock lock) {
+                    // if re-acquire successfully, clear the state.
+                    synchronized (ZKDistributedLock.this) {
+                        lockReacquireFuture = null;
+                    }
+                    reacquireStats.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+                }
+
+                @Override
+                public void onFailure(Throwable cause) {
+                    synchronized (ZKDistributedLock.this) {
+                        if (cause instanceof LockingException) {
+                            lockReacquireException = (LockingException) cause;
+                        } else {
+                            lockReacquireException = new LockingException(lockPath,
+                                    "Exception on re-acquiring lock", cause);
+                        }
+                    }
+                    reacquireStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+                }
+            });
+        }
+        reacquireCount.incrementAndGet();
+        internalReacquireLock(new AtomicInteger(Integer.MAX_VALUE), 0, lockPromise);
+        return lockPromise;
+    }
+
+}
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
new file mode 100644
index 0000000..e2699e7
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLock.java
@@ -0,0 +1,1363 @@
+/**
+ * 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.lock;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.util.FailpointUtils;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+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.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;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.util.SafeRunnable;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+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.
+ * It is not reusable: if lock failed, if zookeeper session is expired, if #unlock is called,
+ * it would be transitioned to expired or closed state.
+ *
+ * The Locking Procedure is described as below.
+ *
+ * <p>
+ * 0. if it is an immediate lock, it would get lock waiters first. if the lock is already held
+ *    by someone. it would fail immediately with {@link org.apache.distributedlog.exceptions.OwnershipAcquireFailedException}
+ *    with current owner. if there is no lock waiters, it would start locking procedure from 1.
+ * 1. prepare: create a sequential znode to identify the lock.
+ * 2. check lock waiters: get all lock waiters to check after prepare. if it is the first waiter,
+ *    claim the ownership; if it is not the first waiter, but first waiter was itself (same client id and same session id)
+ *    claim the ownership too; otherwise, it would set watcher on its sibling and wait it to disappared.
+ * </p>
+ *
+ * <pre>
+ *                      +-----------------+
+ *                      |       INIT      | ------------------------------+
+ *                      +--------+--------+                               |
+ *                               |                                        |
+ *                               |                                        |
+ *                      +--------v--------+                               |
+ *                      |    PREPARING    |----------------------------+  |
+ *                      +--------+--------+                            |  |
+ *                               |                                     |  |
+ *                               |                                     |  |
+ *                      +--------v--------+                            |  |
+ *        +-------------|    PREPARED     |--------------+             |  |
+ *        |             +-----^---------+-+              |             |  |
+ *        |                   |  |      |                |             |  |
+ *        |                   |  |      |                |             |  |
+ *        |                   |  |      |                |             |  |
+ * +------V-----------+       |  |      |       +--------v----------+  |  |
+ * |     WAITING      |-------+  |      |       |    CLAIMED        |  |  |
+ * +------+-----+-----+          |      |       +--+----------+-----+  |  |
+ *        |     |                |      |          |        |          |  |
+ *        |     |                |      |          |        |          |  |
+ *        |     |                |      |          |        |          |  |
+ *        |     |                |    +-v----------v----+   |          |  |
+ *        |     +-------------------->|     EXPIRED     |   |          |  |
+ *        |                      |    +--+--------------+   |          |  |
+ *        |                      |       |                  |          |  |
+ *        |                      |       |                  |          |  |
+ *        |             +--------V-------V-+                |          |  |
+ *        +------------>|     CLOSING      |<---------------+----------+--+
+ *                      +------------------+
+ *                               |
+ *                               |
+ *                               |
+ *                      +--------V---------+
+ *                      |     CLOSED       |
+ *                      +------------------+
+ * </pre>
+ *
+ * <h3>Metrics</h3>
+ * <ul>
+ * <li>tryAcquire: opstats. latency spent on try locking operations. it includes timeouts.
+ * <li>tryTimeouts: counter. the number of timeouts on try locking operations
+ * <li>unlock: opstats. latency spent on unlock operations.
+ * </ul>
+ */
+class ZKSessionLock implements SessionLock {
+
+    static final Logger LOG = LoggerFactory.getLogger(ZKSessionLock.class);
+
+    private static final String LOCK_PATH_PREFIX = "/member_";
+    private static final String LOCK_PART_SEP = "_";
+
+    public static String getLockPathPrefixV1(String lockPath) {
+        // member_
+        return lockPath + LOCK_PATH_PREFIX;
+    }
+
+    public static String getLockPathPrefixV2(String lockPath, String clientId) throws UnsupportedEncodingException {
+        // member_<clientid>_
+        return lockPath + LOCK_PATH_PREFIX + URLEncoder.encode(clientId, UTF_8.name()) + LOCK_PART_SEP;
+    }
+
+    public static String getLockPathPrefixV3(String lockPath, String clientId, long sessionOwner) throws UnsupportedEncodingException {
+        // member_<clientid>_s<owner_session>_
+        StringBuilder sb = new StringBuilder();
+        sb.append(lockPath).append(LOCK_PATH_PREFIX).append(URLEncoder.encode(clientId, UTF_8.name())).append(LOCK_PART_SEP)
+                .append("s").append(String.format("%10d", sessionOwner)).append(LOCK_PART_SEP);
+        return sb.toString();
+    }
+
+    public static byte[] serializeClientId(String clientId) {
+        return clientId.getBytes(UTF_8);
+    }
+
+    public static String deserializeClientId(byte[] data) {
+        return new String(data, UTF_8);
+    }
+
+    public static String getLockIdFromPath(String path) {
+        // We only care about our actual id since we want to compare ourselves to siblings.
+        if (path.contains("/")) {
+            return path.substring(path.lastIndexOf("/") + 1);
+        } else {
+            return path;
+        }
+    }
+
+    static final Comparator<String> MEMBER_COMPARATOR = new Comparator<String>() {
+        public int compare(String o1, String o2) {
+            int l1 = parseMemberID(o1);
+            int l2 = parseMemberID(o2);
+            return l1 - l2;
+        }
+    };
+
+    static enum State {
+        INIT,      // initialized state
+        PREPARING, // preparing to lock, but no lock node created
+        PREPARED,  // lock node created
+        CLAIMED,   // claim lock ownership
+        WAITING,   // waiting for the ownership
+        EXPIRED,   // lock is expired
+        CLOSING,   // lock is being closed
+        CLOSED,    // lock is closed
+    }
+
+    /**
+     * Convenience class for state management. Provide debuggability features by tracing unxpected state
+     * transitions.
+     */
+    static class StateManagement {
+
+        static final Logger LOG = LoggerFactory.getLogger(StateManagement.class);
+
+        private volatile State state;
+
+        StateManagement() {
+            this.state = State.INIT;
+        }
+
+        public void transition(State toState) {
+            if (!validTransition(toState)) {
+                LOG.error("Invalid state transition from {} to {} ",
+                        new Object[] { this.state, toState, getStack() });
+            }
+            this.state = toState;
+        }
+
+        private boolean validTransition(State toState) {
+            switch (toState) {
+                case INIT:
+                    return false;
+                case PREPARING:
+                    return inState(State.INIT);
+                case PREPARED:
+                    return inState(State.PREPARING) || inState(State.WAITING);
+                case CLAIMED:
+                    return inState(State.PREPARED);
+                case WAITING:
+                    return inState(State.PREPARED);
+                case EXPIRED:
+                    return isTryingOrClaimed();
+                case CLOSING:
+                    return !inState(State.CLOSED);
+                case CLOSED:
+                    return inState(State.CLOSING) || inState(State.CLOSED);
+                default:
+                    return false;
+            }
+        }
+
+        private State getState() {
+            return state;
+        }
+
+        private boolean isTryingOrClaimed() {
+            return inState(State.PREPARING) || inState(State.PREPARED) ||
+                inState(State.WAITING) || inState(State.CLAIMED);
+        }
+
+        public boolean isExpiredOrClosing() {
+            return inState(State.CLOSED) || inState(State.EXPIRED) || inState(State.CLOSING);
+        }
+
+        public boolean isExpiredOrClosed() {
+            return inState(State.CLOSED) || inState(State.EXPIRED);
+        }
+
+        public boolean isClosed() {
+            return inState(State.CLOSED);
+        }
+
+        private boolean inState(final State state) {
+            return state == this.state;
+        }
+
+        private Exception getStack() {
+            return new Exception();
+        }
+    }
+
+    private final ZooKeeperClient zkClient;
+    private final ZooKeeper zk;
+    private final String lockPath;
+    // Identify a unique lock
+    private final Pair<String, Long> lockId;
+    private StateManagement lockState;
+    private final DistributedLockContext lockContext;
+
+    private final Promise<Boolean> acquireFuture;
+    private String currentId;
+    private String currentNode;
+    private String watchedNode;
+    private LockWatcher watcher;
+    private final AtomicInteger epoch = new AtomicInteger(0);
+    private final OrderedScheduler lockStateExecutor;
+    private LockListener lockListener = null;
+    private final long lockOpTimeout;
+
+    private final OpStatsLogger tryStats;
+    private final Counter tryTimeouts;
+    private final OpStatsLogger unlockStats;
+
+    ZKSessionLock(ZooKeeperClient zkClient,
+                  String lockPath,
+                  String clientId,
+                  OrderedScheduler lockStateExecutor)
+            throws IOException {
+        this(zkClient,
+                lockPath,
+                clientId,
+                lockStateExecutor,
+                DistributedLogConstants.LOCK_OP_TIMEOUT_DEFAULT * 1000, NullStatsLogger.INSTANCE,
+                new DistributedLockContext());
+    }
+
+    /**
+     * Creates a distributed lock using the given {@code zkClient} to coordinate locking.
+     *
+     * @param zkClient The ZooKeeper client to use.
+     * @param lockPath The path used to manage the lock under.
+     * @param clientId client id use for lock.
+     * @param lockStateExecutor executor to execute all lock state changes.
+     * @param lockOpTimeout timeout of lock operations
+     * @param statsLogger stats logger
+     */
+    public ZKSessionLock(ZooKeeperClient zkClient,
+                         String lockPath,
+                         String clientId,
+                         OrderedScheduler lockStateExecutor,
+                         long lockOpTimeout,
+                         StatsLogger statsLogger,
+                         DistributedLockContext lockContext)
+            throws IOException {
+        this.zkClient = zkClient;
+        try {
+            this.zk = zkClient.get();
+        } catch (ZooKeeperClient.ZooKeeperConnectionException zce) {
+            throw new ZKException("Failed to get zookeeper client for lock " + lockPath,
+                    KeeperException.Code.CONNECTIONLOSS);
+        } catch (InterruptedException e) {
+            throw new DLInterruptedException("Interrupted on getting zookeeper client for lock " + lockPath, e);
+        }
+        this.lockPath = lockPath;
+        this.lockId = Pair.of(clientId, this.zk.getSessionId());
+        this.lockContext = lockContext;
+        this.lockStateExecutor = lockStateExecutor;
+        this.lockState = new StateManagement();
+        this.lockOpTimeout = lockOpTimeout;
+
+        this.tryStats = statsLogger.getOpStatsLogger("tryAcquire");
+        this.tryTimeouts = statsLogger.getCounter("tryTimeouts");
+        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 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);
+                // Note re. logging and exceptions: errors are already logged by unlockAsync.
+                return BoxedUnit.UNIT;
+            }
+        });
+    }
+
+    @Override
+    public ZKSessionLock setLockListener(LockListener lockListener) {
+        this.lockListener = lockListener;
+        return this;
+    }
+
+    String getLockPath() {
+        return this.lockPath;
+    }
+
+    @VisibleForTesting
+    AtomicInteger getEpoch() {
+        return epoch;
+    }
+
+    @VisibleForTesting
+    State getLockState() {
+        return lockState.getState();
+    }
+
+    @VisibleForTesting
+    Pair<String, Long> getLockId() {
+        return lockId;
+    }
+
+    public boolean isLockExpired() {
+        return lockState.isExpiredOrClosing();
+    }
+
+    @Override
+    public boolean isLockHeld() {
+        return lockState.inState(State.CLAIMED);
+    }
+
+    /**
+     * Execute a lock action of a given <i>lockEpoch</i> in ordered safe way.
+     *
+     * @param lockEpoch
+     *          lock epoch
+     * @param func
+     *          function to execute a lock action
+     */
+    protected void executeLockAction(final int lockEpoch, final LockAction func) {
+        lockStateExecutor.submit(lockPath, new SafeRunnable() {
+            @Override
+            public void safeRun() {
+                if (ZKSessionLock.this.epoch.get() == lockEpoch) {
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("{} executing lock action '{}' under epoch {} for lock {}",
+                                new Object[]{lockId, func.getActionName(), lockEpoch, lockPath});
+                    }
+                    func.execute();
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("{} executed lock action '{}' under epoch {} for lock {}",
+                                new Object[]{lockId, func.getActionName(), lockEpoch, lockPath});
+                    }
+                } else {
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("{} skipped executing lock action '{}' for lock {}, since epoch is changed from {} to {}.",
+                                new Object[]{lockId, func.getActionName(), lockPath, lockEpoch, ZKSessionLock.this.epoch.get()});
+                    }
+                }
+            }
+        });
+    }
+
+    /**
+     * Execute a lock action of a given <i>lockEpoch</i> in ordered safe way. If the lock action couln't be
+     * executed due to epoch changed, fail the given <i>promise</i> with
+     * {@link EpochChangedException}
+     *
+     * @param lockEpoch
+     *          lock epoch
+     * @param func
+     *          function to execute a lock action
+     * @param promise
+     *          promise
+     */
+    protected <T> void executeLockAction(final int lockEpoch, final LockAction func, final Promise<T> promise) {
+        lockStateExecutor.submit(lockPath, new SafeRunnable() {
+            @Override
+            public void safeRun() {
+                int currentEpoch = ZKSessionLock.this.epoch.get();
+                if (currentEpoch == lockEpoch) {
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("{} executed lock action '{}' under epoch {} for lock {}",
+                                new Object[]{lockId, func.getActionName(), lockEpoch, lockPath});
+                    }
+                    func.execute();
+                    if (LOG.isTraceEnabled()) {
+                        LOG.trace("{} executed lock action '{}' under epoch {} for lock {}",
+                                new Object[]{lockId, func.getActionName(), lockEpoch, lockPath});
+                    }
+                } else {
+                    if (LOG.isTraceEnabled()) {
+                        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));
+                }
+            }
+        });
+    }
+
+    /**
+     * Parse member id generated by zookeeper from given <i>nodeName</i>
+     *
+     * @param nodeName
+     *          lock node name
+     * @return member id generated by zookeeper
+     */
+    static int parseMemberID(String nodeName) {
+        int id = -1;
+        String[] parts = nodeName.split("_");
+        if (parts.length > 0) {
+            try {
+                id = Integer.parseInt(parts[parts.length - 1]);
+            } catch (NumberFormatException nfe) {
+                // make it to be MAX_VALUE, so the bad znode will never acquire the lock
+                id = Integer.MAX_VALUE;
+            }
+        }
+        return id;
+    }
+
+    static boolean areLockWaitersInSameSession(String node1, String node2) {
+        String[] parts1 = node1.split("_");
+        String[] parts2 = node2.split("_");
+        if (parts1.length != 4 || parts2.length != 4) {
+            return node1.equals(node2);
+        }
+        if (!parts1[2].startsWith("s") || !parts2[2].startsWith("s")) {
+            return node1.equals(node2);
+        }
+        long sessionOwner1 = Long.parseLong(parts1[2].substring(1));
+        long sessionOwner2 = Long.parseLong(parts2[2].substring(1));
+        if (sessionOwner1 != sessionOwner2) {
+            return false;
+        }
+        String clientId1, clientId2;
+        try {
+            clientId1 = URLDecoder.decode(parts1[1], UTF_8.name());
+            clientId2 = URLDecoder.decode(parts2[1], UTF_8.name());
+            return clientId1.equals(clientId2);
+        } catch (UnsupportedEncodingException e) {
+            // if failed to parse client id, we have to get client id by zookeeper#getData.
+            return node1.equals(node2);
+        }
+    }
+
+    /**
+     * Get client id and its ephemeral owner.
+     *
+     * @param zkClient
+     *          zookeeper client
+     * @param lockPath
+     *          lock path
+     * @param nodeName
+     *          node name
+     * @return client id and its ephemeral owner.
+     */
+    static Future<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")) {
+            long sessionOwner = Long.parseLong(parts[2].substring(1));
+            String clientId;
+            try {
+                clientId = URLDecoder.decode(parts[1], UTF_8.name());
+                return Future.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>>();
+        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)));
+                } else {
+                    promise.setValue(Pair.of(deserializeClientId(data), stat.getEphemeralOwner()));
+                }
+            }
+        }, null);
+        return promise;
+    }
+
+    @Override
+    public Future<LockWaiter> asyncTryLock(final long timeout, final TimeUnit unit) {
+        final Promise<String> result = new Promise<String>();
+        final boolean wait = DistributedLogConstants.LOCK_IMMEDIATE != timeout;
+        if (wait) {
+            asyncTryLock(wait, result);
+        } else {
+            // try to check locks first
+            zk.getChildren(lockPath, null, new AsyncCallback.Children2Callback() {
+                @Override
+                public void processResult(final int rc, String path, Object ctx,
+                                          final List<String> children, Stat stat) {
+                    lockStateExecutor.submit(lockPath, new SafeRunnable() {
+                        @Override
+                        public void safeRun() {
+                            if (!lockState.inState(State.INIT)) {
+                                result.setException(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
+                                return;
+                            }
+                            if (KeeperException.Code.OK.intValue() != rc) {
+                                result.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                                return;
+                            }
+
+                            FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockTryAcquire);
+
+                            Collections.sort(children, MEMBER_COMPARATOR);
+                            if (children.size() > 0) {
+                                asyncParseClientID(zk, lockPath, children.get(0)).addEventListener(
+                                        new FutureEventListener<Pair<String, Long>>() {
+                                            @Override
+                                            public void onSuccess(Pair<String, Long> owner) {
+                                                if (!checkOrClaimLockOwner(owner, result)) {
+                                                    acquireFuture.updateIfEmpty(new Return<Boolean>(false));
+                                                }
+                                            }
+
+                                            @Override
+                                            public void onFailure(final Throwable cause) {
+                                                lockStateExecutor.submit(lockPath, new SafeRunnable() {
+                                                    @Override
+                                                    public void safeRun() {
+                                                        result.setException(cause);
+                                                    }
+                                                });
+                                            }
+                                        });
+                            } else {
+                                asyncTryLock(wait, result);
+                            }
+                        }
+                    });
+                }
+            }, 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>() {
+            @Override
+            public LockWaiter apply(final String currentOwner) {
+                final Exception acquireException = new OwnershipAcquireFailedException(lockPath, currentOwner);
+                FutureUtils.within(
+                        acquireFuture,
+                        timeout,
+                        unit,
+                        acquireException,
+                        lockStateExecutor,
+                        lockPath
+                ).addEventListener(new FutureEventListener<Boolean>() {
+
+                    @Override
+                    public void onSuccess(Boolean acquired) {
+                        completeOrFail(acquireException);
+                    }
+
+                    @Override
+                    public void onFailure(final Throwable acquireCause) {
+                        completeOrFail(acquireException);
+                    }
+
+                    private void completeOrFail(final Throwable acquireCause) {
+                        if (isLockHeld()) {
+                            waiterAcquireFuture.setValue(true);
+                        } else {
+                            asyncUnlock().addEventListener(new FutureEventListener<BoxedUnit>() {
+                                @Override
+                                public void onSuccess(BoxedUnit value) {
+                                    waiterAcquireFuture.setException(acquireCause);
+                                }
+
+                                @Override
+                                public void onFailure(Throwable cause) {
+                                    waiterAcquireFuture.setException(acquireCause);
+                                }
+                            });
+                        }
+                    }
+                });;
+                return new LockWaiter(
+                        lockId.getLeft(),
+                        currentOwner,
+                        waiterAcquireFuture);
+            }
+        });
+    }
+
+    private boolean checkOrClaimLockOwner(final Pair<String, Long> currentOwner,
+                                          final Promise<String> result) {
+        if (lockId.compareTo(currentOwner) != 0 && !lockContext.hasLockId(currentOwner)) {
+            lockStateExecutor.submit(lockPath, new SafeRunnable() {
+                @Override
+                public void safeRun() {
+                    result.setValue(currentOwner.getLeft());
+                }
+            });
+            return false;
+        }
+        // current owner is itself
+        final int curEpoch = epoch.incrementAndGet();
+        executeLockAction(curEpoch, new LockAction() {
+            @Override
+            public void execute() {
+                if (!lockState.inState(State.INIT)) {
+                    result.setException(new LockStateChangedException(lockPath, lockId, State.INIT, lockState.getState()));
+                    return;
+                }
+                asyncTryLock(false, result);
+            }
+            @Override
+            public String getActionName() {
+                return "claimOwnership(owner=" + currentOwner + ")";
+            }
+        }, result);
+        return true;
+    }
+
+    /**
+     * Try lock. If it failed, it would cleanup its attempt.
+     *
+     * @param wait
+     *          whether to wait for ownership.
+     * @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>() {
+            @Override
+            public void onSuccess(String currentOwner) {
+                result.setValue(currentOwner);
+            }
+
+            @Override
+            public void onFailure(final Throwable lockCause) {
+                // If tryLock failed due to state changed, we don't need to cleanup
+                if (lockCause instanceof LockStateChangedException) {
+                    LOG.info("skipping cleanup for {} at {} after encountering lock " +
+                            "state change exception : ", new Object[] { lockId, lockPath, lockCause });
+                    result.setException(lockCause);
+                    return;
+                }
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("{} is cleaning up its lock state for {} due to : ",
+                            new Object[] { lockId, lockPath, lockCause });
+                }
+
+                // If we encountered any exception we should cleanup
+                Future<BoxedUnit> unlockResult = asyncUnlock();
+                unlockResult.addEventListener(new FutureEventListener<BoxedUnit>() {
+                    @Override
+                    public void onSuccess(BoxedUnit value) {
+                        result.setException(lockCause);
+                    }
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        result.setException(lockCause);
+                    }
+                });
+            }
+        });
+        asyncTryLockWithoutCleanup(wait, lockResult);
+    }
+
+    /**
+     * 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
+     * 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
+     * transformations attached on promise will be executed in order.
+     *
+     * @param wait
+     *          whether to wait for ownership.
+     * @param promise
+     *          promise to satisfy with current lock owner.
+     */
+    private void asyncTryLockWithoutCleanup(final boolean wait, final Promise<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()));
+                    return;
+                }
+                lockState.transition(State.PREPARING);
+
+                final int curEpoch = epoch.incrementAndGet();
+                watcher = new LockWatcher(curEpoch);
+                // register watcher for session expires
+                zkClient.register(watcher);
+                // Encode both client id and session in the lock node
+                String myPath;
+                try {
+                    // member_<clientid>_s<owner_session>_
+                    myPath = getLockPathPrefixV3(lockPath, lockId.getLeft(), lockId.getRight());
+                } catch (UnsupportedEncodingException uee) {
+                    myPath = getLockPathPrefixV1(lockPath);
+                }
+                zk.create(myPath, serializeClientId(lockId.getLeft()), zkClient.getDefaultACL(), CreateMode.EPHEMERAL_SEQUENTIAL,
+                        new AsyncCallback.StringCallback() {
+                            @Override
+                            public void processResult(final int rc, String path, Object ctx, final String name) {
+                                executeLockAction(curEpoch, new LockAction() {
+                                    @Override
+                                    public void execute() {
+                                        if (KeeperException.Code.OK.intValue() != rc) {
+                                            KeeperException ke = KeeperException.create(KeeperException.Code.get(rc));
+                                            promise.setException(ke);
+                                            return;
+                                        }
+
+                                        if (FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockTryCloseRaceCondition)) {
+                                            lockState.transition(State.CLOSING);
+                                            lockState.transition(State.CLOSED);
+                                        }
+
+                                        if (null != currentNode) {
+                                            LOG.error("Current node for {} overwritten current = {} new = {}",
+                                                new Object[] { lockPath, lockId, getLockIdFromPath(currentNode) });
+                                        }
+
+                                        currentNode = name;
+                                        currentId = getLockIdFromPath(currentNode);
+                                        LOG.trace("{} received member id for lock {}", lockId, currentId);
+
+                                        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>();
+                                            deleteLockNode(deletePromise);
+                                            deletePromise.ensure(new Function0<BoxedUnit>() {
+                                                public BoxedUnit apply() {
+                                                    promise.setException(new LockClosedException(lockPath, lockId, lockState.getState()));
+                                                    return BoxedUnit.UNIT;
+                                                }
+                                            });
+                                            return;
+                                        }
+
+                                        lockState.transition(State.PREPARED);
+                                        checkLockOwnerAndWaitIfPossible(watcher, wait, promise);
+                                    }
+
+                                    @Override
+                                    public String getActionName() {
+                                        return "postPrepare(wait=" + wait + ")";
+                                    }
+                                });
+                            }
+                        }, null);
+            }
+            @Override
+            public String getActionName() {
+                return "prepare(wait=" + wait + ")";
+            }
+        }, promise);
+    }
+
+    @Override
+    public void tryLock(long timeout, TimeUnit unit) throws LockingException {
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+        Future<LockWaiter> tryFuture = asyncTryLock(timeout, unit);
+        LockWaiter waiter = waitForTry(stopwatch, tryFuture);
+        boolean acquired = waiter.waitForAcquireQuietly();
+        if (!acquired) {
+            throw new OwnershipAcquireFailedException(lockPath, waiter.getCurrentOwner());
+        }
+    }
+
+    synchronized LockWaiter waitForTry(Stopwatch stopwatch, Future<LockWaiter> tryFuture)
+            throws LockingException {
+        boolean success = false;
+        boolean stateChanged = false;
+        LockWaiter waiter;
+        try {
+            waiter = Await.result(tryFuture, Duration.fromMilliseconds(lockOpTimeout));
+            success = true;
+        } catch (LockStateChangedException ex) {
+            stateChanged = true;
+            throw ex;
+        } catch (LockingException ex) {
+            throw ex;
+        } catch (TimeoutException toe) {
+            tryTimeouts.inc();
+            throw new LockingException(lockPath, "Timeout during try phase of lock acquire", toe);
+        } catch (Exception ex) {
+            String message = getLockId() + " failed to lock " + lockPath;
+            throw new LockingException(lockPath, message, ex);
+        } finally {
+            if (success) {
+                tryStats.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+            } else {
+                tryStats.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+            }
+            // This can only happen for a Throwable thats not an
+            // Exception, i.e. an Error
+            if (!success && !stateChanged) {
+                unlock();
+            }
+        }
+        return waiter;
+    }
+
+    @Override
+    public Future<BoxedUnit> asyncUnlock() {
+        return asyncUnlock(new LockClosedException(lockPath, lockId, lockState.getState()));
+    }
+
+    Future<BoxedUnit> asyncUnlock(final Throwable cause) {
+        final Promise<BoxedUnit> promise = new Promise<BoxedUnit>();
+
+        // 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
+        // risk of an ABA problem where we delete and recreate a node and then delete it again here.
+        lockStateExecutor.submit(lockPath, new SafeRunnable() {
+            @Override
+            public void safeRun() {
+                acquireFuture.updateIfEmpty(new Throw<Boolean>(cause));
+                unlockInternal(promise);
+                promise.addEventListener(new OpStatsListener<BoxedUnit>(unlockStats));
+            }
+        });
+
+        return promise;
+    }
+
+    @Override
+    public void unlock() {
+        Future<BoxedUnit> unlockResult = asyncUnlock();
+        try {
+            Await.result(unlockResult, Duration.fromMilliseconds(lockOpTimeout));
+        } 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 });
+        } catch (Exception e) {
+            LOG.warn("{} failed to unlock {} : ", new Object[] { lockId, lockPath, e });
+        }
+    }
+
+    // Lock State Changes (all state changes should be executed under a LockAction)
+
+    private void claimOwnership(int lockEpoch) {
+        lockState.transition(State.CLAIMED);
+        // clear previous lock ids
+        lockContext.clearLockIds();
+        // add current lock id
+        lockContext.addLockId(lockId);
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("Notify lock waiters on {} at {} : watcher epoch {}, lock epoch {}",
+                    new Object[] { lockPath, System.currentTimeMillis(),
+                            lockEpoch, ZKSessionLock.this.epoch.get() });
+        }
+        acquireFuture.updateIfEmpty(new Return<Boolean>(true));
+    }
+
+    /**
+     * NOTE: unlockInternal should only after try lock.
+     */
+    private void unlockInternal(final Promise<BoxedUnit> promise) {
+
+        // already closed or expired, nothing to cleanup
+        this.epoch.incrementAndGet();
+        if (null != watcher) {
+            this.zkClient.unregister(watcher);
+        }
+
+        if (lockState.inState(State.CLOSED)) {
+            promise.setValue(BoxedUnit.UNIT);
+            return;
+        }
+
+        LOG.info("Lock {} for {} is closed from state {}.",
+                new Object[] { lockId, lockPath, lockState.getState() });
+
+        final boolean skipCleanup = lockState.inState(State.INIT) || lockState.inState(State.EXPIRED);
+
+        lockState.transition(State.CLOSING);
+
+        if (skipCleanup) {
+            // Nothing to cleanup if INIT (never tried) or EXPIRED (ephemeral node
+            // auto-removed)
+            lockState.transition(State.CLOSED);
+            promise.setValue(BoxedUnit.UNIT);
+            return;
+        }
+
+        // In any other state, we should clean up the member node
+        Promise<BoxedUnit> deletePromise = new Promise<BoxedUnit>();
+        deleteLockNode(deletePromise);
+
+        // Set the state to closed after we've cleaned up
+        deletePromise.addEventListener(new FutureEventListener<BoxedUnit>() {
+            @Override
+            public void onSuccess(BoxedUnit complete) {
+                lockStateExecutor.submit(lockPath, new SafeRunnable() {
+                    @Override
+                    public void safeRun() {
+                        lockState.transition(State.CLOSED);
+                        promise.setValue(BoxedUnit.UNIT);
+                    }
+                });
+            }
+            @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);
+            }
+        });
+    }
+
+    private void deleteLockNode(final Promise<BoxedUnit> promise) {
+        if (null == currentNode) {
+            promise.setValue(BoxedUnit.UNIT);
+            return;
+        }
+
+        zk.delete(currentNode, -1, new AsyncCallback.VoidCallback() {
+            @Override
+            public void processResult(final int rc, final String path, Object ctx) {
+                lockStateExecutor.submit(lockPath, new SafeRunnable() {
+                    @Override
+                    public void safeRun() {
+                        if (KeeperException.Code.OK.intValue() == rc) {
+                            LOG.info("Deleted lock node {} for {} successfully.", path, lockId);
+                        } else if (KeeperException.Code.NONODE.intValue() == rc ||
+                                KeeperException.Code.SESSIONEXPIRED.intValue() == rc) {
+                            LOG.info("Delete node failed. Node already gone for node {} id {}, rc = {}",
+                                    new Object[] { path, lockId, KeeperException.Code.get(rc) });
+                        } else {
+                            LOG.error("Failed on deleting lock node {} for {} : {}",
+                                    new Object[] { path, lockId, KeeperException.Code.get(rc) });
+                        }
+
+                        FailpointUtils.checkFailPointNoThrow(FailpointUtils.FailPointName.FP_LockUnlockCleanup);
+                        promise.setValue(BoxedUnit.UNIT);
+                    }
+                });
+            }
+        }, null);
+    }
+
+    /**
+     * Handle session expired for lock watcher at epoch <i>lockEpoch</i>.
+     *
+     * @param lockEpoch
+     *          lock epoch
+     */
+    private void handleSessionExpired(final int lockEpoch) {
+        executeLockAction(lockEpoch, new LockAction() {
+            @Override
+            public void execute() {
+                if (lockState.inState(State.CLOSED) || lockState.inState(State.CLOSING)) {
+                    // Already fully closed, no need to process expire.
+                    return;
+                }
+
+                boolean shouldNotifyLockListener = lockState.inState(State.CLAIMED);
+
+                lockState.transition(State.EXPIRED);
+
+                // remove the watcher
+                if (null != watcher) {
+                    zkClient.unregister(watcher);
+                }
+
+                // increment epoch to avoid any ongoing locking action
+                ZKSessionLock.this.epoch.incrementAndGet();
+
+                // 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())));
+
+                // session expired, ephemeral node is gone.
+                currentNode = null;
+                currentId = null;
+
+                if (shouldNotifyLockListener) {
+                    // if session expired after claimed, we need to notify the caller to re-lock
+                    if (null != lockListener) {
+                        lockListener.onExpired();
+                    }
+                }
+            }
+
+            @Override
+            public String getActionName() {
+                return "handleSessionExpired(epoch=" + lockEpoch + ")";
+            }
+        });
+    }
+
+    private void handleNodeDelete(int lockEpoch, final WatchedEvent event) {
+        executeLockAction(lockEpoch, new LockAction() {
+            @Override
+            public void execute() {
+                // The lock is either expired or closed
+                if (!lockState.inState(State.WAITING)) {
+                    LOG.info("{} ignore watched node {} deleted event, since lock state has moved to {}.",
+                            new Object[] { lockId, event.getPath(), lockState.getState() });
+                    return;
+                }
+                lockState.transition(State.PREPARED);
+
+                // we don't need to wait and check the result, since:
+                // 1) if it claimed the ownership, it would notify the waiters when claimed ownerships
+                // 2) if it failed, it would also notify the waiters, the waiters would cleanup the state.
+                checkLockOwnerAndWaitIfPossible(watcher, true);
+            }
+
+            @Override
+            public String getActionName() {
+                return "handleNodeDelete(path=" + event.getPath() + ")";
+            }
+        });
+    }
+
+    private Future<String> checkLockOwnerAndWaitIfPossible(final LockWatcher lockWatcher,
+                                                           final boolean wait) {
+        final Promise<String> promise = new Promise<String>();
+        checkLockOwnerAndWaitIfPossible(lockWatcher, wait, promise);
+        return promise;
+    }
+
+    /**
+     * Check Lock Owner Phase 1 : Get all lock waiters.
+     *
+     * @param lockWatcher
+     *          lock watcher.
+     * @param wait
+     *          whether to wait for ownership.
+     * @param promise
+     *          promise to satisfy with current lock owner
+     */
+    private void checkLockOwnerAndWaitIfPossible(final LockWatcher lockWatcher,
+                                                 final boolean wait,
+                                                 final Promise<String> promise) {
+        zk.getChildren(lockPath, false, new AsyncCallback.Children2Callback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx, List<String> children, Stat stat) {
+                processLockWaiters(lockWatcher, wait, rc, children, promise);
+            }
+        }, null);
+    }
+
+    /**
+     * Check Lock Owner Phase 2 : check all lock waiters to get current owner and wait for ownership if necessary.
+     *
+     * @param lockWatcher
+     *          lock watcher.
+     * @param wait
+     *          whether to wait for ownership.
+     * @param getChildrenRc
+     *          result of getting all lock waiters
+     * @param children
+     *          current lock waiters.
+     * @param promise
+     *          promise to satisfy with current lock owner.
+     */
+    private void processLockWaiters(final LockWatcher lockWatcher,
+                                    final boolean wait,
+                                    final int getChildrenRc,
+                                    final List<String> children,
+                                    final Promise<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()));
+                    return;
+                }
+
+                if (KeeperException.Code.OK.intValue() != getChildrenRc) {
+                    promise.setException(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));
+                    return;
+                }
+
+                // sort the children
+                Collections.sort(children, MEMBER_COMPARATOR);
+                final String cid = currentId;
+                final int memberIndex = children.indexOf(cid);
+                if (LOG.isDebugEnabled()) {
+                    LOG.debug("{} is the number {} member in the list.", cid, memberIndex);
+                }
+                // If we hold the lock
+                if (memberIndex == 0) {
+                    LOG.info("{} acquired the lock {}.", cid, lockPath);
+                    claimOwnership(lockWatcher.epoch);
+                    promise.setValue(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>>() {
+                        @Override
+                        public void onSuccess(Pair<String, Long> currentOwner) {
+                            watchLockOwner(lockWatcher, wait,
+                                    cid, children.get(memberIndex - 1), children.get(0), currentOwner, promise);
+                        }
+                        @Override
+                        public void onFailure(final Throwable cause) {
+                            // ensure promise is satisfied in lock thread
+                            executeLockAction(lockWatcher.epoch, new LockAction() {
+                                @Override
+                                public void execute() {
+                                    promise.setException(cause);
+                                }
+
+                                @Override
+                                public String getActionName() {
+                                    return "handleFailureOnParseClientID(lockPath=" + lockPath + ")";
+                                }
+                            }, promise);
+                        }
+                    });
+                } else {
+                    LOG.error("Member {} doesn't exist in the members list {} for lock {}.",
+                            new Object[]{ cid, children, lockPath});
+                    promise.setException(
+                            new UnexpectedException("Member " + cid + " doesn't exist in member list " +
+                                    children + " for lock " + lockPath));
+                }
+            }
+
+            @Override
+            public String getActionName() {
+                return "processLockWaiters(rc=" + getChildrenRc + ", waiters=" + children + ")";
+            }
+        }, promise);
+    }
+
+    /**
+     * Check Lock Owner Phase 3: watch sibling node for lock ownership.
+     *
+     * @param lockWatcher
+     *          lock watcher.
+     * @param wait
+     *          whether to wait for ownership.
+     * @param myNode
+     *          my lock node.
+     * @param siblingNode
+     *          my sibling lock node.
+     * @param ownerNode
+     *          owner lock node.
+     * @param currentOwner
+     *          current owner info.
+     * @param promise
+     *          promise to satisfy with current lock owner.
+     */
+    private void watchLockOwner(final LockWatcher lockWatcher,
+                                final boolean wait,
+                                final String myNode,
+                                final String siblingNode,
+                                final String ownerNode,
+                                final Pair<String, Long> currentOwner,
+                                final Promise<String> promise) {
+        executeLockAction(lockWatcher.epoch, new LockAction() {
+            @Override
+            public void execute() {
+                boolean shouldWatch;
+                final boolean shouldClaimOwnership;
+                if (lockContext.hasLockId(currentOwner) && siblingNode.equals(ownerNode)) {
+                    // if the current owner is the znode left from previous session
+                    // we should watch it and claim ownership
+                    shouldWatch = true;
+                    shouldClaimOwnership = true;
+                    LOG.info("LockWatcher {} for {} found its previous session {} held lock, watch it to claim ownership.",
+                            new Object[] { myNode, lockPath, currentOwner });
+                } else if (lockId.compareTo(currentOwner) == 0 && areLockWaitersInSameSession(siblingNode, ownerNode)) {
+                    // I found that my sibling is the current owner with same lock id (client id & session id)
+                    // It must be left by any race condition from same zookeeper client
+                    shouldWatch = true;
+                    shouldClaimOwnership = true;
+                    LOG.info("LockWatcher {} for {} found itself {} already held lock at sibling node {}, watch it to claim ownership.",
+                            new Object[]{myNode, lockPath, lockId, siblingNode});
+                } else {
+                    shouldWatch = wait;
+                    if (wait) {
+                        if (LOG.isDebugEnabled()) {
+                            LOG.debug("Current LockWatcher for {} with ephemeral node {}, is waiting for {} to release lock at {}.",
+                                    new Object[]{lockPath, myNode, siblingNode, System.currentTimeMillis()});
+                        }
+                    }
+                    shouldClaimOwnership = false;
+                }
+
+                // watch sibling for lock ownership
+                if (shouldWatch) {
+                    watchedNode = String.format("%s/%s", lockPath, siblingNode);
+                    zk.exists(watchedNode, lockWatcher, new AsyncCallback.StatCallback() {
+                        @Override
+                        public void processResult(final int rc, String path, Object ctx, final Stat stat) {
+                            executeLockAction(lockWatcher.epoch, new LockAction() {
+                                @Override
+                                public void execute() {
+                                    if (!lockState.inState(State.PREPARED)) {
+                                        promise.setException(new LockStateChangedException(lockPath, lockId, State.PREPARED, lockState.getState()));
+                                        return;
+                                    }
+
+                                    if (KeeperException.Code.OK.intValue() == rc) {
+                                        if (shouldClaimOwnership) {
+                                            // watch owner successfully
+                                            LOG.info("LockWatcher {} claimed ownership for {} after set watcher on {}.",
+                                                    new Object[]{ myNode, lockPath, ownerNode });
+                                            claimOwnership(lockWatcher.epoch);
+                                            promise.setValue(currentOwner.getLeft());
+                                        } else {
+                                            // watch sibling successfully
+                                            lockState.transition(State.WAITING);
+                                            promise.setValue(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)));
+                                    }
+                                }
+
+                                @Override
+                                public String getActionName() {
+                                    StringBuilder sb = new StringBuilder();
+                                    sb.append("postWatchLockOwner(myNode=").append(myNode).append(", siblingNode=")
+                                            .append(siblingNode).append(", ownerNode=").append(ownerNode).append(")");
+                                    return sb.toString();
+                                }
+                            }, promise);
+                        }
+                    }, null);
+                } else {
+                    promise.setValue(currentOwner.getLeft());
+                }
+            }
+
+            @Override
+            public String getActionName() {
+                StringBuilder sb = new StringBuilder();
+                sb.append("watchLockOwner(myNode=").append(myNode).append(", siblingNode=")
+                        .append(siblingNode).append(", ownerNode=").append(ownerNode).append(")");
+                return sb.toString();
+            }
+        }, promise);
+    }
+
+    class LockWatcher implements Watcher {
+
+        // Enforce a epoch number to avoid a race on canceling attempt
+        final int epoch;
+
+        LockWatcher(int epoch) {
+            this.epoch = epoch;
+        }
+
+        @Override
+        public void process(WatchedEvent event) {
+            LOG.debug("Received event {} from lock {} at {} : watcher epoch {}, lock epoch {}.",
+                    new Object[] { event, lockPath, System.currentTimeMillis(), epoch, ZKSessionLock.this.epoch.get() });
+            if (event.getType() == Watcher.Event.EventType.None) {
+                switch (event.getState()) {
+                    case SyncConnected:
+                        break;
+                    case Expired:
+                        LOG.info("Session {} is expired for lock {} at {} : watcher epoch {}, lock epoch {}.",
+                                new Object[] { lockId.getRight(), lockPath, System.currentTimeMillis(),
+                                        epoch, ZKSessionLock.this.epoch.get() });
+                        handleSessionExpired(epoch);
+                        break;
+                    default:
+                        break;
+                }
+            } else if (event.getType() == Event.EventType.NodeDeleted) {
+                // this handles the case where we have aborted a lock and deleted ourselves but still have a
+                // watch on the nextLowestNode. This is a workaround since ZK doesn't support unsub.
+                if (!event.getPath().equals(watchedNode)) {
+                    LOG.warn("{} (watching {}) ignored watched event from {} ",
+                            new Object[] { lockId, watchedNode, event.getPath() });
+                    return;
+                }
+                handleNodeDelete(epoch, event);
+            } else {
+                LOG.warn("Unexpected ZK event: {}", event.getType().name());
+            }
+        }
+
+    }
+}
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
new file mode 100644
index 0000000..3cb25f0
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/ZKSessionLockFactory.java
@@ -0,0 +1,133 @@
+/**
+ * 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.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.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * Factory to create zookeeper based locks.
+ */
+public class ZKSessionLockFactory implements SessionLockFactory {
+
+    private final ZooKeeperClient zkc;
+    private final String clientId;
+    private final OrderedScheduler lockStateExecutor;
+    private final long lockOpTimeout;
+    private final int lockCreationRetries;
+    private final long zkRetryBackoffMs;
+
+    // Stats
+    private final StatsLogger lockStatsLogger;
+
+    public ZKSessionLockFactory(ZooKeeperClient zkc,
+                                String clientId,
+                                OrderedScheduler lockStateExecutor,
+                                int lockCreationRetries,
+                                long lockOpTimeout,
+                                long zkRetryBackoffMs,
+                                StatsLogger statsLogger) {
+        this.zkc = zkc;
+        this.clientId = clientId;
+        this.lockStateExecutor = lockStateExecutor;
+        this.lockCreationRetries = lockCreationRetries;
+        this.lockOpTimeout = lockOpTimeout;
+        this.zkRetryBackoffMs = zkRetryBackoffMs;
+
+        this.lockStatsLogger = statsLogger.scope("lock");
+    }
+
+    @Override
+    public Future<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;
+            }
+        });
+        createLock(
+                lockPath,
+                context,
+                interruptedException,
+                numRetries,
+                createPromise,
+                0L);
+        return createPromise;
+    }
+
+    void createLock(final String lockPath,
+                    final DistributedLockContext context,
+                    final AtomicReference<Throwable> interruptedException,
+                    final AtomicInteger numRetries,
+                    final Promise<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()));
+                    return;
+                }
+                try {
+                    SessionLock lock = new ZKSessionLock(
+                            zkc,
+                            lockPath,
+                            clientId,
+                            lockStateExecutor,
+                            lockOpTimeout,
+                            lockStatsLogger,
+                            context);
+                    createPromise.updateIfEmpty(new Return<SessionLock>(lock));
+                } catch (DLInterruptedException dlie) {
+                    // if the creation is interrupted, throw the exception without retrie.
+                    createPromise.updateIfEmpty(new Throw<SessionLock>(dlie));
+                    return;
+                } catch (IOException e) {
+                    if (numRetries.getAndDecrement() < 0) {
+                        createPromise.updateIfEmpty(new Throw<SessionLock>(e));
+                        return;
+                    }
+                    createLock(
+                            lockPath,
+                            context,
+                            interruptedException,
+                            numRetries,
+                            createPromise,
+                            zkRetryBackoffMs);
+                }
+            }
+        }, delayMs, TimeUnit.MILLISECONDS);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/lock/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/package-info.java
new file mode 100644
index 0000000..baeaab2
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/lock/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Distributed locking mechanism in distributedlog
+ */
+package org.apache.distributedlog.lock;
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
new file mode 100644
index 0000000..8440509
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryReader.java
@@ -0,0 +1,113 @@
+/**
+ * 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.logsegment;
+
+import com.google.common.annotations.Beta;
+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
+ * {@link org.apache.distributedlog.Entry}) from a log segment
+ */
+@Beta
+public interface LogSegmentEntryReader extends AsyncCloseable {
+
+    interface StateChangeListener {
+
+        /**
+         * Notify when caught up on inprogress.
+         */
+        void onCaughtupOnInprogress();
+
+    }
+
+    /**
+     * Start the reader. The method to signal the implementation
+     * to start preparing the data for consumption {@link #readNext(int)}
+     */
+    void start();
+
+    /**
+     * Register the state change listener
+     *
+     * @param listener register the state change listener
+     * @return entry reader
+     */
+    LogSegmentEntryReader registerListener(StateChangeListener listener);
+
+    /**
+     * Unregister the state change listener
+     *
+     * @param listener register the state change listener
+     * @return entry reader
+     */
+    LogSegmentEntryReader unregisterListener(StateChangeListener listener);
+
+    /**
+     * Return the log segment metadata for this reader.
+     *
+     * @return the log segment metadata
+     */
+    LogSegmentMetadata getSegment();
+
+    /**
+     * Update the log segment each time when the metadata has changed.
+     *
+     * @param segment new metadata of the log segment.
+     */
+    void onLogSegmentMetadataUpdated(LogSegmentMetadata segment);
+
+    /**
+     * Read next <i>numEntries</i> entries from current log segment.
+     * <p>
+     * <i>numEntries</i> will be best-effort.
+     *
+     * @param numEntries num entries to read from current log segment
+     * @return A promise that when satisified will contain a non-empty list of entries with their content.
+     * @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);
+
+    /**
+     * Return the last add confirmed entry id (LAC).
+     *
+     * @return the last add confirmed entry id.
+     */
+    long getLastAddConfirmed();
+
+    /**
+     * Is the reader reading beyond last add confirmed.
+     *
+     * @return true if the reader is reading beyond last add confirmed
+     */
+    boolean isBeyondLastAddConfirmed();
+
+    /**
+     * Has the log segment reader caught up with the inprogress log segment.
+     *
+     * @return true only if the log segment is inprogress and it is caught up, otherwise return false.
+     */
+    boolean hasCaughtUpOnInprogress();
+
+}
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
new file mode 100644
index 0000000..40be67b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryStore.java
@@ -0,0 +1,73 @@
+/**
+ * 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.logsegment;
+
+import com.google.common.annotations.Beta;
+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;
+
+/**
+ * Log Segment Store to read log segments
+ */
+@Beta
+public interface LogSegmentEntryStore {
+
+    /**
+     * Delete the actual log segment from the entry store.
+     *
+     * @param segment log segment metadata
+     * @return future represent the delete result
+     */
+    Future<LogSegmentMetadata> deleteLogSegment(LogSegmentMetadata segment);
+
+    /**
+     * Create a new log segment allocator for allocating log segment entry writers.
+     *
+     * @param metadata the metadata for the log stream
+     * @return future represent the log segment allocator
+     */
+    Allocator<LogSegmentEntryWriter, Object> newLogSegmentAllocator(
+            LogMetadataForWriter metadata,
+            DynamicDistributedLogConfiguration dynConf) throws IOException;
+
+    /**
+     * Open the reader for reading data to the log <i>segment</i>.
+     *
+     * @param segment the log <i>segment</i> to read data from
+     * @param startEntryId the start entry id
+     * @return future represent the opened reader
+     */
+    Future<LogSegmentEntryReader> openReader(LogSegmentMetadata segment,
+                                             long startEntryId);
+
+    /**
+     * Open the reader for reading entries from a random access log <i>segment</i>.
+     *
+     * @param segment the log <i>segment</i> to read entries from
+     * @param fence the flag to fence log segment
+     * @return future represent the opened random access reader
+     */
+    Future<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
new file mode 100644
index 0000000..660592e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentEntryWriter.java
@@ -0,0 +1,72 @@
+/**
+ * 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.logsegment;
+
+import com.google.common.annotations.Beta;
+import org.apache.distributedlog.Entry;
+import org.apache.distributedlog.util.Sizable;
+import org.apache.bookkeeper.client.AsyncCallback;
+
+/**
+ * An interface class to write the enveloped entry (serialized bytes of
+ * {@link Entry} into the log segment.
+ *
+ * <p>It is typically used by {@link LogSegmentWriter}.
+ *
+ * @see LogSegmentWriter
+ *
+ * TODO: The interface is leveraging bookkeeper's callback and status code now
+ *       Consider making it more generic.
+ */
+@Beta
+public interface LogSegmentEntryWriter extends Sizable {
+
+    /**
+     * Get the log segment id.
+     *
+     * @return log segment id.
+     */
+    long getLogSegmentId();
+
+    /**
+     * Close the entry writer.
+     */
+    void asyncClose(AsyncCallback.CloseCallback callback, Object ctx);
+
+    /**
+     * Async add entry to the log segment.
+     * <p>The implementation semantic follows
+     * {@link org.apache.bookkeeper.client.LedgerHandle#asyncAddEntry(
+     * byte[], int, int, AsyncCallback.AddCallback, Object)}
+     *
+     * @param data
+     *          data to add
+     * @param offset
+     *          offset in the data
+     * @param length
+     *          length of the data
+     * @param callback
+     *          callback
+     * @param ctx
+     *          ctx
+     * @see org.apache.bookkeeper.client.LedgerHandle#asyncAddEntry(
+     * byte[], int, int, AsyncCallback.AddCallback, Object)
+     */
+    void asyncAddEntry(byte[] data, int offset, int length,
+                       AsyncCallback.AddCallback callback, Object ctx);
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentFilter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentFilter.java
new file mode 100644
index 0000000..6a6d5e5
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentFilter.java
@@ -0,0 +1,42 @@
+/**
+ * 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.logsegment;
+
+import java.util.Collection;
+
+/**
+ * Filter to filter log segments
+ */
+public interface LogSegmentFilter {
+
+    public static final LogSegmentFilter DEFAULT_FILTER = new LogSegmentFilter() {
+        @Override
+        public Collection<String> filter(Collection<String> fullList) {
+            return fullList;
+        }
+    };
+
+    /**
+     * Filter the log segments from the full log segment list.
+     *
+     * @param fullList
+     *          full list of log segment names.
+     * @return filtered log segment names
+     */
+    Collection<String> filter(Collection<String> fullList);
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataCache.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataCache.java
new file mode 100644
index 0000000..abc0c74
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataCache.java
@@ -0,0 +1,98 @@
+/**
+ * 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.logsegment;
+
+import com.google.common.base.Ticker;
+import com.google.common.cache.Cache;
+import com.google.common.cache.CacheBuilder;
+import com.google.common.cache.RemovalListener;
+import com.google.common.cache.RemovalNotification;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Cache the log segment metadata
+ */
+public class LogSegmentMetadataCache implements RemovalListener<String, LogSegmentMetadata> {
+
+    private static final Logger logger = LoggerFactory.getLogger(LogSegmentMetadataCache.class);
+
+    private final Cache<String, LogSegmentMetadata> cache;
+    private final boolean isCacheEnabled;
+
+    public LogSegmentMetadataCache(DistributedLogConfiguration conf,
+                                   Ticker ticker) {
+        cache = CacheBuilder.newBuilder()
+                .concurrencyLevel(conf.getNumWorkerThreads())
+                .initialCapacity(1024)
+                .expireAfterAccess(conf.getLogSegmentCacheTTLMs(), TimeUnit.MILLISECONDS)
+                .maximumSize(conf.getLogSegmentCacheMaxSize())
+                .removalListener(this)
+                .ticker(ticker)
+                .recordStats()
+                .build();
+        this.isCacheEnabled = conf.isLogSegmentCacheEnabled();
+        logger.info("Log segment cache is enabled = {}", this.isCacheEnabled);
+    }
+
+    /**
+     * Add the log <i>segment</i> of <i>path</i> to the cache.
+     *
+     * @param path the path of the log segment
+     * @param segment log segment metadata
+     */
+    public void put(String path, LogSegmentMetadata segment) {
+        if (isCacheEnabled) {
+            cache.put(path, segment);
+        }
+    }
+
+    /**
+     * Invalid the cache entry associated with <i>path</i>.
+     *
+     * @param path the path of the log segment
+     */
+    public void invalidate(String path) {
+        if (isCacheEnabled) {
+            cache.invalidate(path);
+        }
+    }
+
+    /**
+     * Retrieve the log segment of <i>path</i> from the cache.
+     *
+     * @param path the path of the log segment.
+     * @return log segment metadata if exists, otherwise null.
+     */
+    public LogSegmentMetadata get(String path) {
+        return cache.getIfPresent(path);
+    }
+
+    @Override
+    public void onRemoval(RemovalNotification<String, LogSegmentMetadata> notification) {
+        if (notification.wasEvicted()) {
+            if (logger.isDebugEnabled()) {
+                logger.debug("Log segment of {} was evicted.", notification.getKey());
+            }
+        }
+    }
+}
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
new file mode 100644
index 0000000..fdf72b1
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentMetadataStore.java
@@ -0,0 +1,163 @@
+/**
+ * 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.logsegment;
+
+import com.google.common.annotations.Beta;
+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;
+
+/**
+ * Interface for log segment metadata store. All operations that modify log segments should
+ * be executed under a {@link Transaction}.
+ */
+@Beta
+public interface LogSegmentMetadataStore extends Closeable {
+
+    /**
+     * Start the transaction on changing log segment metadata store.
+     *
+     * @return transaction of the log segment metadata store.
+     */
+    Transaction<Object> transaction();
+
+    // The reason to keep storing log segment sequence number & log record transaction id
+    // in this log segment metadata store interface is to share the transaction that used
+    // to start/complete log segment. It is a bit hard to separate them out right now.
+
+    /**
+     * Store the maximum log segment sequence number on <code>path</code>.
+     *
+     * @param txn
+     *          transaction to execute for storing log segment sequence number.
+     * @param logMetadata
+     *          metadata of the log stream
+     * @param sequenceNumber
+     *          log segment sequence number to store
+     * @param listener
+     *          listener on the result to this operation
+     */
+    void storeMaxLogSegmentSequenceNumber(Transaction<Object> txn,
+                                          LogMetadata logMetadata,
+                                          Versioned<Long> sequenceNumber,
+                                          OpListener<Version> listener);
+
+    /**
+     * Store the maximum transaction id for <code>path</code>
+     *
+     * @param txn
+     *          transaction to execute for storing transaction id
+     * @param logMetadata
+     *          metadata of the log stream
+     * @param transactionId
+     *          transaction id to store
+     * @param listener
+     *          listener on the result to this operation
+     */
+    void storeMaxTxnId(Transaction<Object> txn,
+                       LogMetadataForWriter logMetadata,
+                       Versioned<Long> transactionId,
+                       OpListener<Version> listener);
+
+    /**
+     * Create a log segment <code>segment</code> under transaction <code>txn</code>.
+     *
+     * NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
+     *       immediately. the operation should be executed via {@link Transaction#execute()}
+     *
+     * @param txn
+     *          transaction to execute for this operation
+     * @param segment
+     *          segment to create
+     * @param opListener
+     *          the listener on the operation result
+     */
+    void createLogSegment(Transaction<Object> txn,
+                          LogSegmentMetadata segment,
+                          OpListener<Void> opListener);
+
+    /**
+     * Delete a log segment <code>segment</code> under transaction <code>txn</code>.
+     *
+     * NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
+     *       immediately. the operation should be executed via {@link Transaction#execute()}
+     *
+     * @param txn
+     *          transaction to execute for this operation
+     * @param segment
+     *          segment to delete
+     */
+    void deleteLogSegment(Transaction<Object> txn,
+                          LogSegmentMetadata segment,
+                          OpListener<Void> opListener);
+
+    /**
+     * Update a log segment <code>segment</code> under transaction <code>txn</code>.
+     *
+     * NOTE: this operation shouldn't be a blocking call. and it shouldn't execute the operation
+     *       immediately. the operation should be executed via {@link Transaction#execute()}
+     *
+     * @param txn
+     *          transaction to execute for this operation
+     * @param segment
+     *          segment to update
+     */
+    void updateLogSegment(Transaction<Object> txn, LogSegmentMetadata segment);
+
+    /**
+     * Retrieve the log segment associated <code>path</code>.
+     *
+     * @param logSegmentPath
+     *          path to store log segment metadata
+     * @return future of the retrieved log segment metadata
+     */
+    Future<LogSegmentMetadata> getLogSegment(String logSegmentPath);
+
+    /**
+     * Retrieve the list of log segments under <code>logSegmentsPath</code> and register a <i>listener</i>
+     * for subsequent changes for the list of log segments.
+     *
+     * @param logSegmentsPath
+     *          path to store list of log segments
+     * @param listener
+     *          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);
+
+    /**
+     * Unregister a log segment <code>listener</code> on log segment changes under <code>logSegmentsPath</code>.
+     *
+     * @param logSegmentsPath
+     *          log segments path
+     * @param listener
+     *          log segment listener on log segment changes
+     */
+    void unregisterLogSegmentListener(String logSegmentsPath,
+                                      LogSegmentNamesListener listener);
+}
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
new file mode 100644
index 0000000..948ce30
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java
@@ -0,0 +1,47 @@
+/**
+ * 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.logsegment;
+
+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}
+ * from a random access log segment.
+ */
+public interface LogSegmentRandomAccessEntryReader extends AsyncCloseable {
+
+    /**
+     * Read entries [startEntryId, endEntryId] from a random access log segment.
+     *
+     * @param startEntryId start entry id
+     * @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);
+
+    /**
+     * Return the last add confirmed entry id (LAC).
+     *
+     * @return the last add confirmed entry id.
+     */
+    long getLastAddConfirmed();
+}
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
new file mode 100644
index 0000000..39c94f4
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/LogSegmentWriter.java
@@ -0,0 +1,87 @@
+/**
+ * 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.logsegment;
+
+import com.google.common.annotations.Beta;
+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.
+ */
+@Beta
+public interface LogSegmentWriter extends AsyncCloseable, AsyncAbortable {
+
+    /**
+     * Get the unique log segment id.
+     *
+     * @return log segment id.
+     */
+    public long getLogSegmentId();
+
+    /**
+     * Write a log record to a log segment.
+     *
+     * @param record single log record
+     * @return a future representing write result. A {@link DLSN} is returned if write succeeds,
+     *         otherwise, exceptions are returned.
+     * @throws org.apache.distributedlog.exceptions.LogRecordTooLongException if log record is too long
+     * @throws org.apache.distributedlog.exceptions.InvalidEnvelopedEntryException on invalid enveloped entry
+     * @throws LockingException if failed to acquire lock for the writer
+     * @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);
+
+    /**
+     * This isn't a simple synchronous version of {@code asyncWrite}. It has different semantic.
+     * This method only writes data to the buffer and flushes buffer if needed.
+     *
+     * TODO: we should remove this method. when we rewrite synchronous writer based on asynchronous writer,
+     *       since this is the semantic needed to be provided in higher level but just calling write & flush.
+     *
+     * @param record single log record
+     * @throws IOException when tried to flush the buffer.
+     * @see LogSegmentWriter#asyncWrite(LogRecord)
+     */
+    public void write(LogRecord record) throws IOException;
+
+    /**
+     * Transmit the buffered data and wait for it being persisted and return the last acknowledged
+     * transaction id.
+     *
+     * @return future representing the transmit result with last acknowledged transaction id.
+     */
+    public Future<Long> flush();
+
+    /**
+     * Commit the current acknowledged data. It is the consequent operation of {@link #flush()},
+     * which makes all the acknowledged data visible to
+     *
+     * @return future representing the commit result.
+     */
+    public Future<Long> commit();
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java
new file mode 100644
index 0000000..7929505
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/PerStreamLogSegmentCache.java
@@ -0,0 +1,243 @@
+/**
+ * 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.logsegment;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Managing log segments in local cache.
+ *
+ * <p>
+ * Caching of log segment metadata assumes that the data contained in the ZNodes for individual
+ * log segments is never updated after creation i.e we never call setData. A log segment
+ * is finalized by creating a new ZNode and deleting the in progress node. This code will have
+ * to change if we change the behavior
+ * </p>
+ */
+public class PerStreamLogSegmentCache {
+
+    static final Logger LOG = LoggerFactory.getLogger(PerStreamLogSegmentCache.class);
+
+    protected final String streamName;
+    protected final boolean validateLogSegmentSequenceNumber;
+    protected final Map<String, LogSegmentMetadata> logSegments =
+            new HashMap<String, LogSegmentMetadata>();
+    protected final ConcurrentMap<Long, LogSegmentMetadata> lid2LogSegments =
+            new ConcurrentHashMap<Long, LogSegmentMetadata>();
+
+    @VisibleForTesting
+    PerStreamLogSegmentCache(String streamName) {
+        this(streamName, true);
+    }
+
+    public PerStreamLogSegmentCache(String streamName,
+                                    boolean validateLogSegmentSequenceNumber) {
+        this.streamName = streamName;
+        this.validateLogSegmentSequenceNumber = validateLogSegmentSequenceNumber;
+    }
+
+    /**
+     * Retrieve log segments from the cache.
+     *
+     * - first sort the log segments in ascending order
+     * - do validation and assign corresponding sequence id
+     * - apply comparator after validation
+     *
+     * @param comparator
+     *          comparator to sort the returned log segments.
+     * @return list of sorted and filtered log segments.
+     * @throws UnexpectedException if unexpected condition detected (e.g. ledger sequence number gap)
+     */
+    public List<LogSegmentMetadata> getLogSegments(Comparator<LogSegmentMetadata> comparator)
+        throws UnexpectedException {
+        List<LogSegmentMetadata> segmentsToReturn;
+        synchronized (logSegments) {
+            segmentsToReturn = new ArrayList<LogSegmentMetadata>(logSegments.size());
+            segmentsToReturn.addAll(logSegments.values());
+        }
+        Collections.sort(segmentsToReturn, LogSegmentMetadata.COMPARATOR);
+
+        LogSegmentMetadata prevSegment = null;
+        if (validateLogSegmentSequenceNumber) {
+            // validation ledger sequence number to ensure the log segments are unique.
+            for (int i = 0; i < segmentsToReturn.size(); i++) {
+                LogSegmentMetadata segment = segmentsToReturn.get(i);
+
+                if (null != prevSegment
+                        && prevSegment.getVersion() >= LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value
+                        && segment.getVersion() >= LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value
+                        && prevSegment.getLogSegmentSequenceNumber() + 1 != segment.getLogSegmentSequenceNumber()) {
+                    LOG.error("{} found ledger sequence number gap between log segment {} and {}",
+                            new Object[] { streamName, prevSegment, segment });
+                    throw new UnexpectedException(streamName + " found ledger sequence number gap between log segment "
+                            + prevSegment.getLogSegmentSequenceNumber() + " and " + segment.getLogSegmentSequenceNumber());
+                }
+                prevSegment = segment;
+            }
+        }
+
+        prevSegment = null;
+        long startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
+        for (int i = 0; i < segmentsToReturn.size(); i++) {
+                LogSegmentMetadata segment = segmentsToReturn.get(i);
+            // assign sequence id
+            if (!segment.isInProgress()) {
+                if (segment.supportsSequenceId()) {
+                    startSequenceId = segment.getStartSequenceId() + segment.getRecordCount();
+                    if (null != prevSegment && prevSegment.supportsSequenceId()
+                            && prevSegment.getStartSequenceId() > segment.getStartSequenceId()) {
+                        LOG.warn("{} found decreasing start sequence id in log segment {}, previous is {}",
+                                new Object[] { streamName, segment, prevSegment });
+                    }
+                } else {
+                    startSequenceId = DistributedLogConstants.UNASSIGNED_SEQUENCE_ID;
+                }
+            } else {
+                if (segment.supportsSequenceId()) {
+                    LogSegmentMetadata newSegment = segment.mutator()
+                            .setStartSequenceId(startSequenceId == DistributedLogConstants.UNASSIGNED_SEQUENCE_ID ? 0L : startSequenceId)
+                            .build();
+                    segmentsToReturn.set(i, newSegment);
+                }
+
+                break;
+            }
+            prevSegment = segment;
+        }
+        if (comparator != LogSegmentMetadata.COMPARATOR) {
+            Collections.sort(segmentsToReturn, comparator);
+        }
+        return segmentsToReturn;
+    }
+
+    /**
+     * Add the segment <i>metadata</i> for <i>name</i> in the cache.
+     *
+     * @param name
+     *          segment name.
+     * @param metadata
+     *          segment metadata.
+     */
+    public void add(String name, LogSegmentMetadata metadata) {
+        synchronized (logSegments) {
+            if (!logSegments.containsKey(name)) {
+                logSegments.put(name, metadata);
+                LOG.info("{} added log segment ({} : {}) to cache.",
+                        new Object[]{ streamName, name, metadata });
+            }
+            LogSegmentMetadata oldMetadata = lid2LogSegments.remove(metadata.getLogSegmentId());
+            if (null == oldMetadata) {
+                lid2LogSegments.put(metadata.getLogSegmentId(), metadata);
+            } else {
+                if (oldMetadata.isInProgress() && !metadata.isInProgress()) {
+                    lid2LogSegments.put(metadata.getLogSegmentId(), metadata);
+                } else {
+                    lid2LogSegments.put(oldMetadata.getLogSegmentId(), oldMetadata);
+                }
+            }
+        }
+    }
+
+    /**
+     * Retrieve log segment <code>name</code> from the cache.
+     *
+     * @param name
+     *          name of the log segment.
+     * @return log segment metadata
+     */
+    public LogSegmentMetadata get(String name) {
+        synchronized (logSegments) {
+            return logSegments.get(name);
+        }
+    }
+
+    /**
+     * Update the log segment cache with removed/added segments.
+     *
+     * @param segmentsRemoved
+     *          segments that removed
+     * @param segmentsAdded
+     *          segments that added
+     */
+    public void update(Set<String> segmentsRemoved,
+                       Map<String, LogSegmentMetadata> segmentsAdded) {
+        synchronized (logSegments) {
+            for (Map.Entry<String, LogSegmentMetadata> entry : segmentsAdded.entrySet()) {
+                add(entry.getKey(), entry.getValue());
+            }
+            for (String segment : segmentsRemoved) {
+                remove(segment);
+            }
+        }
+    }
+
+    /**
+     * Diff with new received segment list <code>segmentReceived</code>.
+     *
+     * @param segmentsReceived
+     *          new received segment list
+     * @return segments added (left) and removed (right).
+     */
+    public Pair<Set<String>, Set<String>> diff(Set<String> segmentsReceived) {
+        Set<String> segmentsAdded;
+        Set<String> segmentsRemoved;
+        synchronized (logSegments) {
+            Set<String> segmentsCached = logSegments.keySet();
+            segmentsAdded = Sets.difference(segmentsReceived, segmentsCached).immutableCopy();
+            segmentsRemoved = Sets.difference(segmentsCached, segmentsReceived).immutableCopy();
+        }
+        return Pair.of(segmentsAdded, segmentsRemoved);
+    }
+
+    /**
+     * Remove log segment <code>name</code> from the cache.
+     *
+     * @param name
+     *          name of the log segment.
+     * @return log segment metadata.
+     */
+    public LogSegmentMetadata remove(String name) {
+        synchronized (logSegments) {
+            LogSegmentMetadata metadata = logSegments.remove(name);
+            if (null != metadata) {
+                lid2LogSegments.remove(metadata.getLogSegmentId(), metadata);
+                LOG.debug("Removed log segment ({} : {}) from cache.", name, metadata);
+            }
+            return metadata;
+        }
+    }
+
+
+}
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
new file mode 100644
index 0000000..ce98ac9
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/RollingPolicy.java
@@ -0,0 +1,33 @@
+/**
+ * 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.logsegment;
+
+import org.apache.distributedlog.util.Sizable;
+
+public interface RollingPolicy {
+    /**
+     * Determines if a rollover may be appropriate at this time.
+     *
+     * @param sizable
+     *          Any object that is sizable.
+     * @param lastRolloverTimeMs
+     *          last rolling time in millis.
+     * @return true if a rollover is required. otherwise, false.
+     */
+    boolean shouldRollover(Sizable sizable, long lastRolloverTimeMs);
+}
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
new file mode 100644
index 0000000..544f58e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/SizeBasedRollingPolicy.java
@@ -0,0 +1,34 @@
+/**
+ * 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.logsegment;
+
+import org.apache.distributedlog.util.Sizable;
+
+public class SizeBasedRollingPolicy implements RollingPolicy {
+
+    final long maxSize;
+
+    public SizeBasedRollingPolicy(long maxSize) {
+        this.maxSize = maxSize;
+    }
+
+    @Override
+    public boolean shouldRollover(Sizable sizable, long lastRolloverTimeMs) {
+        return sizable.size() > maxSize;
+    }
+}
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
new file mode 100644
index 0000000..141f139
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/TimeBasedRollingPolicy.java
@@ -0,0 +1,46 @@
+/**
+ * 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.logsegment;
+
+import org.apache.distributedlog.util.Sizable;
+import org.apache.distributedlog.util.Utils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TimeBasedRollingPolicy implements RollingPolicy {
+
+    final static Logger LOG = LoggerFactory.getLogger(TimeBasedRollingPolicy.class);
+
+    final long rollingIntervalMs;
+
+    public TimeBasedRollingPolicy(long rollingIntervalMs) {
+        this.rollingIntervalMs = rollingIntervalMs;
+    }
+
+    @Override
+    public boolean shouldRollover(Sizable sizable, long lastRolloverTimeMs) {
+        long elapsedMs = Utils.elapsedMSec(lastRolloverTimeMs);
+        boolean shouldSwitch = elapsedMs > rollingIntervalMs;
+        if (shouldSwitch) {
+            LOG.debug("Last Finalize Time: {} elapsed time (MSec): {}", lastRolloverTimeMs,
+                      elapsedMs);
+        }
+        return shouldSwitch;
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/package-info.java
new file mode 100644
index 0000000..5b0c497
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/logsegment/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Log Segment Management
+ */
+package org.apache.distributedlog.logsegment;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLConfig.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLConfig.java
new file mode 100644
index 0000000..f9cd27b
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLConfig.java
@@ -0,0 +1,39 @@
+/**
+ * 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.metadata;
+
+import java.io.IOException;
+
+/**
+ * Specific config of a given implementation of DL
+ */
+public interface DLConfig {
+    /**
+     * Serialize the dl config into a string.
+     */
+    public String serialize();
+
+    /**
+     * Deserialize the dl config from a readable stream.
+     *
+     * @param data
+     *          bytes to desrialize dl config.
+     * @throws IOException if fail to deserialize the dl config string representation.
+     */
+    public void deserialize(byte[] data) throws IOException;
+}
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
new file mode 100644
index 0000000..7339d55
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DLMetadata.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.metadata;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.util.Utils;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.StringReader;
+import java.net.URI;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Metadata of a given DL instance.
+ */
+public class DLMetadata {
+
+    static final Logger LOG = LoggerFactory.getLogger(DLMetadata.class);
+
+    static final String LINE_SPLITTER = "\n";
+    static final String BK_DL_TYPE = "BKDL";
+    static final int METADATA_FORMAT_VERSION = 1;
+
+    // metadata format version
+    private int metadataFormatVersion = 0;
+    // underlying dl type
+    private String dlType;
+    // underlying dl config
+    private DLConfig dlConfig;
+
+    public DLMetadata(String dlType, DLConfig dlConfig) {
+        this(dlType, dlConfig, METADATA_FORMAT_VERSION);
+    }
+
+    DLMetadata(String dlType, DLConfig dlConfig, int metadataFormatVersion) {
+        this.dlType = dlType;
+        this.dlConfig = dlConfig;
+        this.metadataFormatVersion = metadataFormatVersion;
+    }
+
+    /**
+     * @return DL type
+     */
+    public String getDLType() {
+        return dlType;
+    }
+
+    /**
+     * @return DL Config
+     */
+    public DLConfig getDLConfig() {
+        return dlConfig;
+    }
+
+    /**
+     * Serialize the DL metadata into bytes array.
+     *
+     * @return bytes of DL metadata.
+     */
+    public byte[] serialize() {
+        StringBuilder sb = new StringBuilder();
+        sb.append(metadataFormatVersion).append(LINE_SPLITTER);
+        sb.append(dlType).append(LINE_SPLITTER);
+        sb.append(dlConfig.serialize());
+        LOG.debug("Serialized dl metadata {}.", sb);
+        return sb.toString().getBytes(UTF_8);
+    }
+
+    @Override
+    public int hashCode() {
+        return dlType.hashCode() * 13 + dlConfig.hashCode();
+    }
+
+    @Override
+    public String toString() {
+        return new String(serialize(), UTF_8);
+    }
+
+    public void update(URI uri) throws IOException {
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
+                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                .retryThreadCount(conf.getZKClientNumberRetryThreads())
+                .requestRateLimit(conf.getZKRequestRateLimit())
+                .zkAclId(conf.getZkAclId())
+                .uri(uri)
+                .build();
+        byte[] data = serialize();
+        try {
+            zkc.get().setData(uri.getPath(), data, -1);
+        } catch (KeeperException e) {
+            throw new IOException("Fail to update dl metadata " + new String(data, UTF_8)
+                    + " to uri " + uri, e);
+        } catch (InterruptedException e) {
+            throw new IOException("Interrupted when updating dl metadata "
+                    + new String(data, UTF_8) + " to uri " + uri, e);
+        } finally {
+            zkc.close();
+        }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof DLMetadata)) {
+            return false;
+        }
+        DLMetadata other = (DLMetadata) o;
+        return dlType.equals(other.dlType) && dlConfig.equals(other.dlConfig);
+    }
+
+    public void create(URI uri) throws IOException {
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
+                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                .retryThreadCount(conf.getZKClientNumberRetryThreads())
+                .requestRateLimit(conf.getZKRequestRateLimit())
+                .zkAclId(conf.getZkAclId())
+                .uri(uri)
+                .build();
+        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);
+        } finally {
+            zkc.close();
+        }
+    }
+
+    public static void unbind(URI uri) throws IOException {
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        ZooKeeperClient zkc = ZooKeeperClientBuilder.newBuilder()
+                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                .retryThreadCount(conf.getZKClientNumberRetryThreads())
+                .requestRateLimit(conf.getZKRequestRateLimit())
+                .zkAclId(conf.getZkAclId())
+                .uri(uri)
+                .build();
+        byte[] data = new byte[0];
+        try {
+            zkc.get().setData(uri.getPath(), data, -1);
+        } catch (KeeperException ke) {
+            throw new IOException("Fail to unbound dl metadata on uri " + uri, ke);
+        } catch (InterruptedException ie) {
+            throw new IOException("Interrupted when unbinding dl metadata on uri " + uri, ie);
+        } finally {
+            zkc.close();
+        }
+    }
+
+    /**
+     * Deserialize dl metadata of given <i>uri</i> from a given bytes array.
+     *
+     * @param uri
+     *          uri that stored dl metadata bindings
+     * @param data
+     *          bytes of dl metadata
+     * @return dl metadata
+     * @throws IOException if failed to parse the bytes array
+     */
+    public static DLMetadata deserialize(URI uri, byte[] data) throws IOException {
+        String metadata = new String(data, UTF_8);
+        LOG.debug("Parsing dl metadata {}.", metadata);
+        BufferedReader br = new BufferedReader(new StringReader(metadata));
+        String versionLine = br.readLine();
+        if (null == versionLine) {
+            throw new IOException("Empty DL Metadata.");
+        }
+        int version;
+        try {
+            version = Integer.parseInt(versionLine);
+        } catch (NumberFormatException nfe) {
+            version = -1;
+        }
+        if (METADATA_FORMAT_VERSION != version) {
+            throw new IOException("Metadata version not compatible. Expected "
+                    + METADATA_FORMAT_VERSION + " but got " + version);
+        }
+        String type = br.readLine();
+        if (!BK_DL_TYPE.equals(type)) {
+            throw new IOException("Invalid DL type : " + type);
+        }
+        DLConfig dlConfig = new BKDLConfig(uri);
+        StringBuilder sb = new StringBuilder();
+        String line;
+        while (null != (line = br.readLine())) {
+            sb.append(line);
+        }
+        dlConfig.deserialize(sb.toString().getBytes(UTF_8));
+        return new DLMetadata(type, dlConfig, version);
+    }
+
+    public static DLMetadata create(BKDLConfig bkdlConfig) {
+        return new DLMetadata(BK_DL_TYPE, bkdlConfig);
+    }
+
+}
+
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
new file mode 100644
index 0000000..6b835b9
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
@@ -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.
+ */
+package org.apache.distributedlog.metadata;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
+import org.apache.distributedlog.util.Transaction;
+import com.twitter.util.Future;
+
+public class DryrunLogSegmentMetadataStoreUpdater extends LogSegmentMetadataStoreUpdater {
+
+    public DryrunLogSegmentMetadataStoreUpdater(DistributedLogConfiguration conf,
+                                                LogSegmentMetadataStore metadataStore) {
+        super(conf, metadataStore);
+    }
+
+    @Override
+    public Transaction<Object> transaction() {
+        return new Transaction<Object>() {
+            @Override
+            public void addOp(Op<Object> operation) {
+                // no-op
+            }
+
+            @Override
+            public Future<Void> execute() {
+                return Future.Void();
+            }
+
+            @Override
+            public void abort(Throwable reason) {
+                // no-op
+            }
+        };
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadata.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadata.java
new file mode 100644
index 0000000..b78ebcf
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadata.java
@@ -0,0 +1,175 @@
+/**
+ * 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.metadata;
+
+import java.net.URI;
+
+/**
+ * Class to represent the layout and metadata of the zookeeper-based log metadata
+ */
+public class LogMetadata {
+
+    protected static String getLogComponentPath(URI uri, String logName, String logIdentifier, String component) {
+        return String.format("%s/%s/%s%s", uri.getPath(), logName, logIdentifier, component);
+    }
+
+    /**
+     * Get the top stream path for a given log.
+     *
+     * @param uri namespace to store the log
+     * @param logName name of the log
+     * @return top stream path
+     */
+    public static String getLogStreamPath(URI uri, String logName) {
+        return String.format("%s/%s", uri.getPath(), logName);
+    }
+
+    /**
+     * Get the log root path for a given log.
+     *
+     * @param uri
+     *          namespace to store the log
+     * @param logName
+     *          name of the log
+     * @param logIdentifier
+     *          identifier of the log
+     * @return log root path
+     */
+    public static String getLogRootPath(URI uri, String logName, String logIdentifier) {
+        return getLogComponentPath(uri, logName, logIdentifier, "");
+    }
+
+    /**
+     * Get the logsegments root path for a given log.
+     *
+     * @param uri
+     *          namespace to store the log
+     * @param logName
+     *          name of the log
+     * @param logIdentifier
+     *          identifier of the log
+     * @return logsegments root path
+     */
+    public static String getLogSegmentsPath(URI uri, String logName, String logIdentifier) {
+        return getLogComponentPath(uri, logName, logIdentifier, LOGSEGMENTS_PATH);
+    }
+
+    public static final int LAYOUT_VERSION = -1;
+    public final static String LOGSEGMENTS_PATH = "/ledgers";
+    public final static String VERSION_PATH = "/version";
+    // writer znodes
+    public final static String MAX_TXID_PATH = "/maxtxid";
+    public final static String LOCK_PATH = "/lock";
+    public final static String ALLOCATION_PATH = "/allocation";
+    // reader znodes
+    public final static String READ_LOCK_PATH = "/readLock";
+
+    protected final URI uri;
+    protected final String logName;
+    protected final String logIdentifier;
+
+    // Root path of the log
+    protected final String logRootPath;
+    // Components
+    protected final String logSegmentsPath;
+    protected final String lockPath;
+    protected final String maxTxIdPath;
+    protected final String allocationPath;
+
+    /**
+     * metadata representation of a log
+     *
+     * @param uri
+     *          namespace to store the log
+     * @param logName
+     *          name of the log
+     * @param logIdentifier
+     *          identifier of the log
+     */
+    protected LogMetadata(URI uri,
+                          String logName,
+                          String logIdentifier) {
+        this.uri = uri;
+        this.logName = logName;
+        this.logIdentifier = logIdentifier;
+        this.logRootPath = getLogRootPath(uri, logName, logIdentifier);
+        this.logSegmentsPath = logRootPath + LOGSEGMENTS_PATH;
+        this.lockPath = logRootPath + LOCK_PATH;
+        this.maxTxIdPath = logRootPath + MAX_TXID_PATH;
+        this.allocationPath = logRootPath + ALLOCATION_PATH;
+    }
+
+    public URI getUri() {
+        return uri;
+    }
+
+    public String getLogName() {
+        return logName;
+    }
+
+    /**
+     * Get the root path of the log.
+     *
+     * @return root path of the log.
+     */
+    public String getLogRootPath() {
+        return logRootPath;
+    }
+
+    /**
+     * Get the root path for log segments.
+     *
+     * @return root path for log segments
+     */
+    public String getLogSegmentsPath() {
+        return this.logSegmentsPath;
+    }
+
+    /**
+     * Get the path for a log segment of the log.
+     *
+     * @param segmentName
+     *          segment name
+     * @return path for the log segment
+     */
+    public String getLogSegmentPath(String segmentName) {
+        return this.logSegmentsPath + "/" + segmentName;
+    }
+
+    public String getLockPath() {
+        return lockPath;
+    }
+
+    public String getMaxTxIdPath() {
+        return maxTxIdPath;
+    }
+
+    public String getAllocationPath() {
+        return allocationPath;
+    }
+
+    /**
+     * Get the fully qualified name of the log.
+     *
+     * @return fully qualified name
+     */
+    public String getFullyQualifiedName() {
+        return String.format("%s:%s", logName, logIdentifier);
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java
new file mode 100644
index 0000000..1adebfe
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForReader.java
@@ -0,0 +1,103 @@
+/**
+ * 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.metadata;
+
+import com.google.common.base.Optional;
+
+import java.net.URI;
+
+/**
+ * Log Metadata for Reader
+ */
+public class LogMetadataForReader extends LogMetadata {
+
+    /**
+     * Get the root path to store subscription infos of a log.
+     *
+     * @param uri
+     *          namespace of the log
+     * @param logName
+     *          name of the log
+     * @param logIdentifier
+     *          identifier of the log
+     * @return subscribers root path
+     */
+    public static String getSubscribersPath(URI uri, String logName, String logIdentifier) {
+        return getLogComponentPath(uri, logName, logIdentifier, SUBSCRIBERS_PATH);
+    }
+
+    /**
+     * Get the path that stores subscription info for a <code>subscriberId</code> for a <code>log</code>.
+     *
+     * @param uri
+     *          namespace of the log
+     * @param logName
+     *          name of the log
+     * @param logIdentifier
+     *          identifier of the log
+     * @param subscriberId
+     *          subscriber id of the log
+     * @return subscriber's path
+     */
+    public static String getSubscriberPath(URI uri, String logName, String logIdentifier, String subscriberId) {
+        return String.format("%s/%s", getSubscribersPath(uri, logName, logIdentifier), subscriberId);
+    }
+
+    /**
+     * Create a metadata representation of a log for reader.
+     *
+     * @param uri
+     *          namespace to store the log
+     * @param logName
+     *          name of the log
+     * @param logIdentifier
+     *          identifier of the log
+     * @return metadata representation of a log for reader
+     */
+    public static LogMetadataForReader of(URI uri, String logName, String logIdentifier) {
+        return new LogMetadataForReader(uri, logName, logIdentifier);
+    }
+
+    final static String SUBSCRIBERS_PATH = "/subscribers";
+
+    /**
+     * metadata representation of a log
+     *
+     * @param uri           namespace to store the log
+     * @param logName       name of the log
+     * @param logIdentifier identifier of the log
+     */
+    private LogMetadataForReader(URI uri, String logName, String logIdentifier) {
+        super(uri, logName, logIdentifier);
+    }
+
+    /**
+     * Get the readlock path for the log or a subscriber of the log.
+     *
+     * @param subscriberId
+     *          subscriber id. it is optional.
+     * @return read lock path
+     */
+    public String getReadLockPath(Optional<String> subscriberId) {
+        if (subscriberId.isPresent()) {
+            return logRootPath + SUBSCRIBERS_PATH + "/" + subscriberId.get() + READ_LOCK_PATH;
+        } else {
+            return logRootPath + READ_LOCK_PATH;
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForWriter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForWriter.java
new file mode 100644
index 0000000..35e645e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataForWriter.java
@@ -0,0 +1,64 @@
+/**
+ * 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.metadata;
+
+import org.apache.bookkeeper.versioning.Versioned;
+
+import java.net.URI;
+
+/**
+ * Log Metadata for writer
+ */
+public class LogMetadataForWriter extends LogMetadata {
+
+    private final Versioned<byte[]> maxLSSNData;
+    private final Versioned<byte[]> maxTxIdData;
+    private final Versioned<byte[]> allocationData;
+
+    /**
+     * metadata representation of a log
+     *
+     * @param uri           namespace to store the log
+     * @param logName       name of the log
+     * @param logIdentifier identifier of the log
+     */
+    public LogMetadataForWriter(URI uri,
+                                String logName,
+                                String logIdentifier,
+                                Versioned<byte[]> maxLSSNData,
+                                Versioned<byte[]> maxTxIdData,
+                                Versioned<byte[]> allocationData) {
+        super(uri, logName, logIdentifier);
+        this.maxLSSNData = maxLSSNData;
+        this.maxTxIdData = maxTxIdData;
+        this.allocationData = allocationData;
+    }
+
+    public Versioned<byte[]> getMaxLSSNData() {
+        return maxLSSNData;
+    }
+
+    public Versioned<byte[]> getMaxTxIdData() {
+        return maxTxIdData;
+    }
+
+    public Versioned<byte[]> getAllocationData() {
+        return allocationData;
+    }
+
+}
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
new file mode 100644
index 0000000..f8fd777
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogMetadataStore.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.metadata;
+
+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;
+
+/**
+ * Interface for log metadata store.
+ */
+@Beta
+public interface LogMetadataStore {
+
+    /**
+     * Create a stream and return it is namespace location.
+     *
+     * @param logName
+     *          name of the log
+     * @return namespace location that stores this stream.
+     */
+    Future<URI> createLog(String logName);
+
+    /**
+     * Get the location of the log.
+     *
+     * @param logName
+     *          name of the log
+     * @return namespace location that stores this stream.
+     */
+    Future<Optional<URI>> getLogLocation(String logName);
+
+    /**
+     * Retrieves logs from the namespace.
+     *
+     * @return iterator of logs of the namespace.
+     */
+    Future<Iterator<String>> getLogs();
+
+    /**
+     * Register a namespace listener on streams changes.
+     *
+     * @param listener
+     *          namespace listener
+     */
+    void registerNamespaceListener(NamespaceListener listener);
+}
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
new file mode 100644
index 0000000..ee9b00e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
@@ -0,0 +1,184 @@
+/**
+ * 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.metadata;
+
+import com.google.common.base.Preconditions;
+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 {
+
+    static final Logger LOG = LoggerFactory.getLogger(LogSegmentMetadataStoreUpdater.class);
+
+    public static MetadataUpdater createMetadataUpdater(DistributedLogConfiguration conf,
+                                                        LogSegmentMetadataStore metadataStore) {
+        return new LogSegmentMetadataStoreUpdater(conf, metadataStore);
+    }
+
+    protected final LogSegmentMetadataStore metadataStore;
+    protected final LogSegmentMetadata.LogSegmentMetadataVersion metadataVersion;
+
+    protected LogSegmentMetadataStoreUpdater(DistributedLogConfiguration conf,
+                                             LogSegmentMetadataStore metadataStore) {
+        this.metadataStore = metadataStore;
+        this.metadataVersion = LogSegmentMetadata.LogSegmentMetadataVersion.of(conf.getDLLedgerMetadataLayoutVersion());
+    }
+
+    private String formatLogSegmentSequenceNumber(long logSegmentSeqNo) {
+        return String.format("%018d", logSegmentSeqNo);
+    }
+
+    @Override
+    public Transaction<Object> transaction() {
+        return metadataStore.transaction();
+    }
+
+    @Override
+    public Future<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.");
+        Preconditions.checkArgument(segment.isDLSNinThisSegment(dlsn),
+                "DLSN " + dlsn + " doesn't belong to segment " + segment);
+        final LogSegmentMetadata newSegment = segment.mutator()
+                .setLastDLSN(dlsn)
+                .setLastTxId(record.getTransactionId())
+                .setRecordCount(record)
+                .build();
+        return updateSegmentMetadata(newSegment);
+    }
+
+    @Override
+    public Future<LogSegmentMetadata> changeSequenceNumber(LogSegmentMetadata segment,
+                                                           long logSegmentSeqNo) {
+        String newZkPath = segment.getZkPath()
+                .replace(formatLogSegmentSequenceNumber(segment.getLogSegmentSequenceNumber()),
+                        formatLogSegmentSequenceNumber(logSegmentSeqNo));
+        final LogSegmentMetadata newSegment = segment.mutator()
+                .setLogSegmentSequenceNumber(logSegmentSeqNo)
+                .setZkPath(newZkPath)
+                .build();
+        return addNewSegmentAndDeleteOldSegment(newSegment, segment);
+    }
+
+    /**
+     * Change the truncation status of a <i>log segment</i> to be active
+     *
+     * @param segment log segment to change truncation status to active.
+     * @return new log segment
+     */
+    @Override
+    public Future<LogSegmentMetadata> setLogSegmentActive(LogSegmentMetadata segment) {
+        final LogSegmentMetadata newSegment = segment.mutator()
+            .setTruncationStatus(LogSegmentMetadata.TruncationStatus.ACTIVE)
+            .build();
+        return addNewSegmentAndDeleteOldSegment(newSegment, segment);
+    }
+
+    /**
+     * Change the truncation status of a <i>log segment</i> to truncated
+     *
+     * @param segment log segment to change truncation status to truncated.
+     * @return new log segment
+     */
+    @Override
+    public Future<LogSegmentMetadata> setLogSegmentTruncated(LogSegmentMetadata segment) {
+        final LogSegmentMetadata newSegment = segment.mutator()
+            .setTruncationStatus(LogSegmentMetadata.TruncationStatus.TRUNCATED)
+            .build();
+        return addNewSegmentAndDeleteOldSegment(newSegment, segment);
+    }
+
+    @Override
+    public LogSegmentMetadata setLogSegmentTruncated(Transaction<Object> txn, LogSegmentMetadata segment) {
+        final LogSegmentMetadata newSegment = segment.mutator()
+            .setTruncationStatus(LogSegmentMetadata.TruncationStatus.TRUNCATED)
+            .build();
+        addNewSegmentAndDeleteOldSegment(txn, newSegment, segment);
+        return newSegment;
+    }
+
+    /**
+     * Change the truncation status of a <i>log segment</i> to partially truncated
+     *
+     * @param segment log segment to change sequence number.
+     * @param minActiveDLSN DLSN within the log segment before which log has been truncated
+     * @return new log segment
+     */
+    @Override
+    public Future<LogSegmentMetadata> setLogSegmentPartiallyTruncated(LogSegmentMetadata segment, DLSN minActiveDLSN) {
+        final LogSegmentMetadata newSegment = segment.mutator()
+            .setTruncationStatus(LogSegmentMetadata.TruncationStatus.PARTIALLY_TRUNCATED)
+            .setMinActiveDLSN(minActiveDLSN)
+            .build();
+        return addNewSegmentAndDeleteOldSegment(newSegment, segment);
+    }
+
+    @Override
+    public LogSegmentMetadata setLogSegmentPartiallyTruncated(Transaction<Object> txn,
+                                                              LogSegmentMetadata segment,
+                                                              DLSN minActiveDLSN) {
+        final LogSegmentMetadata newSegment = segment.mutator()
+                .setTruncationStatus(LogSegmentMetadata.TruncationStatus.PARTIALLY_TRUNCATED)
+                .setMinActiveDLSN(minActiveDLSN)
+                .build();
+        addNewSegmentAndDeleteOldSegment(txn, newSegment, segment);
+        return newSegment;
+    }
+
+    protected Future<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;
+            }
+        });
+    }
+
+    protected Future<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;
+            }
+        });
+    }
+
+    protected void addNewSegmentAndDeleteOldSegment(Transaction<Object> txn,
+                                                    LogSegmentMetadata newSegment,
+                                                    LogSegmentMetadata oldSegment) {
+        metadataStore.deleteLogSegment(txn, oldSegment, null);
+        metadataStore.createLogSegment(txn, newSegment, null);
+    }
+
+}
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
new file mode 100644
index 0000000..f1e8f06
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/LogStreamMetadataStore.java
@@ -0,0 +1,114 @@
+/**
+ * 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.metadata;
+
+import com.google.common.annotations.Beta;
+import com.google.common.base.Optional;
+import org.apache.distributedlog.lock.DistributedLock;
+import org.apache.distributedlog.logsegment.LogSegmentMetadataStore;
+import org.apache.distributedlog.util.PermitManager;
+import org.apache.distributedlog.util.Transaction;
+import com.twitter.util.Future;
+
+import java.io.Closeable;
+import java.net.URI;
+
+/**
+ * The interface to manage the log stream metadata. The implementation is responsible
+ * for creating the metadata layout.
+ */
+@Beta
+public interface LogStreamMetadataStore extends Closeable {
+
+    /**
+     * Create a transaction for the metadata operations happening in the metadata store.
+     *
+     * @return transaction for the metadata operations
+     */
+    Transaction<Object> newTransaction();
+
+    /**
+     * Ensure the existence of a log stream
+     *
+     * @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
+     */
+    Future<Void> logExists(URI uri, String logName);
+
+    /**
+     * Create the read lock for the log stream.
+     *
+     * @param metadata the metadata for a log stream
+     * @param readerId the reader id used for lock
+     * @return the read lock
+     */
+    Future<DistributedLock> createReadLock(LogMetadataForReader metadata,
+                                           Optional<String> readerId);
+
+    /**
+     * Create the write lock for the log stream.
+     *
+     * @param metadata the metadata for a log stream
+     * @return the write lock
+     */
+    DistributedLock createWriteLock(LogMetadataForWriter metadata);
+
+    /**
+     * Create the metadata of a log.
+     *
+     * @param uri the location to store the metadata of the log
+     * @param streamName the name of the log stream
+     * @param ownAllocator whether to use its own allocator or external allocator
+     * @param createIfNotExists flag to create the stream if it doesn't exist
+     * @return the metadata of the log
+     */
+    Future<LogMetadataForWriter> getLog(URI uri,
+                                        String streamName,
+                                        boolean ownAllocator,
+                                        boolean createIfNotExists);
+
+    /**
+     * Delete the metadata of a log.
+     *
+     * @param uri the location to store the metadata of the log
+     * @param streamName the name of the log stream
+     * @return future represents the result of the deletion.
+     */
+    Future<Void> deleteLog(URI uri, String streamName);
+
+    /**
+     * Get the log segment metadata store.
+     *
+     * @return the log segment metadata store.
+     */
+    LogSegmentMetadataStore getLogSegmentMetadataStore();
+
+    /**
+     * Get the permit manager for this metadata store. It can be used for limiting the concurrent
+     * metadata operations. The implementation can disable handing over the permits when the metadata
+     * store is unavailable (for example zookeeper session expired).
+     *
+     * @return the permit manager
+     */
+    PermitManager getPermitManager();
+
+
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataResolver.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataResolver.java
new file mode 100644
index 0000000..ae0002e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataResolver.java
@@ -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.
+ */
+package org.apache.distributedlog.metadata;
+
+import java.io.IOException;
+import java.net.URI;
+
+/**
+ * Resolver to resolve the metadata used to instantiate a DL instance.
+ *
+ * <p>
+ * E.g. we stored a common dl config under /messaging/distributedlog to use
+ * bookkeeper cluster x. so all the distributedlog instances under this path
+ * inherit this dl config. if a dl D is allocated under /messaging/distributedlog,
+ * but use a different cluster y, so its metadata is stored /messaging/distributedlog/D.
+ * The resolver resolve the URI
+ * </p>
+ *
+ * <p>
+ * The resolver looks up the uri path and tries to interpret the path segments from
+ * bottom-to-top to see if there is a DL metadata bound. It stops when it found valid
+ * dl metadata.
+ * </p>
+ */
+public interface MetadataResolver {
+
+    /**
+     * Resolve the path to get the DL metadata.
+     *
+     * @param uri
+     *          dl uri
+     * @return dl metadata.
+     * @throws IOException
+     */
+    public DLMetadata resolve(URI uri) throws IOException;
+}
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
new file mode 100644
index 0000000..06a0600
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/MetadataUpdater.java
@@ -0,0 +1,120 @@
+/**
+ * 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.metadata;
+
+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.
+ */
+public interface MetadataUpdater {
+
+    /**
+     * Start a transaction on metadata updates
+     *
+     * @return transaction
+     */
+    Transaction<Object> transaction();
+
+    /**
+     * Update the log segment metadata with correct last <i>record</i>.
+     *
+     * @param segment
+     *          log segment to update last dlsn.
+     * @param record
+     *          correct last record.
+     * @return new log segment
+     */
+    Future<LogSegmentMetadata> updateLastRecord(LogSegmentMetadata segment,
+                                                LogRecordWithDLSN record);
+
+    /**
+     * Change ledger sequence number of <i>segment</i> to given <i>logSegmentSeqNo</i>.
+     *
+     * @param segment
+     *          log segment to change sequence number.
+     * @param logSegmentSeqNo
+     *          ledger sequence number to change.
+     * @return new log segment
+     */
+    Future<LogSegmentMetadata> changeSequenceNumber(LogSegmentMetadata segment,
+                                                    long logSegmentSeqNo);
+
+    /**
+     * Change the truncation status of a <i>log segment</i> to be active
+     *
+     * @param segment
+     *          log segment to change truncation status to active.
+     * @return new log segment
+     */
+    Future<LogSegmentMetadata> setLogSegmentActive(LogSegmentMetadata segment);
+
+    /**
+     * Change the truncation status of a <i>log segment</i> to truncated
+     *
+     * @param segment
+     *          log segment to change truncation status to truncated.
+     * @return new log segment
+     */
+    Future<LogSegmentMetadata> setLogSegmentTruncated(LogSegmentMetadata segment);
+
+    /**
+     * Change the truncation status of a <i>log segment</i> to truncated. The operation won't be executed
+     * immediately. The update only happens after {@link Transaction#execute()}.
+     *
+     * @param txn
+     *          transaction used to set the log segment status
+     * @param segment
+     *          segment to set truncation status to truncated
+     * @return log segment that truncation status is set to truncated.
+     */
+    LogSegmentMetadata setLogSegmentTruncated(Transaction<Object> txn, LogSegmentMetadata segment);
+
+    /**
+     * Change the truncation status of a <i>log segment</i> to partially truncated
+     *
+     * @param segment
+     *          log segment to change sequence number.
+     * @param minActiveDLSN
+     *          DLSN within the log segment before which log has been truncated
+     * @return new log segment
+     */
+    Future<LogSegmentMetadata> setLogSegmentPartiallyTruncated(LogSegmentMetadata segment,
+                                                               DLSN minActiveDLSN);
+
+    /**
+     * Change the truncation status of a <i>log segment</i> to partially truncated. The operation won't be
+     * executed until {@link Transaction#execute()}.
+     *
+     * @param txn
+     *          transaction used to set the log segment status
+     * @param segment
+     *          segment to set truncation status to partially truncated
+     * @param minActiveDLSN
+     *          DLSN within the log segment before which log has been truncated
+     * @return log segment that truncation status has been set to partially truncated
+     */
+    LogSegmentMetadata setLogSegmentPartiallyTruncated(Transaction<Object> txn,
+                                                       LogSegmentMetadata segment,
+                                                       DLSN minActiveDLSN);
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/package-info.java
new file mode 100644
index 0000000..a076cd4
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/metadata/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Metadata management for distributedlog.
+ */
+package org.apache.distributedlog.metadata;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespace.java b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespace.java
new file mode 100644
index 0000000..4cbee98
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespace.java
@@ -0,0 +1,190 @@
+/**
+ * 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.Beta;
+import com.google.common.base.Optional;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.callback.NamespaceListener;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.InvalidStreamNameException;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * A namespace is the basic unit for managing a set of distributedlogs.
+ *
+ * <h4>Namespace Interface</h4>
+ *
+ * <P>
+ * The <code>DistributedLogNamespace</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,
+ *         watch the changes of logs.
+ *     <LI>Access Control -- manage the access controls for logs in the namespace.
+ * </OL>
+ * </P>
+ *
+ * <h4>Namespace Location</h4>
+ *
+ * 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.
+ *
+ * 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.
+ *
+ * @see DistributedLogManager
+ * @since 0.3.32
+ */
+@Beta
+public interface DistributedLogNamespace {
+
+    /**
+     * Get the namespace driver used by this namespace.
+     *
+     * @return namespace driver
+     */
+    NamespaceDriver getNamespaceDriver();
+
+    //
+    // Method to operate logs
+    //
+
+    /**
+     * Create a log named <i>logName</i>.
+     *
+     * @param logName
+     *          name of the log
+     * @throws InvalidStreamNameException if log name is invalid.
+     * @throws IOException when encountered issues with backend.
+     */
+    void createLog(String logName)
+            throws InvalidStreamNameException, IOException;
+
+    /**
+     * Delete a log named <i>logName</i>.
+     *
+     * @param logName
+     *          name of the log
+     * @throws InvalidStreamNameException if log name is invalid
+     * @throws LogNotFoundException if log doesn't exist
+     * @throws IOException when encountered issues with backend
+     */
+    void deleteLog(String logName)
+            throws InvalidStreamNameException, LogNotFoundException, IOException;
+
+    /**
+     * Open a log named <i>logName</i>.
+     * A distributedlog manager is returned to access log <i>logName</i>.
+     *
+     * @param logName
+     *          name of the log
+     * @return distributedlog manager instance.
+     * @throws InvalidStreamNameException if log name is invalid.
+     * @throws IOException when encountered issues with backend.
+     */
+    DistributedLogManager openLog(String logName)
+            throws InvalidStreamNameException, IOException;
+
+    /**
+     * Open a log named <i>logName</i> with specific log configurations.
+     *
+     * <p>This method allows the caller to override global configuration settings by
+     * supplying log configuration overrides. Log config overrides come in two flavors,
+     * static and dynamic. Static config never changes in the lifecyle of <code>DistributedLogManager</code>,
+     * dynamic config changes by reloading periodically and safe to access from any context.</p>
+     *
+     * @param logName
+     *          name of the log
+     * @param logConf
+     *          static log configuration
+     * @param dynamicLogConf
+     *          dynamic log configuration
+     * @return distributedlog manager instance.
+     * @throws InvalidStreamNameException if log name is invalid.
+     * @throws IOException when encountered issues with backend.
+     */
+    DistributedLogManager openLog(String logName,
+                                  Optional<DistributedLogConfiguration> logConf,
+                                  Optional<DynamicDistributedLogConfiguration> dynamicLogConf,
+                                  Optional<StatsLogger> perStreamStatsLogger)
+            throws InvalidStreamNameException, IOException;
+
+    /**
+     * Check whether the log <i>logName</i> exist.
+     *
+     * @param logName
+     *          name of the log
+     * @return <code>true</code> if the log exists, otherwise <code>false</code>.
+     * @throws IOException when encountered exceptions on checking
+     */
+    boolean logExists(String logName)
+            throws IOException;
+
+    /**
+     * Retrieve the logs under the namespace.
+     *
+     * @return iterator of the logs under the namespace.
+     * @throws IOException when encountered issues with backend.
+     */
+    Iterator<String> getLogs()
+            throws IOException;
+
+    //
+    // Methods for namespace
+    //
+
+    /**
+     * Register namespace listener on stream updates under the namespace.
+     *
+     * @param listener
+     *          listener to receive stream updates under the namespace
+     */
+    void registerNamespaceListener(NamespaceListener listener);
+
+    /**
+     * Create an access control manager to manage/check acl for logs.
+     *
+     * @return access control manager for logs under the namespace.
+     * @throws IOException
+     */
+    AccessControlManager createAccessControlManager()
+            throws IOException;
+
+    /**
+     * Close the namespace.
+     */
+    void close();
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespaceBuilder.java b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespaceBuilder.java
new file mode 100644
index 0000000..2706201
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/DistributedLogNamespaceBuilder.java
@@ -0,0 +1,278 @@
+/**
+ * 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.Preconditions;
+import org.apache.distributedlog.BKDistributedLogNamespace;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.feature.CoreFeatureKeys;
+import org.apache.distributedlog.injector.AsyncFailureInjector;
+import org.apache.distributedlog.injector.AsyncRandomFailureInjector;
+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.util.SimplePermitLimiter;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.feature.SettableFeatureProvider;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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 {
+
+    private static final Logger logger = LoggerFactory.getLogger(DistributedLogNamespaceBuilder.class);
+
+    public static DistributedLogNamespaceBuilder newBuilder() {
+        return new DistributedLogNamespaceBuilder();
+    }
+
+    private DistributedLogConfiguration _conf = null;
+    private DynamicDistributedLogConfiguration _dynConf = null;
+    private URI _uri = null;
+    private StatsLogger _statsLogger = NullStatsLogger.INSTANCE;
+    private StatsLogger _perLogStatsLogger = NullStatsLogger.INSTANCE;
+    private FeatureProvider _featureProvider = null;
+    private String _clientId = DistributedLogConstants.UNKNOWN_CLIENT_ID;
+    private int _regionId = DistributedLogConstants.LOCAL_REGION_ID;
+
+    // private constructor
+    private DistributedLogNamespaceBuilder() {}
+
+    /**
+     * DistributedLog Configuration used for the namespace.
+     *
+     * @param conf
+     *          distributedlog configuration
+     * @return namespace builder.
+     */
+    public DistributedLogNamespaceBuilder conf(DistributedLogConfiguration conf) {
+        this._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._dynConf = dynConf;
+        return this;
+    }
+
+    /**
+     * Namespace Location.
+     *
+     * @param uri
+     *          namespace location uri.
+     * @see DistributedLogNamespace
+     * @return namespace builder.
+     */
+    public DistributedLogNamespaceBuilder uri(URI uri) {
+        this._uri = uri;
+        return this;
+    }
+
+    /**
+     * Stats Logger used for stats collection
+     *
+     * @param statsLogger
+     *          stats logger
+     * @return namespace builder.
+     */
+    public DistributedLogNamespaceBuilder statsLogger(StatsLogger statsLogger) {
+        this._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._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._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._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._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 {
+        // Check arguments
+        Preconditions.checkNotNull(_conf, "No DistributedLog Configuration.");
+        Preconditions.checkNotNull(_uri, "No DistributedLog URI");
+
+        // validate the configuration
+        _conf.validate();
+        if (null == _dynConf) {
+            _dynConf = ConfUtils.getConstDynConf(_conf);
+        }
+
+        // retrieve the namespace driver
+        NamespaceDriver driver = NamespaceDriverManager.getDriver(_uri);
+        URI normalizedUri = DLUtils.normalizeURI(_uri);
+
+        // build the feature provider
+        FeatureProvider featureProvider;
+        if (null == _featureProvider) {
+            featureProvider = new SettableFeatureProvider("", 0);
+            logger.info("No feature provider is set. All features are disabled now.");
+        } else {
+            featureProvider = _featureProvider;
+        }
+
+        // build the failure injector
+        AsyncFailureInjector failureInjector = AsyncRandomFailureInjector.newBuilder()
+                .injectDelays(_conf.getEIInjectReadAheadDelay(),
+                              _conf.getEIInjectReadAheadDelayPercent(),
+                              _conf.getEIInjectMaxReadAheadDelayMs())
+                .injectErrors(false, 10)
+                .injectStops(_conf.getEIInjectReadAheadStall(), 10)
+                .injectCorruption(_conf.getEIInjectReadAheadBrokenEntries())
+                .build();
+
+        // normalize the per log stats logger
+        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
+        driver.initialize(
+                _conf,
+                _dynConf,
+                normalizedUri,
+                scheduler,
+                featureProvider,
+                failureInjector,
+                _statsLogger,
+                perLogStatsLogger,
+                DLUtils.normalizeClientId(_clientId),
+                _regionId);
+
+        // initialize the write limiter
+        PermitLimiter writeLimiter;
+        if (_conf.getGlobalOutstandingWriteLimit() < 0) {
+            writeLimiter = PermitLimiter.NULL_PERMIT_LIMITER;
+        } else {
+            Feature disableWriteLimitFeature = featureProvider.getFeature(
+                CoreFeatureKeys.DISABLE_WRITE_LIMIT.name().toLowerCase());
+            writeLimiter = new SimplePermitLimiter(
+                _conf.getOutstandingWriteLimitDarkmode(),
+                _conf.getGlobalOutstandingWriteLimit(),
+                _statsLogger.scope("writeLimiter"),
+                true /* singleton */,
+                disableWriteLimitFeature);
+        }
+
+        return new BKDistributedLogNamespace(
+                _conf,
+                normalizedUri,
+                driver,
+                scheduler,
+                featureProvider,
+                writeLimiter,
+                failureInjector,
+                _statsLogger,
+                perLogStatsLogger,
+                DLUtils.normalizeClientId(_clientId),
+                _regionId);
+    }
+}
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
new file mode 100644
index 0000000..2f5adc6
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriver.java
@@ -0,0 +1,138 @@
+/**
+ * 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 org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.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.util.OrderedScheduler;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+import java.io.Closeable;
+import java.io.IOException;
+import java.net.URI;
+
+/**
+ * Manager to manage all the stores required by a namespace.
+ */
+public interface NamespaceDriver extends Closeable {
+
+    enum Role {
+        WRITER,
+        READER
+    }
+
+    /**
+     * Initialize the namespace manager.
+     *
+     * @param conf distributedlog configuration
+     * @param dynConf dynamic distributedlog configuration
+     * @param namespace root uri of the namespace
+     * @param scheduler ordered scheduler
+     * @param featureProvider feature provider
+     * @param statsLogger stats logger
+     * @param perLogStatsLogger per log stream stats logger
+     * @param clientId client id
+     * @return namespace manager
+     * @throws IOException when failed to initialize the namespace manager
+     */
+    NamespaceDriver initialize(DistributedLogConfiguration conf,
+                               DynamicDistributedLogConfiguration dynConf,
+                               URI namespace,
+                               OrderedScheduler scheduler,
+                               FeatureProvider featureProvider,
+                               AsyncFailureInjector failureInjector,
+                               StatsLogger statsLogger,
+                               StatsLogger perLogStatsLogger,
+                               String clientId,
+                               int regionId) throws IOException;
+
+    /**
+     * Get the scheme of the namespace driver.
+     *
+     * @return the scheme of the namespace driver.
+     */
+    String getScheme();
+
+    /**
+     * Get the root uri of the namespace driver.
+     *
+     * @return the root uri of the namespace driver.
+     */
+    URI getUri();
+
+    /**
+     * Retrieve the log {@code metadata store} used by the namespace.
+     *
+     * @return the log metadata store
+     */
+    LogMetadataStore getLogMetadataStore();
+
+    /**
+     * Retrieve the log stream {@code metadata store} used by the namespace.
+     *
+     * @param role the role to retrieve the log stream metadata store.
+     * @return the log stream metadata store
+     */
+    LogStreamMetadataStore getLogStreamMetadataStore(Role role);
+
+    /**
+     * Retrieve the log segment {@code entry store} used by the namespace.
+     *
+     * @param role the role to retrieve the log segment entry store.
+     * @return the log segment entry store.
+     * @throws IOException when failed to open log segment entry store.
+     */
+    LogSegmentEntryStore getLogSegmentEntryStore(Role role);
+
+    /**
+     * Create an access control manager to manage/check acl for logs.
+     *
+     * @return access control manager for logs under the namespace.
+     * @throws IOException
+     */
+    AccessControlManager getAccessControlManager()
+            throws IOException;
+
+    /**
+     * Retrieve the metadata accessor for log stream {@code streamName}.
+     * (TODO: it is a legacy interface. should remove it if we have metadata of stream.)
+     *
+     * @param streamName name of log stream.
+     * @return metadata accessor for log stream {@code streamName}.
+     */
+    MetadataAccessor getMetadataAccessor(String streamName)
+            throws InvalidStreamNameException, IOException;
+
+    /**
+     * Retrieve the subscriptions store for log stream {@code streamName}.
+     *
+     * @return the subscriptions store for log stream {@code streamName}
+     */
+    SubscriptionsStore getSubscriptionsStore(String streamName);
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java
new file mode 100644
index 0000000..b9e3a31
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceDriverManager.java
@@ -0,0 +1,180 @@
+/**
+ * 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.Objects;
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import static com.google.common.base.Preconditions.*;
+
+/**
+ * The basic service for managing a set of namespace drivers.
+ */
+public class NamespaceDriverManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(NamespaceDriverManager.class);
+
+    static class NamespaceDriverInfo {
+
+        final Class<? extends NamespaceDriver> driverClass;
+        final String driverClassName;
+
+        NamespaceDriverInfo(Class<? extends NamespaceDriver> driverClass) {
+            this.driverClass = driverClass;
+            this.driverClassName = this.driverClass.getName();
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder sb = new StringBuilder();
+            sb.append("driver[")
+                    .append(driverClassName)
+                    .append("]");
+            return sb.toString();
+        }
+    }
+
+    private static final ConcurrentMap<String, NamespaceDriverInfo> drivers;
+    private static boolean initialized = false;
+
+    static {
+        drivers = new ConcurrentHashMap<String, NamespaceDriverInfo>();
+        initialize();
+    }
+
+    static void initialize() {
+        if (initialized) {
+            return;
+        }
+        loadInitialDrivers();
+        initialized = true;
+        logger.info("DistributedLog NamespaceDriverManager initialized");
+    }
+
+    private static void loadInitialDrivers() {
+        Set<String> driverList = Sets.newHashSet();
+        // add default bookkeeper based driver
+        driverList.add(BKNamespaceDriver.class.getName());
+        // load drivers from system property
+        String driversStr = System.getProperty("distributedlog.namespace.drivers");
+        if (null != driversStr) {
+            String[] driversArray = StringUtils.split(driversStr, ':');
+            for (String driver : driversArray) {
+                driverList.add(driver);
+            }
+        }
+        // initialize the drivers
+        for (String driverClsName : driverList) {
+            try {
+                NamespaceDriver driver =
+                        ReflectionUtils.newInstance(driverClsName, NamespaceDriver.class);
+                NamespaceDriverInfo driverInfo = new NamespaceDriverInfo(driver.getClass());
+                drivers.put(driver.getScheme().toLowerCase(), driverInfo);
+            } catch (Exception ex) {
+                logger.warn("Failed to load namespace driver {} : ", driverClsName, ex);
+            }
+        }
+    }
+
+    /**
+     * Prevent the NamespaceDriverManager class from being instantiated.
+     */
+    private NamespaceDriverManager() {}
+
+    /**
+     * Register the namespace {@code driver}.
+     *
+     * @param driver the namespace driver
+     * @return the namespace driver manager
+     */
+    public static void registerDriver(String backend, Class<? extends NamespaceDriver> driver) {
+        if (!initialized) {
+            initialize();
+        }
+
+        String scheme = backend.toLowerCase();
+        NamespaceDriverInfo oldDriverInfo = drivers.get(scheme);
+        if (null != oldDriverInfo) {
+            return;
+        }
+        NamespaceDriverInfo newDriverInfo = new NamespaceDriverInfo(driver);
+        oldDriverInfo = drivers.putIfAbsent(scheme, newDriverInfo);
+        if (null != oldDriverInfo) {
+            logger.debug("Driver for {} is already there.", scheme);
+        }
+    }
+
+    /**
+     * Retrieve the namespace driver for {@code scheme}.
+     *
+     * @param scheme the scheme for the namespace driver
+     * @return the namespace driver
+     * @throws NullPointerException when scheme is null
+     */
+    public static NamespaceDriver getDriver(String scheme) {
+        checkNotNull(scheme, "Driver Scheme is null");
+        if (!initialized) {
+            initialize();
+        }
+        NamespaceDriverInfo driverInfo = drivers.get(scheme.toLowerCase());
+        if (null == driverInfo) {
+            throw new IllegalArgumentException("Unknown backend " + scheme);
+        }
+        return ReflectionUtils.newInstance(driverInfo.driverClass);
+    }
+
+    /**
+     * Retrieve the namespace driver for {@code uri}.
+     *
+     * @param uri the distributedlog uri
+     * @return the namespace driver for {@code uri}
+     * @throws NullPointerException if the distributedlog {@code uri} is null or doesn't have scheme
+     *          or there is no namespace driver registered for the scheme
+     * @throws IllegalArgumentException if the distributedlog {@code uri} scheme is illegal
+     */
+    public static NamespaceDriver getDriver(URI uri) {
+        // Validate the uri and load the backend according to scheme
+        checkNotNull(uri, "DistributedLog uri is null");
+        String scheme = uri.getScheme();
+        checkNotNull(scheme, "Invalid distributedlog uri : " + uri);
+        scheme = scheme.toLowerCase();
+        String[] schemeParts = StringUtils.split(scheme, '-');
+        checkArgument(schemeParts.length > 0,
+                "Invalid distributedlog scheme found : " + uri);
+        checkArgument(Objects.equal(DistributedLogConstants.SCHEME_PREFIX, schemeParts[0].toLowerCase()),
+                "Unknown distributedlog scheme found : " + uri);
+        // bookkeeper is the default backend
+        String backend = DistributedLogConstants.BACKEND_BK;
+        if (schemeParts.length > 1) {
+            backend = schemeParts[1];
+        }
+        return getDriver(backend);
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceWatcher.java b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceWatcher.java
new file mode 100644
index 0000000..7c9dda2
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/NamespaceWatcher.java
@@ -0,0 +1,61 @@
+/**
+ * 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 org.apache.distributedlog.callback.NamespaceListener;
+
+import java.util.concurrent.CopyOnWriteArraySet;
+
+/**
+ * Namespace Watcher watching namespace changes.
+ */
+public abstract class NamespaceWatcher {
+
+    protected final CopyOnWriteArraySet<NamespaceListener> listeners =
+            new CopyOnWriteArraySet<NamespaceListener>();
+
+    /**
+     * Register listener for namespace changes.
+     *
+     * @param listener
+     *          listener to add
+     */
+    public void registerListener(NamespaceListener listener) {
+        if (listeners.add(listener)) {
+            watchNamespaceChanges();
+        }
+    }
+
+    /**
+     * Unregister listener from the namespace watcher.
+     *
+     * @param listener
+     *          listener to remove from namespace watcher
+     */
+    public void unregisterListener(NamespaceListener listener) {
+        listeners.remove(listener);
+    }
+
+    /**
+     * Watch the namespace changes. It would be triggered each time
+     * a namspace listener is added. The implementation should handle
+     * this.
+     */
+    protected abstract void watchNamespaceChanges();
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/package-info.java
new file mode 100644
index 0000000..5b86dbe
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/namespace/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * DistributedLog Namespace
+ */
+package org.apache.distributedlog.namespace;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolver.java b/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolver.java
new file mode 100644
index 0000000..b6a0652
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolver.java
@@ -0,0 +1,113 @@
+/**
+ * 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.net;
+
+import org.apache.bookkeeper.net.DNSToSwitchMapping;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * Abstract DNS resolver for bookkeeper ensemble placement.
+ */
+public abstract class DNSResolver implements DNSToSwitchMapping {
+    static final Logger LOG = LoggerFactory.getLogger(DNSResolver.class);
+
+    protected final ConcurrentMap<String, String> domainNameToNetworkLocation =
+            new ConcurrentHashMap<String, String>();
+
+    protected final ConcurrentMap<String, String> hostNameToRegion =
+        new ConcurrentHashMap<String, String>();
+
+    /**
+     * Construct the default dns resolver without host-region overrides.
+     */
+    public DNSResolver() {
+        this("");
+    }
+
+    /**
+     * Construct the dns resolver with host-region overrides.
+     * <p>
+     * <i>hostRegionOverrides</i> is a string of pairs of host-region mapping
+     * (host:region) separated by ';'. during dns resolution, the host will be resolved
+     * to override region. example: <i>host1:region1;host2:region2;...</i>
+     *
+     * @param hostRegionOverrides
+     *          pairs of host-region mapping separated by ';'
+     */
+    public DNSResolver(String hostRegionOverrides) {
+        if (StringUtils.isNotBlank(hostRegionOverrides)) {
+            // Host Region Overrides are of the form
+            // HN1:R1;HN2:R2;...
+            String[] overrides = hostRegionOverrides.split(";");
+
+            for (String override : overrides) {
+                String[] parts = override.split(":");
+                if (parts.length != 2) {
+                    LOG.warn("Incorrect override specified", override);
+                } else {
+                    hostNameToRegion.putIfAbsent(parts[0], parts[1]);
+                }
+            }
+        } // otherwise, no overrides were specified
+    }
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public List<String> resolve(List<String> names) {
+        List<String> networkLocations = new ArrayList<String>(names.size());
+        for (String name : names) {
+            networkLocations.add(resolve(name));
+        }
+        return networkLocations;
+    }
+
+    private String resolve(String domainName) {
+        String networkLocation = domainNameToNetworkLocation.get(domainName);
+        if (null == networkLocation) {
+            networkLocation = resolveToNetworkLocation(domainName);
+            domainNameToNetworkLocation.put(domainName, networkLocation);
+        }
+        return networkLocation;
+    }
+
+    /**
+     * Resolve the <code>domainName</code> to its network location.
+     *
+     * @param domainName
+     *          domain name
+     * @return the network location of <i>domainName</i>
+     */
+    protected abstract String resolveToNetworkLocation(String domainName);
+
+    /**
+     * {@inheritDoc}
+     */
+    @Override
+    public void reloadCachedMappings() {
+        domainNameToNetworkLocation.clear();
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRacks.java b/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRacks.java
new file mode 100644
index 0000000..3408077
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRacks.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.net;
+
+/**
+ * Resolve the dns by racks.
+ * <p>
+ * It resolves domain name like `(region)-(rack)-xxx-xxx.*` to network location
+ * `/(region)/(rack)`. If resolution failed, it returns `/default-region/default-rack`.
+ * <p>
+ * region could be override in <code>hostRegionOverrides</code>. for example, if the
+ * host name is <i>regionA-rack1-xx-yyy</i>, it would be resolved to `/regionA/rack1`
+ * without any overrides. If the specified overrides is <i>regionA-rack1-xx-yyy:regionB</i>,
+ * the resolved network location would be <i>/regionB/rack1</i>.
+ * <p>
+ * Region overrides provide optimization hits to bookkeeper if two `logical` regions are
+ * in same or close locations.
+ *
+ * @see DNSResolver#DNSResolver(String)
+ */
+public class DNSResolverForRacks extends DNSResolver {
+    static final String DEFAULT_RACK = "/default-region/default-rack";
+
+    public DNSResolverForRacks() {
+    }
+
+    public DNSResolverForRacks(String hostRegionOverrides) {
+        super(hostRegionOverrides);
+    }
+
+    @Override
+    protected String resolveToNetworkLocation(String domainName) {
+        String[] parts = domainName.split("\\.");
+        if (parts.length <= 0) {
+            return DEFAULT_RACK;
+        }
+
+        String hostName = parts[0];
+        String[] labels = hostName.split("-");
+        if (labels.length != 4) {
+            return DEFAULT_RACK;
+        }
+
+        String region = hostNameToRegion.get(hostName);
+        if (null == region) {
+            region = labels[0];
+        }
+
+        return String.format("/%s/%s", region, labels[1]);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRows.java b/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRows.java
new file mode 100644
index 0000000..ec604f2
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/net/DNSResolverForRows.java
@@ -0,0 +1,72 @@
+/**
+ * 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.net;
+
+/**
+ * Resolve the dns by rows.
+ * <p>
+ * It resolves domain name like `(region)-(row)xx-xxx-xxx.*` to network location
+ * `/(region)/(row)`. If resolution failed, it returns `/default-region/default-row`.
+ * <p>
+ * region could be override in <code>hostRegionOverrides</code>. for example, if the
+ * host name is <i>regionA-row1-xx-yyy</i>, it would be resolved to `/regionA/row1`
+ * without any overrides. If the specified overrides is <i>regionA-row1-xx-yyy:regionB</i>,
+ * the resolved network location would be <i>/regionB/row1</i>.
+ * <p>
+ * Region overrides provide optimization hits to bookkeeper if two `logical` regions are
+ * in same or close locations.
+ *
+ * @see DNSResolver#DNSResolver(String)
+ */
+public class DNSResolverForRows extends DNSResolver {
+    static final String DEFAULT_ROW = "/default-region/default-row";
+
+    public DNSResolverForRows() {
+    }
+
+    public DNSResolverForRows(String hostRegionOverrides) {
+        super(hostRegionOverrides);
+    }
+
+    @Override
+    protected String resolveToNetworkLocation(String domainName) {
+        String[] parts = domainName.split("\\.");
+        if (parts.length <= 0) {
+            return DEFAULT_ROW;
+        }
+        String hostName = parts[0];
+        String[] labels = hostName.split("-");
+        if (labels.length != 4) {
+            return DEFAULT_ROW;
+        }
+
+        String region = hostNameToRegion.get(hostName);
+        if (null == region) {
+            region = labels[0];
+        }
+
+        final String rack = labels[1];
+
+        if (rack.length() < 2) {
+            // Default to rack name if the rack name format cannot be recognized
+            return String.format("/%s/%s", region, rack);
+        } else {
+            return String.format("/%s/%s", region, rack.substring(0, 2));
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/net/NetUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/net/NetUtils.java
new file mode 100644
index 0000000..765980e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/net/NetUtils.java
@@ -0,0 +1,74 @@
+/**
+ * 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.net;
+
+import org.apache.bookkeeper.net.DNSToSwitchMapping;
+
+import java.lang.reflect.Constructor;
+import java.lang.reflect.InvocationTargetException;
+
+/**
+ * Utils about network
+ */
+public class NetUtils {
+
+    /**
+     * Get the dns resolver from class <code>resolverClassName</code> with optional
+     * <code>hostRegionOverrides</code>.
+     * <p>
+     * It would try to load the class with the constructor with <code>hostRegionOverrides</code>.
+     * If it fails, it would fall back to load the class with default empty constructor.
+     * The interpretion of <code>hostRegionOverrides</code> is up to the implementation.
+     *
+     * @param resolverCls
+     *          resolver class
+     * @param hostRegionOverrides
+     *          host region overrides
+     * @return dns resolver
+     */
+    public static DNSToSwitchMapping getDNSResolver(Class<? extends DNSToSwitchMapping> resolverCls,
+                                                    String hostRegionOverrides) {
+        // first try to construct the dns resolver with overrides
+        Constructor<? extends DNSToSwitchMapping> constructor;
+        Object[] parameters;
+        try {
+            constructor = resolverCls.getDeclaredConstructor(String.class);
+            parameters = new Object[] { hostRegionOverrides };
+        } catch (NoSuchMethodException nsme) {
+            // no constructor with overrides
+            try {
+                constructor = resolverCls.getDeclaredConstructor();
+                parameters = new Object[0];
+            } catch (NoSuchMethodException nsme1) {
+                throw new RuntimeException("Unable to find constructor for dns resolver "
+                        + resolverCls, nsme1);
+            }
+        }
+        constructor.setAccessible(true);
+        try {
+            return constructor.newInstance(parameters);
+        } catch (InstantiationException ie) {
+            throw new RuntimeException("Unable to instantiate dns resolver " + resolverCls, ie);
+        } catch (IllegalAccessException iae) {
+            throw new RuntimeException("Illegal access to dns resolver " + resolverCls, iae);
+        } catch (InvocationTargetException ite) {
+            throw new RuntimeException("Unable to construct dns resolver " + resolverCls, ite);
+        }
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/net/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/net/package-info.java
new file mode 100644
index 0000000..70b1882
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/net/package-info.java
@@ -0,0 +1,33 @@
+/**
+ * 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.
+ */
+/**
+ * This package contains all the utilities of network.
+ *
+ * <h2>DNSResolver</h2>
+ *
+ * DNS resolver is the utility to resolve host name to a string which represents this host's network location.
+ * BookKeeper will use such network locations to place ensemble to ensure rack or region diversity to ensure
+ * data availability in the case of switch/router/region is down.
+ * <p>
+ * Available dns resolvers:
+ * <ul>
+ * <li>{@link org.apache.distributedlog.net.DNSResolverForRacks}
+ * <li>{@link org.apache.distributedlog.net.DNSResolverForRows}
+ * </ul>
+ */
+package org.apache.distributedlog.net;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/package-info.java
new file mode 100644
index 0000000..34a42ea
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Protocol & Core of DistributedLog
+ */
+package org.apache.distributedlog;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/rate/MovingAverageRate.java b/distributedlog-core/src/main/java/org/apache/distributedlog/rate/MovingAverageRate.java
new file mode 100644
index 0000000..14db547
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/rate/MovingAverageRate.java
@@ -0,0 +1,24 @@
+/**
+ * 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;
+
+public interface MovingAverageRate {
+    double get();
+    void add(long amount);
+    void inc();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/rate/MovingAverageRateFactory.java b/distributedlog-core/src/main/java/org/apache/distributedlog/rate/MovingAverageRateFactory.java
new file mode 100644
index 0000000..cd33e24
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/rate/MovingAverageRateFactory.java
@@ -0,0 +1,65 @@
+/**
+ * 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.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;
+
+public class MovingAverageRateFactory {
+
+    private static final int DEFAULT_INTERVAL_SECS = 1;
+
+    private final Timer timer;
+    private final TimerTask timerTask;
+    private final CopyOnWriteArrayList<SampledMovingAverageRate> avgs;
+
+    public MovingAverageRateFactory(Timer timer) {
+        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);
+    }
+
+    public MovingAverageRate create(int intervalSecs) {
+        SampledMovingAverageRate avg = new SampledMovingAverageRate(intervalSecs);
+        avgs.add(avg);
+        return avg;
+    }
+
+    public void close() {
+        timerTask.cancel();
+        avgs.clear();
+    }
+
+    private void sampleAll() {
+        for (SampledMovingAverageRate avg : avgs) {
+            avg.sample();
+        }
+    }
+}
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
new file mode 100644
index 0000000..0b3ccac
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/rate/SampledMovingAverageRate.java
@@ -0,0 +1,58 @@
+/**
+ * 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/readahead/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
new file mode 100644
index 0000000..d81f8a4
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/readahead/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * ReadAhead Mechanism for distributedlog streaming reads
+ */
+package org.apache.distributedlog.readahead;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstDLSNNotLessThanSelector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstDLSNNotLessThanSelector.java
new file mode 100644
index 0000000..1bba627
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstDLSNNotLessThanSelector.java
@@ -0,0 +1,46 @@
+/**
+ * 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.selector;
+
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecordWithDLSN;
+
+/**
+ * Save the first record with a dlsn not less than the dlsn provided.
+ */
+public class FirstDLSNNotLessThanSelector implements LogRecordSelector {
+
+    LogRecordWithDLSN result;
+    final DLSN dlsn;
+
+    public FirstDLSNNotLessThanSelector(DLSN dlsn) {
+        this.dlsn = dlsn;
+    }
+
+    @Override
+    public void process(LogRecordWithDLSN record) {
+        if ((record.getDlsn().compareTo(dlsn) >= 0) && (null == result)) {
+            this.result = record;
+        }
+    }
+
+    @Override
+    public LogRecordWithDLSN result() {
+        return this.result;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstRecordSelector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstRecordSelector.java
new file mode 100644
index 0000000..b6a6adf
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstRecordSelector.java
@@ -0,0 +1,46 @@
+/**
+ * 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.selector;
+
+import org.apache.distributedlog.LogRecordWithDLSN;
+
+/**
+ * Save the first record processed
+ */
+public class FirstRecordSelector implements LogRecordSelector {
+
+    final boolean includeControl;
+    LogRecordWithDLSN firstRecord;
+
+    public FirstRecordSelector(boolean includeControl) {
+        this.includeControl = includeControl;
+    }
+
+    @Override
+    public void process(LogRecordWithDLSN record) {
+        if (null == this.firstRecord
+                && (includeControl || !record.isControl())) {
+            this.firstRecord = record;
+        }
+    }
+
+    @Override
+    public LogRecordWithDLSN result() {
+        return this.firstRecord;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstTxIdNotLessThanSelector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstTxIdNotLessThanSelector.java
new file mode 100644
index 0000000..cd4e5a2
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/FirstTxIdNotLessThanSelector.java
@@ -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.
+ */
+package org.apache.distributedlog.selector;
+
+import org.apache.distributedlog.LogRecordWithDLSN;
+
+/**
+ * Save the first record with transaction id not less than the provided transaction id.
+ * If all records' transaction id is less than provided transaction id, save the last record.
+ */
+public class FirstTxIdNotLessThanSelector implements LogRecordSelector {
+
+    LogRecordWithDLSN result;
+    final long txId;
+    boolean found = false;
+
+    public FirstTxIdNotLessThanSelector(long txId) {
+        this.txId = txId;
+    }
+
+    @Override
+    public void process(LogRecordWithDLSN record) {
+        if (found) {
+            return;
+        }
+        this.result = record;
+        if (record.getTransactionId() >= txId) {
+            found = true;
+        }
+    }
+
+    @Override
+    public LogRecordWithDLSN result() {
+        return this.result;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/selector/LastRecordSelector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/LastRecordSelector.java
new file mode 100644
index 0000000..adabe41
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/LastRecordSelector.java
@@ -0,0 +1,38 @@
+/**
+ * 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.selector;
+
+import org.apache.distributedlog.LogRecordWithDLSN;
+
+/**
+ * Save the last record processed.
+ */
+public class LastRecordSelector implements LogRecordSelector {
+
+    LogRecordWithDLSN lastRecord;
+
+    @Override
+    public void process(LogRecordWithDLSN record) {
+        lastRecord = record;
+    }
+
+    @Override
+    public LogRecordWithDLSN result() {
+        return lastRecord;
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/selector/LogRecordSelector.java b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/LogRecordSelector.java
new file mode 100644
index 0000000..c32ddc9
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/selector/LogRecordSelector.java
@@ -0,0 +1,40 @@
+/**
+ * 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.selector;
+
+import org.apache.distributedlog.LogRecordWithDLSN;
+
+/**
+ * Visitor interface to process a set of records, and return some result.
+ */
+public interface LogRecordSelector {
+    /**
+     * Process a given <code>record</code>.
+     *
+     * @param record
+     *          log record to process
+     */
+    void process(LogRecordWithDLSN record);
+
+    /**
+     * Returned the selected log record after processing a set of records.
+     *
+     * @return the selected log record.
+     */
+    LogRecordWithDLSN result();
+}
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
new file mode 100644
index 0000000..199aa4c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/stats/BKExceptionStatsLogger.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.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/stats/BroadCastStatsLogger.java b/distributedlog-core/src/main/java/org/apache/distributedlog/stats/BroadCastStatsLogger.java
new file mode 100644
index 0000000..b6ca733
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/stats/BroadCastStatsLogger.java
@@ -0,0 +1,194 @@
+/**
+ * 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 com.google.common.base.Preconditions;
+
+import org.apache.bookkeeper.stats.CachingStatsLogger;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.OpStatsData;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * Stats Loggers that broadcast stats to multiple {@link StatsLogger}.
+ */
+public class BroadCastStatsLogger {
+
+    /**
+     * Create a broadcast stats logger of two stats loggers `<code>first</code>` and
+     * `<code>second</code>`. The returned stats logger doesn't allow registering any
+     * {@link Gauge}.
+     *
+     * @param first
+     *          first stats logger
+     * @param second
+     *          second stats logger
+     * @return broadcast stats logger
+     */
+    public static StatsLogger two(StatsLogger first, StatsLogger second) {
+        return new CachingStatsLogger(new Two(first, second));
+    }
+
+    static class Two implements StatsLogger {
+        protected final StatsLogger first;
+        protected final StatsLogger second;
+
+        private Two(StatsLogger first, StatsLogger second) {
+            super();
+            Preconditions.checkNotNull(first);
+            Preconditions.checkNotNull(second);
+            this.first = first;
+            this.second = second;
+        }
+
+        @Override
+        public OpStatsLogger getOpStatsLogger(final String statName) {
+            final OpStatsLogger firstLogger = first.getOpStatsLogger(statName);
+            final OpStatsLogger secondLogger = second.getOpStatsLogger(statName);
+            return new OpStatsLogger() {
+                @Override
+                public void registerFailedEvent(long l) {
+                    firstLogger.registerFailedEvent(l);
+                    secondLogger.registerFailedEvent(l);
+                }
+
+                @Override
+                public void registerSuccessfulEvent(long l) {
+                    firstLogger.registerSuccessfulEvent(l);
+                    secondLogger.registerSuccessfulEvent(l);
+                }
+
+                @Override
+                public OpStatsData toOpStatsData() {
+                    // Eventually consistent.
+                    return firstLogger.toOpStatsData();
+                }
+
+                @Override
+                public void clear() {
+                    firstLogger.clear();
+                    secondLogger.clear();
+                }
+            };
+        }
+
+        @Override
+        public Counter getCounter(final String statName) {
+            final Counter firstCounter = first.getCounter(statName);
+            final Counter secondCounter = second.getCounter(statName);
+            return new Counter() {
+                @Override
+                public void clear() {
+                    firstCounter.clear();
+                    secondCounter.clear();
+                }
+
+                @Override
+                public void inc() {
+                    firstCounter.inc();
+                    secondCounter.inc();
+                }
+
+                @Override
+                public void dec() {
+                    firstCounter.dec();
+                    secondCounter.dec();
+                }
+
+                @Override
+                public void add(long l) {
+                    firstCounter.add(l);
+                    secondCounter.add(l);
+                }
+
+                @Override
+                public Long get() {
+                    // Eventually consistent.
+                    return firstCounter.get();
+                }
+            };
+        }
+
+        @Override
+        public <T extends Number> void registerGauge(String statName, Gauge<T> gauge) {
+            // Different underlying stats loggers have different semantics wrt. gauge registration.
+            throw new RuntimeException("Cannot register a gauge on BroadCastStatsLogger.Two");
+        }
+
+        @Override
+        public <T extends Number> void unregisterGauge(String statName, Gauge<T> gauge) {
+            // no-op
+        }
+
+        @Override
+        public StatsLogger scope(final String scope) {
+            return new Two(first.scope(scope), second.scope(scope));
+        }
+
+        @Override
+        public void removeScope(String scope, StatsLogger statsLogger) {
+            if (!(statsLogger instanceof Two)) {
+                return;
+            }
+
+            Two another = (Two) statsLogger;
+
+            first.removeScope(scope, another.first);
+            second.removeScope(scope, another.second);
+        }
+    }
+
+    /**
+     * Create a broadcast stats logger of two stats loggers <code>master</code> and <code>slave</code>.
+     * It is similar as {@link #two(StatsLogger, StatsLogger)}, but it allows registering {@link Gauge}s.
+     * The {@link Gauge} will be registered under master.
+     *
+     * @param master
+     *          master stats logger to receive {@link Counter}, {@link OpStatsLogger} and {@link Gauge}.
+     * @param slave
+     *          slave stats logger to receive only {@link Counter} and {@link OpStatsLogger}.
+     * @return broadcast stats logger
+     */
+    public static StatsLogger masterslave(StatsLogger master, StatsLogger slave) {
+        return new CachingStatsLogger(new MasterSlave(master, slave));
+    }
+
+    static class MasterSlave extends Two {
+
+        private MasterSlave(StatsLogger master, StatsLogger slave) {
+            super(master, slave);
+        }
+
+        @Override
+        public <T extends Number> void registerGauge(String statName, Gauge<T> gauge) {
+            first.registerGauge(statName, gauge);
+        }
+
+        @Override
+        public <T extends Number> void unregisterGauge(String statName, Gauge<T> gauge) {
+            first.unregisterGauge(statName, gauge);
+        }
+
+        @Override
+        public StatsLogger scope(String scope) {
+            return new MasterSlave(first.scope(scope), second.scope(scope));
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/stats/OpStatsListener.java b/distributedlog-core/src/main/java/org/apache/distributedlog/stats/OpStatsListener.java
new file mode 100644
index 0000000..43641f0
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/stats/OpStatsListener.java
@@ -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.
+ */
+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;
+
+public class OpStatsListener<T> implements FutureEventListener<T> {
+    OpStatsLogger opStatsLogger;
+    Stopwatch stopwatch;
+
+    public OpStatsListener(OpStatsLogger opStatsLogger, Stopwatch stopwatch) {
+        this.opStatsLogger = opStatsLogger;
+        if (null == stopwatch) {
+            this.stopwatch = Stopwatch.createStarted();
+        } else {
+            this.stopwatch = stopwatch;
+        }
+    }
+
+    public OpStatsListener(OpStatsLogger opStatsLogger) {
+        this(opStatsLogger, null);
+    }
+
+    @Override
+    public void onSuccess(T value) {
+        opStatsLogger.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+    }
+
+    @Override
+    public void onFailure(Throwable cause) {
+        opStatsLogger.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionStateStore.java b/distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionStateStore.java
new file mode 100644
index 0000000..ebfc32a
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionStateStore.java
@@ -0,0 +1,42 @@
+/**
+ * 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.subscription;
+
+import java.io.Closeable;
+
+import scala.runtime.BoxedUnit;
+
+import org.apache.distributedlog.DLSN;
+import com.twitter.util.Future;
+
+public interface SubscriptionStateStore extends Closeable {
+    /**
+     * Get the last committed position stored for this subscription
+     *
+     * @return future represents the last commit position
+     */
+    public Future<DLSN> getLastCommitPosition();
+
+    /**
+     * Advances the position associated with the subscriber
+     *
+     * @param newPosition - new commit position
+     * @return future represents the advance result
+     */
+    public Future<BoxedUnit> 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/subscription/SubscriptionsStore.java
new file mode 100644
index 0000000..1974f1e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/subscription/SubscriptionsStore.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.subscription;
+
+import org.apache.distributedlog.DLSN;
+import com.twitter.util.Future;
+import scala.runtime.BoxedUnit;
+
+import java.io.Closeable;
+import java.util.Map;
+
+/**
+ * Store to manage subscriptions
+ */
+public interface SubscriptionsStore extends Closeable {
+
+    /**
+     * Get the last committed position stored for <i>subscriberId</i>.
+     *
+     * @param subscriberId
+     *          subscriber id
+     * @return future representing last committed position.
+     */
+    public 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();
+
+    /**
+     * Advance the last committed position for <i>subscriberId</i>.
+     *
+     * @param subscriberId
+     *          subscriber id.
+     * @param newPosition
+     *          new committed position.
+     * @return future representing advancing result.
+     */
+    public Future<BoxedUnit> advanceCommitPosition(String subscriberId, DLSN newPosition);
+
+    /**
+     * Delete the subscriber <i>subscriberId</i> permanently. Once the subscriber is deleted, all the
+     * data stored under this subscriber will be lost.
+     * @param subscriberId subscriber id
+     * @return future represent success or failure.
+     * 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);
+
+}
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
new file mode 100644
index 0000000..2c27088
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/DistributedLogTool.java
@@ -0,0 +1,2873 @@
+/**
+ * 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.tools;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.FileInputStream;
+import java.io.FileNotFoundException;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.InputStreamReader;
+import java.io.OutputStreamWriter;
+import java.io.PrintWriter;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Enumeration;
+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.SortedMap;
+import java.util.TreeMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.LinkedBlockingQueue;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+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.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.namespace.NamespaceDriver;
+import org.apache.distributedlog.util.Utils;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAccessor;
+import org.apache.bookkeeper.client.BookKeeperAdmin;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.client.LedgerReader;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
+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.commons.codec.binary.Hex;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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.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.exceptions.LogNotFoundException;
+import org.apache.distributedlog.LogReader;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.auditor.DLAuditor;
+import org.apache.distributedlog.bk.LedgerAllocator;
+import org.apache.distributedlog.bk.LedgerAllocatorUtils;
+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 static com.google.common.base.Charsets.UTF_8;
+
+public class DistributedLogTool extends Tool {
+
+    static final Logger logger = LoggerFactory.getLogger(DistributedLogTool.class);
+
+    static final List<String> EMPTY_LIST = Lists.newArrayList();
+
+    static int compareByCompletionTime(long time1, long time2) {
+        return time1 > time2 ? 1 : (time1 < time2 ? -1 : 0);
+    }
+
+    static final Comparator<LogSegmentMetadata> LOGSEGMENT_COMPARATOR_BY_TIME = new Comparator<LogSegmentMetadata>() {
+        @Override
+        public int compare(LogSegmentMetadata o1, LogSegmentMetadata o2) {
+            if (o1.isInProgress() && o2.isInProgress()) {
+                return compareByCompletionTime(o1.getFirstTxId(), o2.getFirstTxId());
+            } else if (!o1.isInProgress() && !o2.isInProgress()) {
+                return compareByCompletionTime(o1.getCompletionTime(), o2.getCompletionTime());
+            } else if (o1.isInProgress() && !o2.isInProgress()) {
+                return compareByCompletionTime(o1.getFirstTxId(), o2.getCompletionTime());
+            } else {
+                return compareByCompletionTime(o1.getCompletionTime(), o2.getFirstTxId());
+            }
+        }
+    };
+
+    static DLSN parseDLSN(String dlsnStr) throws ParseException {
+        if (dlsnStr.equals("InitialDLSN")) {
+            return DLSN.InitialDLSN;
+        }
+        String[] parts = dlsnStr.split(",");
+        if (parts.length != 3) {
+            throw new ParseException("Invalid dlsn : " + dlsnStr);
+        }
+        try {
+            return new DLSN(Long.parseLong(parts[0]), Long.parseLong(parts[1]), Long.parseLong(parts[2]));
+        } catch (Exception nfe) {
+            throw new ParseException("Invalid dlsn : " + dlsnStr);
+        }
+    }
+
+    /**
+     * Per DL Command, which parses basic options. e.g. uri.
+     */
+    protected abstract static class PerDLCommand extends OptsCommand {
+
+        protected Options options = new Options();
+        protected final DistributedLogConfiguration dlConf;
+        protected URI uri;
+        protected String zkAclId = null;
+        protected boolean force = false;
+        protected DistributedLogNamespace namespace = null;
+
+        protected PerDLCommand(String name, String description) {
+            super(name, description);
+            dlConf = new DistributedLogConfiguration();
+            // Tools are allowed to read old metadata as long as they can interpret it
+            dlConf.setDLLedgerMetadataSkipMinVersionCheck(true);
+            options.addOption("u", "uri", true, "DistributedLog URI");
+            options.addOption("c", "conf", true, "DistributedLog Configuration File");
+            options.addOption("a", "zk-acl-id", true, "Zookeeper ACL ID");
+            options.addOption("f", "force", false, "Force command (no warnings or prompts)");
+        }
+
+        @Override
+        protected int runCmd(CommandLine commandLine) throws Exception {
+            try {
+                parseCommandLine(commandLine);
+            } catch (ParseException pe) {
+                System.err.println("ERROR: failed to parse commandline : '" + pe.getMessage() + "'");
+                printUsage();
+                return -1;
+            }
+            try {
+                return runCmd();
+            } finally {
+                if (null != namespace) {
+                    namespace.close();
+                }
+            }
+        }
+
+        protected abstract int runCmd() throws Exception;
+
+        @Override
+        protected Options getOptions() {
+            return options;
+        }
+
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            if (!cmdline.hasOption("u")) {
+                throw new ParseException("No distributedlog uri provided.");
+            }
+            uri = URI.create(cmdline.getOptionValue("u"));
+            if (cmdline.hasOption("c")) {
+                String configFile = cmdline.getOptionValue("c");
+                try {
+                    dlConf.loadConf(new File(configFile).toURI().toURL());
+                } catch (ConfigurationException e) {
+                    throw new ParseException("Failed to load distributedlog configuration from " + configFile + ".");
+                } catch (MalformedURLException e) {
+                    throw new ParseException("Failed to load distributedlog configuration from " + configFile + ": malformed uri.");
+                }
+            }
+            if (cmdline.hasOption("a")) {
+                zkAclId = cmdline.getOptionValue("a");
+            }
+            if (cmdline.hasOption("f")) {
+                force = true;
+            }
+        }
+
+        protected DistributedLogConfiguration getConf() {
+            return dlConf;
+        }
+
+        protected URI getUri() {
+            return uri;
+        }
+
+        protected void setUri(URI uri) {
+            this.uri = uri;
+        }
+
+        protected String getZkAclId() {
+            return zkAclId;
+        }
+
+        protected void setZkAclId(String zkAclId) {
+            this.zkAclId = zkAclId;
+        }
+
+        protected boolean getForce() {
+            return force;
+        }
+
+        protected void setForce(boolean force) {
+            this.force = force;
+        }
+
+        protected DistributedLogNamespace getNamespace() throws IOException {
+            if (null == this.namespace) {
+                this.namespace = DistributedLogNamespaceBuilder.newBuilder()
+                        .uri(getUri())
+                        .conf(getConf())
+                        .build();
+            }
+            return this.namespace;
+        }
+
+        protected LogSegmentMetadataStore getLogSegmentMetadataStore() throws IOException {
+            return getNamespace()
+                    .getNamespaceDriver()
+                    .getLogStreamMetadataStore(NamespaceDriver.Role.READER)
+                    .getLogSegmentMetadataStore();
+        }
+
+        protected ZooKeeperClient getZooKeeperClient() throws IOException {
+            NamespaceDriver driver = getNamespace().getNamespaceDriver();
+            assert(driver instanceof BKNamespaceDriver);
+            return ((BKNamespaceDriver) driver).getWriterZKC();
+        }
+
+        protected BookKeeperClient getBookKeeperClient() throws IOException {
+            NamespaceDriver driver = getNamespace().getNamespaceDriver();
+            assert(driver instanceof BKNamespaceDriver);
+            return ((BKNamespaceDriver) driver).getReaderBKC();
+        }
+    }
+
+    /**
+     * Base class for simple command with no resource setup requirements.
+     */
+    public abstract static class SimpleCommand extends OptsCommand {
+
+        protected final Options options = new Options();
+
+        SimpleCommand(String name, String description) {
+            super(name, description);
+        }
+
+        @Override
+        protected int runCmd(CommandLine commandLine) throws Exception {
+            try {
+                parseCommandLine(commandLine);
+            } catch (ParseException pe) {
+                System.err.println("ERROR: failed to parse commandline : '" + pe.getMessage() + "'");
+                printUsage();
+                return -1;
+            }
+            return runSimpleCmd();
+        }
+
+        abstract protected int runSimpleCmd() throws Exception;
+
+        abstract protected void parseCommandLine(CommandLine cmdline) throws ParseException;
+
+        @Override
+        protected Options getOptions() {
+            return options;
+        }
+    }
+
+    /**
+     * Per Stream Command, which parse common options for per stream. e.g. stream name.
+     */
+    abstract static class PerStreamCommand extends PerDLCommand {
+
+        protected String streamName;
+
+        protected PerStreamCommand(String name, String description) {
+            super(name, description);
+            options.addOption("s", "stream", true, "Stream Name");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("s")) {
+                throw new ParseException("No stream name provided.");
+            }
+            streamName = cmdline.getOptionValue("s");
+        }
+
+        protected String getStreamName() {
+            return streamName;
+        }
+
+        protected void setStreamName(String streamName) {
+            this.streamName = streamName;
+        }
+    }
+
+    /**
+     * NOTE: we might consider adding a command to 'delete' namespace. The implementation of the namespace
+     *       driver should implement the 'delete' operation.
+     */
+    protected static class DeleteAllocatorPoolCommand extends PerDLCommand {
+
+        int concurrency = 1;
+        String allocationPoolPath = DistributedLogConstants.ALLOCATION_POOL_NODE;
+
+        DeleteAllocatorPoolCommand() {
+            super("delete_allocator_pool", "Delete allocator pool for a given distributedlog instance");
+            options.addOption("t", "concurrency", true, "Concurrency on deleting allocator pool.");
+            options.addOption("ap", "allocation-pool-path", true, "Ledger Allocation Pool Path");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (cmdline.hasOption("t")) {
+                concurrency = Integer.parseInt(cmdline.getOptionValue("t"));
+                if (concurrency <= 0) {
+                    throw new ParseException("Invalid concurrency value : " + concurrency + ": it must be greater or equal to 0.");
+                }
+            }
+            if (cmdline.hasOption("ap")) {
+                allocationPoolPath = cmdline.getOptionValue("ap");
+                if (!allocationPoolPath.startsWith(".") || !allocationPoolPath.contains("allocation")) {
+                    throw new ParseException("Invalid allocation pool path : " + allocationPoolPath + ": it must starts with a '.' and must contains 'allocation'");
+                }
+            }
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            String rootPath = getUri().getPath() + "/" + allocationPoolPath;
+            final ScheduledExecutorService allocationExecutor = Executors.newSingleThreadScheduledExecutor();
+            ExecutorService executorService = Executors.newFixedThreadPool(concurrency);
+            Preconditions.checkArgument(getNamespace() instanceof BKDistributedLogNamespace);
+            BKDistributedLogNamespace bkns = (BKDistributedLogNamespace) getNamespace();
+            final ZooKeeperClient zkc = ((BKNamespaceDriver) bkns.getNamespaceDriver()).getWriterZKC();
+            final BookKeeperClient bkc = ((BKNamespaceDriver) bkns.getNamespaceDriver()).getReaderBKC();
+            try {
+                List<String> pools = zkc.get().getChildren(rootPath, false);
+                final LinkedBlockingQueue<String> poolsToDelete = new LinkedBlockingQueue<String>();
+                if (getForce() || IOUtils.confirmPrompt("Are you sure you want to delete allocator pools : " + pools)) {
+                    for (String pool : pools) {
+                        poolsToDelete.add(rootPath + "/" + pool);
+                    }
+                    final CountDownLatch doneLatch = new CountDownLatch(concurrency);
+                    for (int i = 0; i < concurrency; i++) {
+                        final int tid = i;
+                        executorService.submit(new Runnable() {
+                            @Override
+                            public void run() {
+                                while (!poolsToDelete.isEmpty()) {
+                                    String poolPath = poolsToDelete.poll();
+                                    if (null == poolPath) {
+                                        break;
+                                    }
+                                    try {
+                                        LedgerAllocator allocator =
+                                                LedgerAllocatorUtils.createLedgerAllocatorPool(poolPath, 0, getConf(),
+                                                        zkc, bkc,
+                                                        allocationExecutor);
+                                        allocator.delete();
+                                        System.out.println("Deleted allocator pool : " + poolPath + " .");
+                                    } catch (IOException ioe) {
+                                        System.err.println("Failed to delete allocator pool " + poolPath + " : " + ioe.getMessage());
+                                    }
+                                }
+                                doneLatch.countDown();
+                                System.out.println("Thread " + tid + " is done.");
+                            }
+                        });
+                    }
+                    doneLatch.await();
+                }
+            } finally {
+                executorService.shutdown();
+                allocationExecutor.shutdown();
+            }
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "delete_allocator_pool";
+        }
+    }
+
+    public static class ListCommand extends PerDLCommand {
+
+        boolean printMetadata = false;
+        boolean printHex = false;
+
+        ListCommand() {
+            super("list", "list streams of a given distributedlog instance");
+            options.addOption("m", "meta", false, "Print metadata associated with each stream");
+            options.addOption("x", "hex", false, "Print metadata in hex format");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            printMetadata = cmdline.hasOption("m");
+            printHex = cmdline.hasOption("x");
+        }
+
+        @Override
+        protected String getUsage() {
+            return "list [options]";
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            printStreams(getNamespace());
+            return 0;
+        }
+
+        protected void printStreams(DistributedLogNamespace namespace) throws Exception {
+            Iterator<String> streams = namespace.getLogs();
+            System.out.println("Streams under " + getUri() + " : ");
+            System.out.println("--------------------------------");
+            while (streams.hasNext()) {
+                String streamName = streams.next();
+                System.out.println(streamName);
+                if (!printMetadata) {
+                    continue;
+                }
+                MetadataAccessor accessor =
+                        namespace.getNamespaceDriver().getMetadataAccessor(streamName);
+                byte[] metadata = accessor.getMetadata();
+                if (null == metadata || metadata.length == 0) {
+                    continue;
+                }
+                if (printHex) {
+                    System.out.println(Hex.encodeHexString(metadata));
+                } else {
+                    System.out.println(new String(metadata, UTF_8));
+                }
+                System.out.println("");
+            }
+            System.out.println("--------------------------------");
+        }
+    }
+
+    public static class WatchNamespaceCommand extends PerDLCommand implements NamespaceListener {
+        private Set<String> currentSet = Sets.<String>newHashSet();
+        private CountDownLatch doneLatch = new CountDownLatch(1);
+
+        WatchNamespaceCommand() {
+            super("watch", "watch and report changes for a dl namespace");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+        }
+
+        @Override
+        protected String getUsage() {
+            return "watch [options]";
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            watchAndReportChanges(getNamespace());
+            doneLatch.await();
+            return 0;
+        }
+
+        @Override
+        public synchronized void onStreamsChanged(Iterator<String> streams) {
+            Set<String> updatedSet = Sets.newHashSet(streams);
+            Set<String> oldStreams = Sets.difference(currentSet, updatedSet);
+            Set<String> newStreams = Sets.difference(updatedSet, currentSet);
+            currentSet = updatedSet;
+
+            System.out.println("Old streams : ");
+            for (String stream : oldStreams) {
+                System.out.println(stream);
+            }
+
+            System.out.println("New streams : ");
+            for (String stream : newStreams) {
+                System.out.println(stream);
+            }
+
+            System.out.println("");
+        }
+
+        protected void watchAndReportChanges(DistributedLogNamespace namespace) throws Exception {
+            namespace.registerNamespaceListener(this);
+        }
+    }
+
+    protected static class InspectCommand extends PerDLCommand {
+
+        int numThreads = 1;
+        String streamPrefix = null;
+        boolean printInprogressOnly = false;
+        boolean dumpEntries = false;
+        boolean orderByTime = false;
+        boolean printStreamsOnly = false;
+        boolean checkInprogressOnly = false;
+
+        InspectCommand() {
+            super("inspect", "Inspect streams under a given dl uri to find any potential corruptions");
+            options.addOption("t", "threads", true, "Number threads to do inspection.");
+            options.addOption("ft", "filter", true, "Stream filter by prefix");
+            options.addOption("i", "inprogress", false, "Print inprogress log segments only");
+            options.addOption("d", "dump", false, "Dump entries of inprogress log segments");
+            options.addOption("ot", "orderbytime", false, "Order the log segments by completion time");
+            options.addOption("pso", "print-stream-only", false, "Print streams only");
+            options.addOption("cio", "check-inprogress-only", false, "Check duplicated inprogress only");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (cmdline.hasOption("t")) {
+                numThreads = Integer.parseInt(cmdline.getOptionValue("t"));
+            }
+            if (cmdline.hasOption("ft")) {
+                streamPrefix = cmdline.getOptionValue("ft");
+            }
+            printInprogressOnly = cmdline.hasOption("i");
+            dumpEntries = cmdline.hasOption("d");
+            orderByTime = cmdline.hasOption("ot");
+            printStreamsOnly = cmdline.hasOption("pso");
+            checkInprogressOnly = cmdline.hasOption("cio");
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            SortedMap<String, List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates =
+                    new TreeMap<String, List<Pair<LogSegmentMetadata, List<String>>>>();
+            inspectStreams(corruptedCandidates);
+            System.out.println("Corrupted Candidates : ");
+            if (printStreamsOnly) {
+                System.out.println(corruptedCandidates.keySet());
+                return 0;
+            }
+            for (Map.Entry<String, List<Pair<LogSegmentMetadata, List<String>>>> entry : corruptedCandidates.entrySet()) {
+                System.out.println(entry.getKey() + " : \n");
+                for (Pair<LogSegmentMetadata, List<String>> pair : entry.getValue()) {
+                    System.out.println("\t - " + pair.getLeft());
+                    if (printInprogressOnly && dumpEntries) {
+                        int i = 0;
+                        for (String entryData : pair.getRight()) {
+                            System.out.println("\t" + i + "\t: " + entryData);
+                            ++i;
+                        }
+                    }
+                }
+                System.out.println();
+            }
+            return 0;
+        }
+
+        private void inspectStreams(final SortedMap<String, List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates)
+                throws Exception {
+            Iterator<String> streamCollection = getNamespace().getLogs();
+            final List<String> streams = new ArrayList<String>();
+            while (streamCollection.hasNext()) {
+                String s = streamCollection.next();
+                if (null != streamPrefix) {
+                    if (s.startsWith(streamPrefix)) {
+                        streams.add(s);
+                    }
+                } else {
+                    streams.add(s);
+                }
+            }
+            if (0 == streams.size()) {
+                return;
+            }
+            println("Streams : " + streams);
+            if (!getForce() && !IOUtils.confirmPrompt("Are you sure you want to inspect " + streams.size() + " streams")) {
+                return;
+            }
+            numThreads = Math.min(streams.size(), numThreads);
+            final int numStreamsPerThreads = streams.size() / numThreads;
+            Thread[] threads = new Thread[numThreads];
+            for (int i = 0; i < numThreads; i++) {
+                final int tid = i;
+                threads[i] = new Thread("Inspect-" + i) {
+                    @Override
+                    public void run() {
+                        try {
+                            inspectStreams(streams, tid, numStreamsPerThreads, corruptedCandidates);
+                            System.out.println("Thread " + tid + " finished.");
+                        } catch (Exception e) {
+                            System.err.println("Thread " + tid + " quits with exception : " + e.getMessage());
+                        }
+                    }
+                };
+                threads[i].start();
+            }
+            for (int i = 0; i < numThreads; i++) {
+                threads[i].join();
+            }
+        }
+
+        private void inspectStreams(List<String> streams,
+                                    int tid,
+                                    int numStreamsPerThreads,
+                                    SortedMap<String, List<Pair<LogSegmentMetadata, List<String>>>> corruptedCandidates)
+                throws Exception {
+            int startIdx = tid * numStreamsPerThreads;
+            int endIdx = Math.min(streams.size(), (tid + 1) * numStreamsPerThreads);
+            for (int i = startIdx; i < endIdx; i++) {
+                String s = streams.get(i);
+                BookKeeperClient bkc = getBookKeeperClient();
+                DistributedLogManager dlm = getNamespace().openLog(s);
+                try {
+                    List<LogSegmentMetadata> segments = dlm.getLogSegments();
+                    if (segments.size() <= 1) {
+                        continue;
+                    }
+                    boolean isCandidate = false;
+                    if (checkInprogressOnly) {
+                        Set<Long> inprogressSeqNos = new HashSet<Long>();
+                        for (LogSegmentMetadata segment : segments) {
+                            if (segment.isInProgress()) {
+                                inprogressSeqNos.add(segment.getLogSegmentSequenceNumber());
+                            }
+                        }
+                        for (LogSegmentMetadata segment : segments) {
+                            if (!segment.isInProgress() && inprogressSeqNos.contains(segment.getLogSegmentSequenceNumber())) {
+                                isCandidate = true;
+                            }
+                        }
+                    } else {
+                        LogSegmentMetadata firstSegment = segments.get(0);
+                        long lastSeqNo = firstSegment.getLogSegmentSequenceNumber();
+
+                        for (int j = 1; j < segments.size(); j++) {
+                            LogSegmentMetadata nextSegment = segments.get(j);
+                            if (lastSeqNo + 1 != nextSegment.getLogSegmentSequenceNumber()) {
+                                isCandidate = true;
+                                break;
+                            }
+                            ++lastSeqNo;
+                        }
+                    }
+                    if (isCandidate) {
+                        if (orderByTime) {
+                            Collections.sort(segments, LOGSEGMENT_COMPARATOR_BY_TIME);
+                        }
+                        List<Pair<LogSegmentMetadata, List<String>>> ledgers =
+                                new ArrayList<Pair<LogSegmentMetadata, List<String>>>();
+                        for (LogSegmentMetadata seg : segments) {
+                            LogSegmentMetadata segment = seg;
+                            List<String> dumpedEntries = new ArrayList<String>();
+                            if (segment.isInProgress()) {
+                                LedgerHandle lh = bkc.get().openLedgerNoRecovery(segment.getLogSegmentId(), BookKeeper.DigestType.CRC32,
+                                                                                 dlConf.getBKDigestPW().getBytes(UTF_8));
+                                try {
+                                    long lac = lh.readLastConfirmed();
+                                    segment = segment.mutator().setLastEntryId(lac).build();
+                                    if (printInprogressOnly && dumpEntries && lac >= 0) {
+                                        Enumeration<LedgerEntry> entries = lh.readEntries(0L, lac);
+                                        while (entries.hasMoreElements()) {
+                                            LedgerEntry entry = entries.nextElement();
+                                            dumpedEntries.add(new String(entry.getEntry(), UTF_8));
+                                        }
+                                    }
+                                } finally {
+                                    lh.close();
+                                }
+                            }
+                            if (printInprogressOnly) {
+                                if (segment.isInProgress()) {
+                                    ledgers.add(Pair.of(segment, dumpedEntries));
+                                }
+                            } else {
+                                ledgers.add(Pair.of(segment, EMPTY_LIST));
+                            }
+                        }
+                        synchronized (corruptedCandidates) {
+                            corruptedCandidates.put(s, ledgers);
+                        }
+                    }
+                } finally {
+                    dlm.close();
+                }
+            }
+        }
+
+        @Override
+        protected String getUsage() {
+            return "inspect [options]";
+        }
+    }
+
+    protected static class TruncateCommand extends PerDLCommand {
+
+        int numThreads = 1;
+        String streamPrefix = null;
+        boolean deleteStream = false;
+
+        TruncateCommand() {
+            super("truncate", "truncate streams under a given dl uri");
+            options.addOption("t", "threads", true, "Number threads to do truncation");
+            options.addOption("ft", "filter", true, "Stream filter by prefix");
+            options.addOption("d", "delete", false, "Delete Stream");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (cmdline.hasOption("t")) {
+                numThreads = Integer.parseInt(cmdline.getOptionValue("t"));
+            }
+            if (cmdline.hasOption("ft")) {
+                streamPrefix = cmdline.getOptionValue("ft");
+            }
+            if (cmdline.hasOption("d")) {
+                deleteStream = true;
+            }
+        }
+
+        @Override
+        protected String getUsage() {
+            return "truncate [options]";
+        }
+
+        protected void setFilter(String filter) {
+            this.streamPrefix = filter;
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            getConf().setZkAclId(getZkAclId());
+            return truncateStreams(getNamespace());
+        }
+
+        private int truncateStreams(final DistributedLogNamespace namespace) throws Exception {
+            Iterator<String> streamCollection = namespace.getLogs();
+            final List<String> streams = new ArrayList<String>();
+            while (streamCollection.hasNext()) {
+                String s = streamCollection.next();
+                if (null != streamPrefix) {
+                    if (s.startsWith(streamPrefix)) {
+                        streams.add(s);
+                    }
+                } else {
+                    streams.add(s);
+                }
+            }
+            if (0 == streams.size()) {
+                return 0;
+            }
+            System.out.println("Streams : " + streams);
+            if (!getForce() && !IOUtils.confirmPrompt("Do you want to truncate " + streams.size() + " streams ?")) {
+                return 0;
+            }
+            numThreads = Math.min(streams.size(), numThreads);
+            final int numStreamsPerThreads = streams.size() / numThreads + 1;
+            Thread[] threads = new Thread[numThreads];
+            for (int i = 0; i < numThreads; i++) {
+                final int tid = i;
+                threads[i] = new Thread("Truncate-" + i) {
+                    @Override
+                    public void run() {
+                        try {
+                            truncateStreams(namespace, streams, tid, numStreamsPerThreads);
+                            System.out.println("Thread " + tid + " finished.");
+                        } catch (IOException e) {
+                            System.err.println("Thread " + tid + " quits with exception : " + e.getMessage());
+                        }
+                    }
+                };
+                threads[i].start();
+            }
+            for (int i = 0; i < numThreads; i++) {
+                threads[i].join();
+            }
+            return 0;
+        }
+
+        private void truncateStreams(DistributedLogNamespace namespace, List<String> streams,
+                                     int tid, int numStreamsPerThreads) throws IOException {
+            int startIdx = tid * numStreamsPerThreads;
+            int endIdx = Math.min(streams.size(), (tid + 1) * numStreamsPerThreads);
+            for (int i = startIdx; i < endIdx; i++) {
+                String s = streams.get(i);
+                DistributedLogManager dlm = namespace.openLog(s);
+                try {
+                    if (deleteStream) {
+                        dlm.delete();
+                    } else {
+                        dlm.purgeLogsOlderThan(Long.MAX_VALUE);
+                    }
+                } finally {
+                    dlm.close();
+                }
+            }
+        }
+    }
+
+    public static class SimpleBookKeeperClient {
+        BookKeeperClient bkc;
+        ZooKeeperClient zkc;
+
+        public SimpleBookKeeperClient(DistributedLogConfiguration conf, URI uri) {
+            try {
+                zkc = ZooKeeperClientBuilder.newBuilder()
+                    .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                    .zkAclId(conf.getZkAclId())
+                    .uri(uri)
+                    .build();
+                BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
+                BKDLConfig.propagateConfiguration(bkdlConfig, conf);
+                bkc = BookKeeperClientBuilder.newBuilder()
+                        .zkc(zkc)
+                        .dlConfig(conf)
+                        .ledgersPath(bkdlConfig.getBkLedgersPath())
+                        .name("dlog")
+                        .build();
+            } catch (Exception e) {
+                close();
+            }
+        }
+        public BookKeeperClient client() {
+            return bkc;
+        }
+        public void close() {
+            if (null != bkc) {
+                bkc.close();
+            }
+            if (null != zkc) {
+                zkc.close();
+            }
+        }
+    }
+
+    protected static class ShowCommand extends PerStreamCommand {
+
+        SimpleBookKeeperClient bkc = null;
+        boolean listSegments = true;
+        boolean listEppStats = false;
+        long firstLid = 0;
+        long lastLid = -1;
+
+        ShowCommand() {
+            super("show", "show metadata of a given stream and list segments");
+            options.addOption("ns", "no-log-segments", false, "Do not list log segment metadata");
+            options.addOption("lp", "placement-stats", false, "Show ensemble placement stats");
+            options.addOption("fl", "first-ledger", true, "First log sement no");
+            options.addOption("ll", "last-ledger", true, "Last log sement no");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (cmdline.hasOption("fl")) {
+                try {
+                    firstLid = Long.parseLong(cmdline.getOptionValue("fl"));
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid ledger id " + cmdline.getOptionValue("fl"));
+                }
+            }
+            if (firstLid < 0) {
+                throw new IllegalArgumentException("Invalid ledger id " + firstLid);
+            }
+            if (cmdline.hasOption("ll")) {
+                try {
+                    lastLid = Long.parseLong(cmdline.getOptionValue("ll"));
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid ledger id " + cmdline.getOptionValue("ll"));
+                }
+            }
+            if (lastLid != -1 && firstLid > lastLid) {
+                throw new IllegalArgumentException("Invalid ledger ids " + firstLid + " " + lastLid);
+            }
+            listSegments = !cmdline.hasOption("ns");
+            listEppStats = cmdline.hasOption("lp");
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
+            try {
+                if (listEppStats) {
+                    bkc = new SimpleBookKeeperClient(getConf(), getUri());
+                }
+                printMetadata(dlm);
+            } finally {
+                dlm.close();
+                if (null != bkc) {
+                    bkc.close();
+                }
+            }
+            return 0;
+        }
+
+        private void printMetadata(DistributedLogManager dlm) throws Exception {
+            printHeader(dlm);
+            if (listSegments) {
+                System.out.println("Ledgers : ");
+                List<LogSegmentMetadata> segments = dlm.getLogSegments();
+                for (LogSegmentMetadata segment : segments) {
+                    if (include(segment)) {
+                        printLedgerRow(segment);
+                    }
+                }
+            }
+        }
+
+        private void printHeader(DistributedLogManager dlm) throws Exception {
+            DLSN firstDlsn = Await.result(dlm.getFirstDLSNAsync());
+            boolean endOfStreamMarked = dlm.isEndOfStreamMarked();
+            DLSN lastDlsn = dlm.getLastDLSN();
+            long firstTxnId = dlm.getFirstTxId();
+            long lastTxnId = dlm.getLastTxId();
+            long recordCount = dlm.getLogRecordCount();
+            String result = String.format("Stream : (firstTxId=%d, lastTxid=%d, firstDlsn=%s, lastDlsn=%s, endOfStreamMarked=%b, recordCount=%d)",
+                firstTxnId, lastTxnId, getDlsnName(firstDlsn), getDlsnName(lastDlsn), endOfStreamMarked, recordCount);
+            System.out.println(result);
+            if (listEppStats) {
+                printEppStatsHeader(dlm);
+            }
+        }
+
+        boolean include(LogSegmentMetadata segment) {
+            return (firstLid <= segment.getLogSegmentSequenceNumber() && (lastLid == -1 || lastLid >= segment.getLogSegmentSequenceNumber()));
+        }
+
+        private void printEppStatsHeader(DistributedLogManager dlm) throws Exception {
+            String label = "Ledger Placement :";
+            System.out.println(label);
+            Map<BookieSocketAddress, Integer> totals = new HashMap<BookieSocketAddress, Integer>();
+            List<LogSegmentMetadata> segments = dlm.getLogSegments();
+            for (LogSegmentMetadata segment : segments) {
+                if (include(segment)) {
+                    merge(totals, getBookieStats(segment));
+                }
+            }
+            List<Map.Entry<BookieSocketAddress, Integer>> entries = new ArrayList<Map.Entry<BookieSocketAddress, Integer>>(totals.entrySet());
+            Collections.sort(entries, new Comparator<Map.Entry<BookieSocketAddress, Integer>>() {
+                @Override
+                public int compare(Map.Entry<BookieSocketAddress, Integer> o1, Map.Entry<BookieSocketAddress, Integer> o2) {
+                    return o2.getValue() - o1.getValue();
+                }
+            });
+            int width = 0;
+            int totalEntries = 0;
+            for (Map.Entry<BookieSocketAddress, Integer> entry : entries) {
+                width = Math.max(width, label.length() + 1 + entry.getKey().toString().length());
+                totalEntries += entry.getValue();
+            }
+            for (Map.Entry<BookieSocketAddress, Integer> entry : entries) {
+                System.out.println(String.format("%"+width+"s\t%6.2f%%\t\t%d", entry.getKey(), entry.getValue()*1.0/totalEntries, entry.getValue()));
+            }
+        }
+
+        private void printLedgerRow(LogSegmentMetadata segment) throws Exception {
+            System.out.println(segment.getLogSegmentSequenceNumber() + "\t: " + segment);
+        }
+
+        private Map<BookieSocketAddress, Integer> getBookieStats(LogSegmentMetadata segment) throws Exception {
+            Map<BookieSocketAddress, Integer> stats = new HashMap<BookieSocketAddress, Integer>();
+            LedgerHandle lh = bkc.client().get().openLedgerNoRecovery(segment.getLogSegmentId(), BookKeeper.DigestType.CRC32,
+                    getConf().getBKDigestPW().getBytes(UTF_8));
+            long eidFirst = 0;
+            for (SortedMap.Entry<Long, ArrayList<BookieSocketAddress>> entry : LedgerReader.bookiesForLedger(lh).entrySet()) {
+                long eidLast = entry.getKey().longValue();
+                long count = eidLast - eidFirst + 1;
+                for (BookieSocketAddress bookie : entry.getValue()) {
+                    merge(stats, bookie, (int) count);
+                }
+                eidFirst = eidLast;
+            }
+            return stats;
+        }
+
+        void merge(Map<BookieSocketAddress, Integer> m, BookieSocketAddress bookie, Integer count) {
+            if (m.containsKey(bookie)) {
+                m.put(bookie, count + m.get(bookie).intValue());
+            } else {
+                m.put(bookie, count);
+            }
+        }
+
+        void merge(Map<BookieSocketAddress, Integer> m1, Map<BookieSocketAddress, Integer> m2) {
+            for (Map.Entry<BookieSocketAddress, Integer> entry : m2.entrySet()) {
+                merge(m1, entry.getKey(), entry.getValue());
+            }
+        }
+
+        String getDlsnName(DLSN dlsn) {
+            if (dlsn.equals(DLSN.InvalidDLSN)) {
+                return "InvalidDLSN";
+            }
+            return dlsn.toString();
+        }
+
+        @Override
+        protected String getUsage() {
+            return "show [options]";
+        }
+    }
+
+    static class CountCommand extends PerStreamCommand {
+
+        DLSN startDLSN = null;
+        DLSN endDLSN = null;
+
+        protected CountCommand() {
+            super("count", "count number records between dlsns");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            String[] args = cmdline.getArgs();
+            if (args.length < 1) {
+                throw new ParseException("Must specify at least start dlsn.");
+            }
+            if (args.length >= 1) {
+                startDLSN = parseDLSN(args[0]);
+            }
+            if (args.length >= 2) {
+                endDLSN = parseDLSN(args[1]);
+            }
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
+            try {
+                long count = 0;
+                if (null == endDLSN) {
+                    count = countToLastRecord(dlm);
+                } else {
+                    count = countFromStartToEnd(dlm);
+                }
+                System.out.println("total is " + count + " records.");
+                return 0;
+            } finally {
+                dlm.close();
+            }
+        }
+
+        int countFromStartToEnd(DistributedLogManager dlm) throws Exception {
+            int count = 0;
+            try {
+                LogReader reader = dlm.getInputStream(startDLSN);
+                try {
+                    LogRecordWithDLSN record = reader.readNext(false);
+                    LogRecordWithDLSN preRecord = record;
+                    System.out.println("first record : " + record);
+                    while (null != record) {
+                        if (record.getDlsn().compareTo(endDLSN) > 0) {
+                            break;
+                        }
+                        ++count;
+                        if (count % 1000 == 0) {
+                            logger.info("read {} records from {}...", count, getStreamName());
+                        }
+                        preRecord = record;
+                        record = reader.readNext(false);
+                    }
+                    System.out.println("last record : " + preRecord);
+                } finally {
+                    reader.close();
+                }
+            } finally {
+                dlm.close();
+            }
+            return count;
+        }
+
+        long countToLastRecord(DistributedLogManager dlm) throws Exception {
+            return Await.result(dlm.getLogRecordCountAsync(startDLSN)).longValue();
+        }
+
+        @Override
+        protected String getUsage() {
+            return "count <start> <end>";
+        }
+    }
+
+    public static class DeleteCommand extends PerStreamCommand {
+
+        protected DeleteCommand() {
+            super("delete", "delete a given stream");
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            getConf().setZkAclId(getZkAclId());
+            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
+            try {
+                dlm.delete();
+            } finally {
+                dlm.close();
+            }
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "delete";
+        }
+    }
+
+    public static class DeleteLedgersCommand extends PerDLCommand {
+
+        private final List<Long> ledgers = new ArrayList<Long>();
+
+        int numThreads = 1;
+
+        protected DeleteLedgersCommand() {
+            super("delete_ledgers", "delete given ledgers");
+            options.addOption("l", "ledgers", true, "List of ledgers, separated by comma");
+            options.addOption("lf", "ledgers-file", true, "File of list of ledgers, each line has a ledger id");
+            options.addOption("t", "concurrency", true, "Number of threads to run deletions");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (cmdline.hasOption("l") && cmdline.hasOption("lf")) {
+                throw new ParseException("Please specify ledgers: either use list or use file only.");
+            }
+            if (!cmdline.hasOption("l") && !cmdline.hasOption("lf")) {
+                throw new ParseException("No ledgers specified. Please specify ledgers either use list or use file only.");
+            }
+            if (cmdline.hasOption("l")) {
+                String ledgersStr = cmdline.getOptionValue("l");
+                String[] ledgerStrs = ledgersStr.split(",");
+                for (String ledgerStr : ledgerStrs) {
+                    ledgers.add(Long.parseLong(ledgerStr));
+                }
+            }
+            if (cmdline.hasOption("lf")) {
+                BufferedReader br = null;
+                try {
+
+                    br = new BufferedReader(new InputStreamReader(
+                            new FileInputStream(new File(cmdline.getOptionValue("lf"))), UTF_8.name()));
+                    String line;
+                    while ((line = br.readLine()) != null) {
+                        ledgers.add(Long.parseLong(line));
+                    }
+                } catch (FileNotFoundException e) {
+                    throw new ParseException("No ledgers file " + cmdline.getOptionValue("lf") + " found.");
+                } catch (IOException e) {
+                    throw new ParseException("Invalid ledgers file " + cmdline.getOptionValue("lf") + " found.");
+                } finally {
+                    if (null != br) {
+                        try {
+                            br.close();
+                        } catch (IOException e) {
+                            // no-op
+                        }
+                    }
+                }
+            }
+            if (cmdline.hasOption("t")) {
+                numThreads = Integer.parseInt(cmdline.getOptionValue("t"));
+            }
+        }
+
+        @Override
+        protected String getUsage() {
+            return "delete_ledgers [options]";
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
+            try {
+                final AtomicInteger numLedgers = new AtomicInteger(0);
+                final CountDownLatch doneLatch = new CountDownLatch(numThreads);
+                final AtomicInteger numFailures = new AtomicInteger(0);
+                final LinkedBlockingQueue<Long> ledgerQueue =
+                        new LinkedBlockingQueue<Long>();
+                ledgerQueue.addAll(ledgers);
+                for (int i = 0; i < numThreads; i++) {
+                    final int tid = i;
+                    executorService.submit(new Runnable() {
+                        @Override
+                        public void run() {
+                            while (true) {
+                                Long ledger = ledgerQueue.poll();
+                                if (null == ledger) {
+                                    break;
+                                }
+                                try {
+                                    getBookKeeperClient().get().deleteLedger(ledger);
+                                    int numLedgersDeleted = numLedgers.incrementAndGet();
+                                    if (numLedgersDeleted % 1000 == 0) {
+                                        System.out.println("Deleted " + numLedgersDeleted + " ledgers.");
+                                    }
+                                } catch (BKException.BKNoSuchLedgerExistsException e) {
+                                    int numLedgersDeleted = numLedgers.incrementAndGet();
+                                    if (numLedgersDeleted % 1000 == 0) {
+                                        System.out.println("Deleted " + numLedgersDeleted + " ledgers.");
+                                    }
+                                } catch (Exception e) {
+                                    numFailures.incrementAndGet();
+                                    break;
+                                }
+                            }
+                            doneLatch.countDown();
+                            System.out.println("Thread " + tid + " quits");
+                        }
+                    });
+                }
+                doneLatch.await();
+                if (numFailures.get() > 0) {
+                    throw new IOException("Encounter " + numFailures.get() + " failures during deleting ledgers");
+                }
+            } finally {
+                executorService.shutdown();
+            }
+            return 0;
+        }
+    }
+
+    public static class CreateCommand extends PerDLCommand {
+
+        final List<String> streams = new ArrayList<String>();
+
+        String streamPrefix = null;
+        String streamExpression = null;
+
+        CreateCommand() {
+            super("create", "create streams under a given namespace");
+            options.addOption("r", "prefix", true, "Prefix of stream name. E.g. 'QuantumLeapTest-'.");
+            options.addOption("e", "expression", true, "Expression to generate stream suffix. " +
+                              "Currently we support range 'x-y', list 'x,y,z' and name 'xyz'");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (cmdline.hasOption("r")) {
+                streamPrefix = cmdline.getOptionValue("r");
+            }
+            if (cmdline.hasOption("e")) {
+                streamExpression = cmdline.getOptionValue("e");
+            }
+            if (null == streamPrefix || null == streamExpression) {
+                throw new ParseException("Please specify stream prefix & expression.");
+            }
+        }
+
+        protected void generateStreams(String streamPrefix, String streamExpression) throws ParseException {
+            // parse the stream expression
+            if (streamExpression.contains("-")) {
+                // a range expression
+                String[] parts = streamExpression.split("-");
+                if (parts.length != 2) {
+                    throw new ParseException("Invalid stream index range : " + streamExpression);
+                }
+                try {
+                    int start = Integer.parseInt(parts[0]);
+                    int end = Integer.parseInt(parts[1]);
+                    if (start > end) {
+                        throw new ParseException("Invalid stream index range : " + streamExpression);
+                    }
+                    for (int i = start; i <= end; i++) {
+                        streams.add(streamPrefix + i);
+                    }
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid stream index range : " + streamExpression);
+                }
+            } else if (streamExpression.contains(",")) {
+                // a list expression
+                String[] parts = streamExpression.split(",");
+                try {
+                    for (String part : parts) {
+                        int idx = Integer.parseInt(part);
+                        streams.add(streamPrefix + idx);
+                    }
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid stream suffix list : " + streamExpression);
+                }
+            } else {
+                streams.add(streamPrefix + streamExpression);
+            }
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            generateStreams(streamPrefix, streamExpression);
+            if (streams.isEmpty()) {
+                System.out.println("Nothing to create.");
+                return 0;
+            }
+            if (!getForce() && !IOUtils.confirmPrompt("You are going to create streams : " + streams)) {
+                return 0;
+            }
+            getConf().setZkAclId(getZkAclId());
+            for (String stream : streams) {
+                getNamespace().createLog(stream);
+            }
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "create [options]";
+        }
+
+        protected void setPrefix(String prefix) {
+            this.streamPrefix = prefix;
+        }
+
+        protected void setExpression(String expression) {
+            this.streamExpression = expression;
+        }
+    }
+
+    protected static class DumpCommand extends PerStreamCommand {
+
+        boolean printHex = false;
+        boolean skipPayload = false;
+        Long fromTxnId = null;
+        DLSN fromDLSN = null;
+        int count = 100;
+
+        DumpCommand() {
+            super("dump", "dump records of a given stream");
+            options.addOption("x", "hex", false, "Print record in hex format");
+            options.addOption("sp", "skip-payload", false, "Skip printing the payload of the record");
+            options.addOption("o", "offset", true, "Txn ID to start dumping.");
+            options.addOption("n", "seqno", true, "Sequence Number to start dumping");
+            options.addOption("e", "eid", true, "Entry ID to start dumping");
+            options.addOption("t", "slot", true, "Slot to start dumping");
+            options.addOption("l", "limit", true, "Number of entries to dump. Default is 100.");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            printHex = cmdline.hasOption("x");
+            skipPayload = cmdline.hasOption("sp");
+            if (cmdline.hasOption("o")) {
+                try {
+                    fromTxnId = Long.parseLong(cmdline.getOptionValue("o"));
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid txn id " + cmdline.getOptionValue("o"));
+                }
+            }
+            if (cmdline.hasOption("l")) {
+                try {
+                    count = Integer.parseInt(cmdline.getOptionValue("l"));
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid count " + cmdline.getOptionValue("l"));
+                }
+                if (count <= 0) {
+                    throw new ParseException("Negative count found : " + count);
+                }
+            }
+            if (cmdline.hasOption("n")) {
+                long seqno;
+                try {
+                    seqno = Long.parseLong(cmdline.getOptionValue("n"));
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid sequence number " + cmdline.getOptionValue("n"));
+                }
+                long eid;
+                if (cmdline.hasOption("e")) {
+                    eid = Long.parseLong(cmdline.getOptionValue("e"));
+                } else {
+                    eid = 0;
+                }
+                long slot;
+                if (cmdline.hasOption("t")) {
+                    slot = Long.parseLong(cmdline.getOptionValue("t"));
+                } else {
+                    slot = 0;
+                }
+                fromDLSN = new DLSN(seqno, eid, slot);
+            }
+            if (null == fromTxnId && null == fromDLSN) {
+                throw new ParseException("No start Txn/DLSN is specified.");
+            }
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
+            long totalCount = dlm.getLogRecordCount();
+            try {
+                AsyncLogReader reader;
+                Object startOffset;
+                try {
+                    DLSN lastDLSN = Await.result(dlm.getLastDLSNAsync());
+                    System.out.println("Last DLSN : " + lastDLSN);
+                    if (null == fromDLSN) {
+                        reader = dlm.getAsyncLogReader(fromTxnId);
+                        startOffset = fromTxnId;
+                    } else {
+                        reader = dlm.getAsyncLogReader(fromDLSN);
+                        startOffset = fromDLSN;
+                    }
+                } catch (LogNotFoundException lee) {
+                    System.out.println("No stream found to dump records.");
+                    return 0;
+                }
+                try {
+                    System.out.println(String.format("Dump records for %s (from = %s, dump count = %d, total records = %d)",
+                            getStreamName(), startOffset, count, totalCount));
+
+                    dumpRecords(reader);
+                } finally {
+                    Utils.close(reader);
+                }
+            } finally {
+                dlm.close();
+            }
+            return 0;
+        }
+
+        private void dumpRecords(AsyncLogReader reader) throws Exception {
+            int numRead = 0;
+            LogRecord record = Await.result(reader.readNext());
+            while (record != null) {
+                // dump the record
+                dumpRecord(record);
+                ++numRead;
+                if (numRead >= count) {
+                    break;
+                }
+                record = Await.result(reader.readNext());
+            }
+            if (numRead == 0) {
+                System.out.println("No records.");
+            } else {
+                System.out.println("------------------------------------------------");
+            }
+        }
+
+        private void dumpRecord(LogRecord record) {
+            System.out.println("------------------------------------------------");
+            if (record instanceof LogRecordWithDLSN) {
+                System.out.println("Record (txn = " + record.getTransactionId() + ", bytes = "
+                        + record.getPayload().length + ", dlsn = "
+                        + ((LogRecordWithDLSN) record).getDlsn() + ", sequence id = "
+                        + ((LogRecordWithDLSN) record).getSequenceId() + ")");
+            } else {
+                System.out.println("Record (txn = " + record.getTransactionId() + ", bytes = "
+                        + record.getPayload().length + ")");
+            }
+            System.out.println("");
+
+            if (skipPayload) {
+                return;
+            }
+
+            if (printHex) {
+                System.out.println(Hex.encodeHexString(record.getPayload()));
+            } else {
+                System.out.println(new String(record.getPayload(), UTF_8));
+            }
+        }
+
+        @Override
+        protected String getUsage() {
+            return "dump [options]";
+        }
+
+        protected void setFromTxnId(Long fromTxnId) {
+            this.fromTxnId = fromTxnId;
+        }
+    }
+
+    /**
+     * TODO: refactor inspect & inspectstream
+     * TODO: support force
+     *
+     * inspectstream -lac -gap (different options for different operations for a single stream)
+     * inspect -lac -gap (inspect the namespace, which will use inspect stream)
+     */
+    static class InspectStreamCommand extends PerStreamCommand {
+
+        InspectStreamCommand() {
+            super("inspectstream", "Inspect a given stream to identify any metadata corruptions");
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            DistributedLogManager dlm = getNamespace().openLog(getStreamName());
+            try {
+                return inspectAndRepair(dlm.getLogSegments());
+            } finally {
+                dlm.close();
+            }
+        }
+
+        protected int inspectAndRepair(List<LogSegmentMetadata> segments) throws Exception {
+            LogSegmentMetadataStore metadataStore = getLogSegmentMetadataStore();
+            ZooKeeperClient zkc = getZooKeeperClient();
+            BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, getUri());
+            BKDLConfig.propagateConfiguration(bkdlConfig, getConf());
+            BookKeeperClient bkc = BookKeeperClientBuilder.newBuilder()
+                    .dlConfig(getConf())
+                    .zkServers(bkdlConfig.getBkZkServersForReader())
+                    .ledgersPath(bkdlConfig.getBkLedgersPath())
+                    .name("dlog")
+                    .build();
+            try {
+                List<LogSegmentMetadata> segmentsToRepair = inspectLogSegments(bkc, segments);
+                if (segmentsToRepair.isEmpty()) {
+                    System.out.println("The stream is good. No log segments to repair.");
+                    return 0;
+                }
+                System.out.println(segmentsToRepair.size() + " segments to repair : ");
+                System.out.println(segmentsToRepair);
+                System.out.println();
+                if (!IOUtils.confirmPrompt("Do you want to repair them (Y/N): ")) {
+                    return 0;
+                }
+                repairLogSegments(metadataStore, bkc, segmentsToRepair);
+                return 0;
+            } finally {
+                bkc.close();
+            }
+        }
+
+        protected List<LogSegmentMetadata> inspectLogSegments(
+                BookKeeperClient bkc, List<LogSegmentMetadata> segments) throws Exception {
+            List<LogSegmentMetadata> segmentsToRepair = new ArrayList<LogSegmentMetadata>();
+            for (LogSegmentMetadata segment : segments) {
+                if (!segment.isInProgress() && !inspectLogSegment(bkc, segment)) {
+                    segmentsToRepair.add(segment);
+                }
+            }
+            return segmentsToRepair;
+        }
+
+        /**
+         * Inspect a given log segment.
+         *
+         * @param bkc
+         *          bookkeeper client
+         * @param metadata
+         *          metadata of the log segment to
+         * @return true if it is a good stream, false if the stream has inconsistent metadata.
+         * @throws Exception
+         */
+        protected boolean inspectLogSegment(BookKeeperClient bkc,
+                                            LogSegmentMetadata metadata) throws Exception {
+            if (metadata.isInProgress()) {
+                System.out.println("Skip inprogress log segment " + metadata);
+                return true;
+            }
+            long ledgerId = metadata.getLogSegmentId();
+            LedgerHandle lh = bkc.get().openLedger(ledgerId, BookKeeper.DigestType.CRC32,
+                    getConf().getBKDigestPW().getBytes(UTF_8));
+            LedgerHandle readLh = bkc.get().openLedger(ledgerId, BookKeeper.DigestType.CRC32,
+                    getConf().getBKDigestPW().getBytes(UTF_8));
+            LedgerReader lr = new LedgerReader(bkc.get());
+            final AtomicReference<List<LedgerEntry>> entriesHolder = new AtomicReference<List<LedgerEntry>>(null);
+            final AtomicInteger rcHolder = new AtomicInteger(-1234);
+            final CountDownLatch doneLatch = new CountDownLatch(1);
+            try {
+                lr.forwardReadEntriesFromLastConfirmed(readLh, new BookkeeperInternalCallbacks.GenericCallback<List<LedgerEntry>>() {
+                    @Override
+                    public void operationComplete(int rc, List<LedgerEntry> entries) {
+                        rcHolder.set(rc);
+                        entriesHolder.set(entries);
+                        doneLatch.countDown();
+                    }
+                });
+                doneLatch.await();
+                if (BKException.Code.OK != rcHolder.get()) {
+                    throw BKException.create(rcHolder.get());
+                }
+                List<LedgerEntry> entries = entriesHolder.get();
+                long lastEntryId;
+                if (entries.isEmpty()) {
+                    lastEntryId = LedgerHandle.INVALID_ENTRY_ID;
+                } else {
+                    LedgerEntry lastEntry = entries.get(entries.size() - 1);
+                    lastEntryId = lastEntry.getEntryId();
+                }
+                if (lastEntryId != lh.getLastAddConfirmed()) {
+                    System.out.println("Inconsistent Last Add Confirmed Found for LogSegment " + metadata.getLogSegmentSequenceNumber() + ": ");
+                    System.out.println("\t metadata: " + metadata);
+                    System.out.println("\t lac in ledger metadata is " + lh.getLastAddConfirmed() + ", but lac in bookies is " + lastEntryId);
+                    return false;
+                } else {
+                    return true;
+                }
+            } finally {
+                lh.close();
+                readLh.close();
+            }
+        }
+
+        protected void repairLogSegments(LogSegmentMetadataStore metadataStore,
+                                         BookKeeperClient bkc,
+                                         List<LogSegmentMetadata> segments) throws Exception {
+            BookKeeperAdmin bkAdmin = new BookKeeperAdmin(bkc.get());
+            try {
+                MetadataUpdater metadataUpdater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(
+                        getConf(), metadataStore);
+                for (LogSegmentMetadata segment : segments) {
+                    repairLogSegment(bkAdmin, metadataUpdater, segment);
+                }
+            } finally {
+                bkAdmin.close();
+            }
+        }
+
+        protected void repairLogSegment(BookKeeperAdmin bkAdmin,
+                                        MetadataUpdater metadataUpdater,
+                                        LogSegmentMetadata segment) throws Exception {
+            if (segment.isInProgress()) {
+                System.out.println("Skip inprogress log segment " + segment);
+                return;
+            }
+            LedgerHandle lh = bkAdmin.openLedger(segment.getLogSegmentId(), true);
+            long lac = lh.getLastAddConfirmed();
+            Enumeration<LedgerEntry> entries = lh.readEntries(lac, lac);
+            if (!entries.hasMoreElements()) {
+                throw new IOException("Entry " + lac + " isn't found for " + segment);
+            }
+            LedgerEntry lastEntry = entries.nextElement();
+            Entry.Reader reader = Entry.newBuilder()
+                    .setLogSegmentInfo(segment.getLogSegmentSequenceNumber(), segment.getStartSequenceId())
+                    .setEntryId(lastEntry.getEntryId())
+                    .setEnvelopeEntry(LogSegmentMetadata.supportsEnvelopedEntries(segment.getVersion()))
+                    .setInputStream(lastEntry.getEntryInputStream())
+                    .buildReader();
+            LogRecordWithDLSN record = reader.nextRecord();
+            LogRecordWithDLSN lastRecord = null;
+            while (null != record) {
+                lastRecord = record;
+                record = reader.nextRecord();
+            }
+            if (null == lastRecord) {
+                throw new IOException("No record found in entry " + lac + " for " + segment);
+            }
+            System.out.println("Updating last record for " + segment + " to " + lastRecord);
+            if (!IOUtils.confirmPrompt("Do you want to make this change (Y/N): ")) {
+                return;
+            }
+            metadataUpdater.updateLastRecord(segment, lastRecord);
+        }
+
+        @Override
+        protected String getUsage() {
+            return "inspectstream [options]";
+        }
+    }
+
+    static interface BKCommandRunner {
+        int run(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception;
+    }
+
+    abstract static class PerBKCommand extends PerDLCommand {
+
+        protected PerBKCommand(String name, String description) {
+            super(name, description);
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            return runBKCommand(new BKCommandRunner() {
+                @Override
+                public int run(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception {
+                    return runBKCmd(zkc, bkc);
+                }
+            });
+        }
+
+        protected int runBKCommand(BKCommandRunner runner) throws Exception {
+            return runner.run(getZooKeeperClient(), getBookKeeperClient());
+        }
+
+        abstract protected int runBKCmd(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception;
+    }
+
+    static class RecoverCommand extends PerBKCommand {
+
+        final List<Long> ledgers = new ArrayList<Long>();
+        boolean query = false;
+        boolean dryrun = false;
+        boolean skipOpenLedgers = false;
+        boolean fenceOnly = false;
+        int fenceRate = 1;
+        int concurrency = 1;
+        final Set<BookieSocketAddress> bookiesSrc = new HashSet<BookieSocketAddress>();
+        int partition = 0;
+        int numPartitions = 0;
+
+        RecoverCommand() {
+            super("recover", "Recover the ledger data that stored on failed bookies");
+            options.addOption("l", "ledger", true, "Specific ledger to recover");
+            options.addOption("lf", "ledgerfile", true, "File contains ledgers list");
+            options.addOption("q", "query", false, "Query the ledgers that contain given bookies");
+            options.addOption("d", "dryrun", false, "Print the recovery plan w/o actually recovering");
+            options.addOption("cy", "concurrency", true, "Number of ledgers could be recovered in parallel");
+            options.addOption("sk", "skipOpenLedgers", false, "Skip recovering open ledgers");
+            options.addOption("p", "partition", true, "partition");
+            options.addOption("n", "num-partitions", true, "num partitions");
+            options.addOption("fo", "fence-only", true, "fence the ledgers only w/o re-replicating entries");
+            options.addOption("fr", "fence-rate", true, "rate on fencing ledgers");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            query = cmdline.hasOption("q");
+            force = cmdline.hasOption("f");
+            dryrun = cmdline.hasOption("d");
+            skipOpenLedgers = cmdline.hasOption("sk");
+            fenceOnly = cmdline.hasOption("fo");
+            if (cmdline.hasOption("l")) {
+                String[] lidStrs = cmdline.getOptionValue("l").split(",");
+                try {
+                    for (String lidStr : lidStrs) {
+                        ledgers.add(Long.parseLong(lidStr));
+                    }
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid ledger id provided : " + cmdline.getOptionValue("l"));
+                }
+            }
+            if (cmdline.hasOption("lf")) {
+                String file = cmdline.getOptionValue("lf");
+                try {
+                    BufferedReader br = new BufferedReader(
+                            new InputStreamReader(new FileInputStream(file), UTF_8.name()));
+                    try {
+                        String line = br.readLine();
+
+                        while (line != null) {
+                            ledgers.add(Long.parseLong(line));
+                            line = br.readLine();
+                        }
+                    } finally {
+                        br.close();
+                    }
+                } catch (IOException e) {
+                    throw new ParseException("Invalid ledgers file provided : " + file);
+                }
+            }
+            if (cmdline.hasOption("cy")) {
+                try {
+                    concurrency = Integer.parseInt(cmdline.getOptionValue("cy"));
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid concurrency provided : " + cmdline.getOptionValue("cy"));
+                }
+            }
+            if (cmdline.hasOption("p")) {
+                partition = Integer.parseInt(cmdline.getOptionValue("p"));
+            }
+            if (cmdline.hasOption("n")) {
+                numPartitions = Integer.parseInt(cmdline.getOptionValue("n"));
+            }
+            if (cmdline.hasOption("fr")) {
+                fenceRate = Integer.parseInt(cmdline.getOptionValue("fr"));
+            }
+            // Get bookies list to recover
+            String[] args = cmdline.getArgs();
+            final String[] bookieStrs = args[0].split(",");
+            for (String bookieStr : bookieStrs) {
+                final String bookieStrParts[] = bookieStr.split(":");
+                if (bookieStrParts.length != 2) {
+                    throw new ParseException("BookieSrcs has invalid bookie address format (host:port expected) : "
+                            + bookieStr);
+                }
+                try {
+                    bookiesSrc.add(new BookieSocketAddress(bookieStrParts[0],
+                            Integer.parseInt(bookieStrParts[1])));
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid ledger id provided : " + cmdline.getOptionValue("l"));
+                }
+            }
+        }
+
+        @Override
+        protected int runBKCmd(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception {
+            BookKeeperAdmin bkAdmin = new BookKeeperAdmin(bkc.get());
+            try {
+                if (query) {
+                    return bkQuery(bkAdmin, bookiesSrc);
+                }
+                if (fenceOnly) {
+                    return bkFence(bkc, ledgers, fenceRate);
+                }
+                if (!force) {
+                    System.out.println("Bookies : " + bookiesSrc);
+                    if (!IOUtils.confirmPrompt("Do you want to recover them: (Y/N)")) {
+                        return -1;
+                    }
+                }
+                if (!ledgers.isEmpty()) {
+                    System.out.println("Ledgers : " + ledgers);
+                    long numProcessed = 0;
+                    Iterator<Long> ledgersIter = ledgers.iterator();
+                    LinkedBlockingQueue<Long> ledgersToProcess = new LinkedBlockingQueue<Long>();
+                    while (ledgersIter.hasNext()) {
+                        long lid = ledgersIter.next();
+                        if (numPartitions <=0 || (numPartitions > 0 && lid % numPartitions == partition)) {
+                            ledgersToProcess.add(lid);
+                            ++numProcessed;
+                        }
+                        if (ledgersToProcess.size() == 10000) {
+                            System.out.println("Processing " + numProcessed + " ledgers");
+                            bkRecovery(ledgersToProcess, bookiesSrc, dryrun, skipOpenLedgers);
+                            ledgersToProcess.clear();
+                            System.out.println("Processed " + numProcessed + " ledgers");
+                        }
+                    }
+                    if (!ledgersToProcess.isEmpty()) {
+                        System.out.println("Processing " + numProcessed + " ledgers");
+                        bkRecovery(ledgersToProcess, bookiesSrc, dryrun, skipOpenLedgers);
+                        System.out.println("Processed " + numProcessed + " ledgers");
+                    }
+                    System.out.println("Done.");
+                    CountDownLatch latch = new CountDownLatch(1);
+                    latch.await();
+                    return 0;
+                }
+                return bkRecovery(bkAdmin, bookiesSrc, dryrun, skipOpenLedgers);
+            } finally {
+                bkAdmin.close();
+            }
+        }
+
+        private int bkFence(final BookKeeperClient bkc, List<Long> ledgers, int fenceRate) throws Exception {
+            if (ledgers.isEmpty()) {
+                System.out.println("Nothing to fence. Done.");
+                return 0;
+            }
+            ExecutorService executorService = Executors.newCachedThreadPool();
+            final RateLimiter rateLimiter = RateLimiter.create(fenceRate);
+            final byte[] passwd = getConf().getBKDigestPW().getBytes(UTF_8);
+            final CountDownLatch latch = new CountDownLatch(ledgers.size());
+            final AtomicInteger numPendings = new AtomicInteger(ledgers.size());
+            final LinkedBlockingQueue<Long> ledgersQueue = new LinkedBlockingQueue<Long>();
+            ledgersQueue.addAll(ledgers);
+
+            for (int i = 0; i < concurrency; i++) {
+                executorService.submit(new Runnable() {
+                    @Override
+                    public void run() {
+                        while (!ledgersQueue.isEmpty()) {
+                            rateLimiter.acquire();
+                            Long lid = ledgersQueue.poll();
+                            if (null == lid) {
+                                break;
+                            }
+                            System.out.println("Fencing ledger " + lid);
+                            int numRetries = 3;
+                            while (numRetries > 0) {
+                                try {
+                                    LedgerHandle lh = bkc.get().openLedger(lid, BookKeeper.DigestType.CRC32, passwd);
+                                    lh.close();
+                                    System.out.println("Fenced ledger " + lid + ", " + numPendings.decrementAndGet() + " left.");
+                                    latch.countDown();
+                                } catch (BKException.BKNoSuchLedgerExistsException bke) {
+                                    System.out.println("Skipped fence non-exist ledger " + lid + ", " + numPendings.decrementAndGet() + " left.");
+                                    latch.countDown();
+                                } catch (BKException.BKLedgerRecoveryException lre) {
+                                    --numRetries;
+                                    continue;
+                                } catch (Exception e) {
+                                    e.printStackTrace();
+                                    break;
+                                }
+                                numRetries = 0;
+                            }
+                        }
+                        System.out.println("Thread exits");
+                    }
+                });
+            }
+            latch.await();
+            SchedulerUtils.shutdownScheduler(executorService, 2, TimeUnit.MINUTES);
+            return 0;
+        }
+
+        private int bkQuery(BookKeeperAdmin bkAdmin, Set<BookieSocketAddress> bookieAddrs)
+                throws InterruptedException, BKException {
+            SortedMap<Long, LedgerMetadata> ledgersContainBookies =
+                    bkAdmin.getLedgersContainBookies(bookieAddrs);
+            System.err.println("NOTE: Bookies in inspection list are marked with '*'.");
+            for (Map.Entry<Long, LedgerMetadata> ledger : ledgersContainBookies.entrySet()) {
+                System.out.println("ledger " + ledger.getKey() + " : " + ledger.getValue().getState());
+                Map<Long, Integer> numBookiesToReplacePerEnsemble =
+                        inspectLedger(ledger.getValue(), bookieAddrs);
+                System.out.print("summary: [");
+                for (Map.Entry<Long, Integer> entry : numBookiesToReplacePerEnsemble.entrySet()) {
+                    System.out.print(entry.getKey() + "=" + entry.getValue() + ", ");
+                }
+                System.out.println("]");
+                System.out.println();
+            }
+            System.out.println("Done");
+            return 0;
+        }
+
+        private Map<Long, Integer> inspectLedger(LedgerMetadata metadata, Set<BookieSocketAddress> bookiesToInspect) {
+            Map<Long, Integer> numBookiesToReplacePerEnsemble = new TreeMap<Long, Integer>();
+            for (Map.Entry<Long, ArrayList<BookieSocketAddress>> ensemble : metadata.getEnsembles().entrySet()) {
+                ArrayList<BookieSocketAddress> bookieList = ensemble.getValue();
+                System.out.print(ensemble.getKey() + ":\t");
+                int numBookiesToReplace = 0;
+                for (BookieSocketAddress bookie: bookieList) {
+                    System.out.print(bookie.toString());
+                    if (bookiesToInspect.contains(bookie)) {
+                        System.out.print("*");
+                        ++numBookiesToReplace;
+                    } else {
+                        System.out.print(" ");
+                    }
+                    System.out.print(" ");
+                }
+                System.out.println();
+                numBookiesToReplacePerEnsemble.put(ensemble.getKey(), numBookiesToReplace);
+            }
+            return numBookiesToReplacePerEnsemble;
+        }
+
+        private int bkRecovery(final LinkedBlockingQueue<Long> ledgers, final Set<BookieSocketAddress> bookieAddrs,
+                               final boolean dryrun, final boolean skipOpenLedgers)
+                throws Exception {
+            return runBKCommand(new BKCommandRunner() {
+                @Override
+                public int run(ZooKeeperClient zkc, BookKeeperClient bkc) throws Exception {
+                    BookKeeperAdmin bkAdmin = new BookKeeperAdmin(bkc.get());
+                    try {
+                        bkRecovery(bkAdmin, ledgers, bookieAddrs, dryrun, skipOpenLedgers);
+                        return 0;
+                    } finally {
+                        bkAdmin.close();
+                    }
+                }
+            });
+        }
+
+        private int bkRecovery(final BookKeeperAdmin bkAdmin, final LinkedBlockingQueue<Long> ledgers,
+                               final Set<BookieSocketAddress> bookieAddrs,
+                               final boolean dryrun, final boolean skipOpenLedgers)
+                throws InterruptedException, BKException {
+            final AtomicInteger numPendings = new AtomicInteger(ledgers.size());
+            final ExecutorService executorService = Executors.newCachedThreadPool();
+            final CountDownLatch doneLatch = new CountDownLatch(concurrency);
+            Runnable r = new Runnable() {
+                @Override
+                public void run() {
+                    while (!ledgers.isEmpty()) {
+                        long lid = -1L;
+                        try {
+                            lid = ledgers.take();
+                            System.out.println("Recovering ledger " + lid);
+                            bkAdmin.recoverBookieData(lid, bookieAddrs, dryrun, skipOpenLedgers);
+                            System.out.println("Recovered ledger completed : " + lid + ", " + numPendings.decrementAndGet() + " left");
+                        } catch (InterruptedException e) {
+                            Thread.currentThread().interrupt();
+                            doneLatch.countDown();
+                            break;
+                        } catch (BKException ke) {
+                            System.out.println("Recovered ledger failed : " + lid + ", rc = " + BKException.getMessage(ke.getCode()));
+                        }
+                    }
+                    doneLatch.countDown();
+                }
+            };
+            for (int i = 0; i < concurrency; i++) {
+                executorService.submit(r);
+            }
+            doneLatch.await();
+            SchedulerUtils.shutdownScheduler(executorService, 2, TimeUnit.MINUTES);
+            return 0;
+        }
+
+        private int bkRecovery(BookKeeperAdmin bkAdmin, Set<BookieSocketAddress> bookieAddrs,
+                               boolean dryrun, boolean skipOpenLedgers)
+                throws InterruptedException, BKException {
+            bkAdmin.recoverBookieData(bookieAddrs, dryrun, skipOpenLedgers);
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "recover [options] <bookiesSrc>";
+        }
+    }
+
+    /**
+     * Per Ledger Command, which parse common options for per ledger. e.g. ledger id.
+     */
+    abstract static class PerLedgerCommand extends PerDLCommand {
+
+        protected long ledgerId;
+
+        protected PerLedgerCommand(String name, String description) {
+            super(name, description);
+            options.addOption("l", "ledger", true, "Ledger ID");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("l")) {
+                throw new ParseException("No ledger provided.");
+            }
+            ledgerId = Long.parseLong(cmdline.getOptionValue("l"));
+        }
+
+        protected long getLedgerID() {
+            return ledgerId;
+        }
+
+        protected void setLedgerId(long ledgerId) {
+            this.ledgerId = ledgerId;
+        }
+    }
+
+    protected static class RecoverLedgerCommand extends PerLedgerCommand {
+
+        RecoverLedgerCommand() {
+            super("recoverledger", "force recover ledger");
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            LedgerHandle lh = getBookKeeperClient().get().openLedgerNoRecovery(
+                    getLedgerID(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes(UTF_8));
+            final CountDownLatch doneLatch = new CountDownLatch(1);
+            final AtomicInteger resultHolder = new AtomicInteger(-1234);
+            BookkeeperInternalCallbacks.GenericCallback<Void> recoverCb =
+                    new BookkeeperInternalCallbacks.GenericCallback<Void>() {
+                @Override
+                public void operationComplete(int rc, Void result) {
+                    resultHolder.set(rc);
+                    doneLatch.countDown();
+                }
+            };
+            try {
+                BookKeeperAccessor.forceRecoverLedger(lh, recoverCb);
+                doneLatch.await();
+                if (BKException.Code.OK != resultHolder.get()) {
+                    throw BKException.create(resultHolder.get());
+                }
+            } finally {
+                lh.close();
+            }
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "recoverledger [options]";
+        }
+    }
+
+    protected static class FindLedgerCommand extends PerLedgerCommand {
+
+        FindLedgerCommand() {
+            super("findledger", "find the stream for a given ledger");
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            Iterator<String> logs = getNamespace().getLogs();
+            while (logs.hasNext()) {
+                String logName = logs.next();
+                if (processLog(logName)) {
+                    System.out.println("Found ledger " + getLedgerID() + " at log stream '" + logName + "'");
+                }
+            }
+            return 0;
+        }
+
+        boolean processLog(String logName) throws Exception {
+            DistributedLogManager dlm = getNamespace().openLog(logName);
+            try {
+                List<LogSegmentMetadata> segments = dlm.getLogSegments();
+                for (LogSegmentMetadata segment : segments) {
+                    if (getLedgerID() == segment.getLogSegmentId()) {
+                        System.out.println("Found ledger " + getLedgerID() + " at log segment "
+                                + segment + " for stream '" + logName + "'");
+                        return true;
+                    }
+                }
+                return false;
+            } finally {
+                dlm.close();
+            }
+        }
+    }
+
+    protected static class ReadLastConfirmedCommand extends PerLedgerCommand {
+
+        ReadLastConfirmedCommand() {
+            super("readlac", "read last add confirmed for a given ledger");
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            LedgerHandle lh = getBookKeeperClient().get().openLedgerNoRecovery(
+                    getLedgerID(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes(UTF_8));
+            try {
+                long lac = lh.readLastConfirmed();
+                System.out.println("LastAddConfirmed: " + lac);
+            } finally {
+                lh.close();
+            }
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "readlac [options]";
+        }
+    }
+
+    protected static class ReadEntriesCommand extends PerLedgerCommand {
+
+        Long fromEntryId;
+        Long untilEntryId;
+        boolean printHex = false;
+        boolean skipPayload = false;
+        boolean readAllBookies = false;
+        boolean readLac = false;
+        boolean corruptOnly = false;
+
+        int metadataVersion = LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION;
+
+        ReadEntriesCommand() {
+            super("readentries", "read entries for a given ledger");
+            options.addOption("x", "hex", false, "Print record in hex format");
+            options.addOption("sp", "skip-payload", false, "Skip printing the payload of the record");
+            options.addOption("fid", "from", true, "Entry id to start reading");
+            options.addOption("uid", "until", true, "Entry id to read until");
+            options.addOption("bks", "all-bookies", false, "Read entry from all bookies");
+            options.addOption("lac", "last-add-confirmed", false, "Return last add confirmed rather than entry payload");
+            options.addOption("ver", "metadata-version", true, "The log segment metadata version to use");
+            options.addOption("bad", "corrupt-only", false, "Display info for corrupt entries only");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            printHex = cmdline.hasOption("x");
+            skipPayload = cmdline.hasOption("sp");
+            if (cmdline.hasOption("fid")) {
+                fromEntryId = Long.parseLong(cmdline.getOptionValue("fid"));
+            }
+            if (cmdline.hasOption("uid")) {
+                untilEntryId = Long.parseLong(cmdline.getOptionValue("uid"));
+            }
+            if (cmdline.hasOption("ver")) {
+                metadataVersion = Integer.parseInt(cmdline.getOptionValue("ver"));
+            }
+            corruptOnly = cmdline.hasOption("bad");
+            readAllBookies = cmdline.hasOption("bks");
+            readLac = cmdline.hasOption("lac");
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            LedgerHandle lh = getBookKeeperClient().get().openLedgerNoRecovery(getLedgerID(), BookKeeper.DigestType.CRC32,
+                    dlConf.getBKDigestPW().getBytes(UTF_8));
+            try {
+                if (null == fromEntryId) {
+                    fromEntryId = 0L;
+                }
+                if (null == untilEntryId) {
+                    untilEntryId = lh.readLastConfirmed();
+                }
+                if (untilEntryId >= fromEntryId) {
+                    if (readAllBookies) {
+                        LedgerReader lr = new LedgerReader(getBookKeeperClient().get());
+                        if (readLac) {
+                            readLacsFromAllBookies(lr, lh, fromEntryId, untilEntryId);
+                        } else {
+                            readEntriesFromAllBookies(lr, lh, fromEntryId, untilEntryId);
+                        }
+                    } else {
+                        simpleReadEntries(lh, fromEntryId, untilEntryId);
+                    }
+                } else {
+                    System.out.println("No entries.");
+                }
+            } finally {
+                lh.close();
+            }
+            return 0;
+        }
+
+        private void readEntriesFromAllBookies(LedgerReader ledgerReader, LedgerHandle lh, long fromEntryId, long untilEntryId)
+                throws Exception {
+            for (long eid = fromEntryId; eid <= untilEntryId; ++eid) {
+                final CountDownLatch doneLatch = new CountDownLatch(1);
+                final AtomicReference<Set<LedgerReader.ReadResult<InputStream>>> resultHolder =
+                        new AtomicReference<Set<LedgerReader.ReadResult<InputStream>>>();
+                ledgerReader.readEntriesFromAllBookies(lh, eid, new BookkeeperInternalCallbacks.GenericCallback<Set<LedgerReader.ReadResult<InputStream>>>() {
+                    @Override
+                    public void operationComplete(int rc, Set<LedgerReader.ReadResult<InputStream>> readResults) {
+                        if (BKException.Code.OK == rc) {
+                            resultHolder.set(readResults);
+                        } else {
+                            resultHolder.set(null);
+                        }
+                        doneLatch.countDown();
+                    }
+                });
+                doneLatch.await();
+                Set<LedgerReader.ReadResult<InputStream>> readResults = resultHolder.get();
+                if (null == readResults) {
+                    throw new IOException("Failed to read entry " + eid);
+                }
+                boolean printHeader = true;
+                for (LedgerReader.ReadResult<InputStream> rr : readResults) {
+                    if (corruptOnly) {
+                        if (BKException.Code.DigestMatchException == rr.getResultCode()) {
+                            if (printHeader) {
+                                System.out.println("\t" + eid + "\t:");
+                                printHeader = false;
+                            }
+                            System.out.println("\tbookie=" + rr.getBookieAddress());
+                            System.out.println("\t-------------------------------");
+                            System.out.println("status = " + BKException.getMessage(rr.getResultCode()));
+                            System.out.println("\t-------------------------------");
+                        }
+                    } else {
+                        if (printHeader) {
+                            System.out.println("\t" + eid + "\t:");
+                            printHeader = false;
+                        }
+                        System.out.println("\tbookie=" + rr.getBookieAddress());
+                        System.out.println("\t-------------------------------");
+                        if (BKException.Code.OK == rr.getResultCode()) {
+                            Entry.Reader reader = Entry.newBuilder()
+                                    .setLogSegmentInfo(lh.getId(), 0L)
+                                    .setEntryId(eid)
+                                    .setInputStream(rr.getValue())
+                                    .setEnvelopeEntry(LogSegmentMetadata.supportsEnvelopedEntries(metadataVersion))
+                                    .buildReader();
+                            printEntry(reader);
+                        } else {
+                            System.out.println("status = " + BKException.getMessage(rr.getResultCode()));
+                        }
+                        System.out.println("\t-------------------------------");
+                    }
+                }
+            }
+        }
+
+        private void readLacsFromAllBookies(LedgerReader ledgerReader, LedgerHandle lh, long fromEntryId, long untilEntryId)
+                throws Exception {
+            for (long eid = fromEntryId; eid <= untilEntryId; ++eid) {
+                final CountDownLatch doneLatch = new CountDownLatch(1);
+                final AtomicReference<Set<LedgerReader.ReadResult<Long>>> resultHolder =
+                        new AtomicReference<Set<LedgerReader.ReadResult<Long>>>();
+                ledgerReader.readLacs(lh, eid, new BookkeeperInternalCallbacks.GenericCallback<Set<LedgerReader.ReadResult<Long>>>() {
+                    @Override
+                    public void operationComplete(int rc, Set<LedgerReader.ReadResult<Long>> readResults) {
+                        if (BKException.Code.OK == rc) {
+                            resultHolder.set(readResults);
+                        } else {
+                            resultHolder.set(null);
+                        }
+                        doneLatch.countDown();
+                    }
+                });
+                doneLatch.await();
+                Set<LedgerReader.ReadResult<Long>> readResults = resultHolder.get();
+                if (null == readResults) {
+                    throw new IOException("Failed to read entry " + eid);
+                }
+                System.out.println("\t" + eid + "\t:");
+                for (LedgerReader.ReadResult<Long> rr : readResults) {
+                    System.out.println("\tbookie=" + rr.getBookieAddress());
+                    System.out.println("\t-------------------------------");
+                    if (BKException.Code.OK == rr.getResultCode()) {
+                        System.out.println("Eid = " + rr.getEntryId() + ", Lac = " + rr.getValue());
+                    } else {
+                        System.out.println("status = " + BKException.getMessage(rr.getResultCode()));
+                    }
+                    System.out.println("\t-------------------------------");
+                }
+            }
+        }
+
+        private void simpleReadEntries(LedgerHandle lh, long fromEntryId, long untilEntryId) throws Exception {
+            Enumeration<LedgerEntry> entries = lh.readEntries(fromEntryId, untilEntryId);
+            long i = fromEntryId;
+            System.out.println("Entries:");
+            while (entries.hasMoreElements()) {
+                LedgerEntry entry = entries.nextElement();
+                System.out.println("\t" + i  + "(eid=" + entry.getEntryId() + ")\t: ");
+                Entry.Reader reader = Entry.newBuilder()
+                        .setLogSegmentInfo(0L, 0L)
+                        .setEntryId(entry.getEntryId())
+                        .setInputStream(entry.getEntryInputStream())
+                        .setEnvelopeEntry(LogSegmentMetadata.supportsEnvelopedEntries(metadataVersion))
+                        .buildReader();
+                printEntry(reader);
+                ++i;
+            }
+        }
+
+        private void printEntry(Entry.Reader reader) throws Exception {
+            LogRecordWithDLSN record = reader.nextRecord();
+            while (null != record) {
+                System.out.println("\t" + record);
+                if (!skipPayload) {
+                    if (printHex) {
+                        System.out.println(Hex.encodeHexString(record.getPayload()));
+                    } else {
+                        System.out.println(new String(record.getPayload(), UTF_8));
+                    }
+                }
+                System.out.println("");
+                record = reader.nextRecord();
+            }
+        }
+
+        @Override
+        protected String getUsage() {
+            return "readentries [options]";
+        }
+    }
+
+    protected static abstract class AuditCommand extends OptsCommand {
+
+        protected final Options options = new Options();
+        protected final DistributedLogConfiguration dlConf;
+        protected final List<URI> uris = new ArrayList<URI>();
+        protected String zkAclId = null;
+        protected boolean force = false;
+
+        protected AuditCommand(String name, String description) {
+            super(name, description);
+            dlConf = new DistributedLogConfiguration();
+            options.addOption("u", "uris", true, "List of distributedlog uris, separated by comma");
+            options.addOption("c", "conf", true, "DistributedLog Configuration File");
+            options.addOption("a", "zk-acl-id", true, "ZooKeeper ACL ID");
+            options.addOption("f", "force", false, "Force command (no warnings or prompts)");
+        }
+
+        @Override
+        protected int runCmd(CommandLine commandLine) throws Exception {
+            try {
+                parseCommandLine(commandLine);
+            } catch (ParseException pe) {
+                System.err.println("ERROR: failed to parse commandline : '" + pe.getMessage() + "'");
+                printUsage();
+                return -1;
+            }
+            return runCmd();
+        }
+
+        protected abstract int runCmd() throws Exception;
+
+        @Override
+        protected Options getOptions() {
+            return options;
+        }
+
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            if (!cmdline.hasOption("u")) {
+                throw new ParseException("No distributedlog uri provided.");
+            }
+            String urisStr = cmdline.getOptionValue("u");
+            for (String uriStr : urisStr.split(",")) {
+                uris.add(URI.create(uriStr));
+            }
+            if (cmdline.hasOption("c")) {
+                String configFile = cmdline.getOptionValue("c");
+                try {
+                    dlConf.loadConf(new File(configFile).toURI().toURL());
+                } catch (ConfigurationException e) {
+                    throw new ParseException("Failed to load distributedlog configuration from " + configFile + ".");
+                } catch (MalformedURLException e) {
+                    throw new ParseException("Failed to load distributedlog configuration from malformed "
+                            + configFile + ".");
+                }
+            }
+            if (cmdline.hasOption("a")) {
+                zkAclId = cmdline.getOptionValue("a");
+            }
+            if (cmdline.hasOption("f")) {
+                force = true;
+            }
+        }
+
+        protected DistributedLogConfiguration getConf() {
+            return dlConf;
+        }
+
+        protected List<URI> getUris() {
+            return uris;
+        }
+
+        protected String getZkAclId() {
+            return zkAclId;
+        }
+
+        protected boolean getForce() {
+            return force;
+        }
+
+    }
+
+    static class AuditLedgersCommand extends AuditCommand {
+
+        String ledgersFilePrefix;
+        final List<List<String>> allocationPaths =
+                new ArrayList<List<String>>();
+
+        AuditLedgersCommand() {
+            super("audit_ledgers", "Audit ledgers between bookkeeper and DL uris");
+            options.addOption("lf", "ledgers-file", true, "Prefix of filename to store ledgers");
+            options.addOption("ap", "allocation-paths", true, "Allocation paths per uri. E.g ap10;ap11,ap20");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (cmdline.hasOption("lf")) {
+                ledgersFilePrefix = cmdline.getOptionValue("lf");
+            } else {
+                throw new ParseException("No file specified to store leak ledgers");
+            }
+            if (cmdline.hasOption("ap")) {
+                String[] aps = cmdline.getOptionValue("ap").split(",");
+                for(String ap : aps) {
+                    List<String> list = new ArrayList<String>();
+                    String[] array = ap.split(";");
+                    Collections.addAll(list, array);
+                    allocationPaths.add(list);
+                }
+            } else {
+                throw new ParseException("No allocation paths provided.");
+            }
+        }
+
+        void dumpLedgers(Set<Long> ledgers, File targetFile) throws Exception {
+            PrintWriter pw = new PrintWriter(new OutputStreamWriter(new FileOutputStream(targetFile), UTF_8.name()));
+            try {
+                for (Long ledger : ledgers) {
+                    pw.println(ledger);
+                }
+            } finally {
+                pw.close();
+            }
+            System.out.println("Dump " + ledgers.size() + " ledgers to file : " + targetFile);
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            if (!getForce() && !IOUtils.confirmPrompt("Do you want to audit uris : "
+                    + getUris() + ", allocation paths = " + allocationPaths)) {
+                return 0;
+            }
+
+            DLAuditor dlAuditor = new DLAuditor(getConf());
+            try {
+                Pair<Set<Long>, Set<Long>> bkdlLedgers = dlAuditor.collectLedgers(getUris(), allocationPaths);
+                dumpLedgers(bkdlLedgers.getLeft(), new File(ledgersFilePrefix + "-bkledgers.txt"));
+                dumpLedgers(bkdlLedgers.getRight(), new File(ledgersFilePrefix + "-dlledgers.txt"));
+                dumpLedgers(Sets.difference(bkdlLedgers.getLeft(), bkdlLedgers.getRight()),
+                            new File(ledgersFilePrefix + "-leakledgers.txt"));
+            } finally {
+                dlAuditor.close();
+            }
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "audit_ledgers [options]";
+        }
+    }
+
+    public static class AuditDLSpaceCommand extends PerDLCommand {
+
+        private String regex = null;
+
+        AuditDLSpaceCommand() {
+            super("audit_dl_space", "Audit stream space usage for a given dl uri");
+            options.addOption("groupByRegex", true, "Group by the result of applying the regex to stream name");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (cmdline.hasOption("groupByRegex")) {
+                regex = cmdline.getOptionValue("groupByRegex");
+            }
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            DLAuditor dlAuditor = new DLAuditor(getConf());
+            try {
+                Map<String, Long> streamSpaceMap = dlAuditor.calculateStreamSpaceUsage(getUri());
+                if (null != regex) {
+                    printGroupByRegexSpaceUsage(streamSpaceMap, regex);
+                } else {
+                    printSpaceUsage(streamSpaceMap);
+                }
+            } finally {
+                dlAuditor.close();
+            }
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "audit_dl_space [options]";
+        }
+
+        private void printSpaceUsage(Map<String, Long> spaceMap) throws Exception {
+            for (Map.Entry<String, Long> entry : spaceMap.entrySet()) {
+                System.out.println(entry.getKey() + "\t" + entry.getValue());
+            }
+        }
+
+        private void printGroupByRegexSpaceUsage(Map<String, Long> streamSpaceMap, String regex) throws Exception {
+            Pattern pattern = Pattern.compile(regex);
+            Map<String, Long> groupedUsageMap = new HashMap<String, Long>();
+            for (Map.Entry<String, Long> entry : streamSpaceMap.entrySet()) {
+                Matcher matcher = pattern.matcher(entry.getKey());
+                String key = entry.getKey();
+                boolean matches = matcher.matches();
+                if (matches) {
+                    key = matcher.group(1);
+                }
+                Long value = entry.getValue();
+                if (groupedUsageMap.containsKey(key)) {
+                    value += groupedUsageMap.get(key);
+                }
+                groupedUsageMap.put(key, value);
+            }
+            printSpaceUsage(groupedUsageMap);
+        }
+    }
+
+    public static class AuditBKSpaceCommand extends PerDLCommand {
+
+        AuditBKSpaceCommand() {
+            super("audit_bk_space", "Audit bk space usage for a given dl uri");
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            DLAuditor dlAuditor = new DLAuditor(getConf());
+            try {
+                long spaceUsage = dlAuditor.calculateLedgerSpaceUsage(uri);
+                System.out.println("bookkeeper ledgers space usage \t " + spaceUsage);
+            } finally {
+                dlAuditor.close();
+            }
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "audit_bk_space [options]";
+        }
+    }
+
+    protected static class TruncateStreamCommand extends PerStreamCommand {
+
+        DLSN dlsn = DLSN.InvalidDLSN;
+
+        TruncateStreamCommand() {
+            super("truncate_stream", "truncate a stream at a specific position");
+            options.addOption("dlsn", true, "Truncate all records older than this dlsn");
+        }
+
+        public void setDlsn(DLSN dlsn) {
+            this.dlsn = dlsn;
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (cmdline.hasOption("dlsn")) {
+                dlsn = parseDLSN(cmdline.getOptionValue("dlsn"));
+            }
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            getConf().setZkAclId(getZkAclId());
+            return truncateStream(getNamespace(), getStreamName(), dlsn);
+        }
+
+        private int truncateStream(final DistributedLogNamespace 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 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))) {
+                            System.out.println("Failed to truncate.");
+                        }
+                        return 0;
+                    } finally {
+                        Utils.close(writer);
+                    }
+                }
+            } catch (Exception ex) {
+                System.err.println("Failed to truncate " + ex);
+                return 1;
+            } finally {
+                dlm.close();
+            }
+        }
+    }
+
+    public static class DeserializeDLSNCommand extends SimpleCommand {
+
+        String base64Dlsn = "";
+
+        DeserializeDLSNCommand() {
+            super("deserialize_dlsn", "Deserialize DLSN");
+            options.addOption("b64", "base64", true, "Base64 encoded dlsn");
+        }
+
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            if (cmdline.hasOption("b64")) {
+                base64Dlsn = cmdline.getOptionValue("b64");
+            } else {
+                throw new IllegalArgumentException("Argument b64 is required");
+            }
+        }
+
+        @Override
+        protected int runSimpleCmd() throws Exception {
+            System.out.println(DLSN.deserialize(base64Dlsn).toString());
+            return 0;
+        }
+    }
+
+    public static class SerializeDLSNCommand extends SimpleCommand {
+
+        private DLSN dlsn = DLSN.InitialDLSN;
+        private boolean hex = false;
+
+        SerializeDLSNCommand() {
+            super("serialize_dlsn", "Serialize DLSN. Default format is base64 string.");
+            options.addOption("dlsn", true, "DLSN in comma separated format to serialize");
+            options.addOption("x", "hex", false, "Emit hex-encoded string DLSN instead of base 64");
+        }
+
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            if (cmdline.hasOption("dlsn")) {
+                dlsn = parseDLSN(cmdline.getOptionValue("dlsn"));
+            }
+            hex = cmdline.hasOption("x");
+        }
+
+        @Override
+        protected int runSimpleCmd() throws Exception {
+            if (hex) {
+                byte[] bytes = dlsn.serializeBytes();
+                String hexString = Hex.encodeHexString(bytes);
+                System.out.println(hexString);
+            } else {
+                System.out.println(dlsn.serialize());
+            }
+            return 0;
+        }
+    }
+
+    public static class DeleteSubscriberCommand extends PerDLCommand {
+
+        int numThreads = 1;
+        String streamPrefix = null;
+        String subscriberId = null;
+
+        DeleteSubscriberCommand() {
+            super("delete_subscriber", "Delete the subscriber in subscription store. ");
+            options.addOption("s", "subscriberId", true, "SubscriberId to remove from the stream");
+            options.addOption("t", "threads", true, "Number of threads");
+            options.addOption("ft", "filter", true, "Stream filter by prefix");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("s")) {
+                throw new ParseException("No subscriberId provided.");
+            } else {
+                subscriberId = cmdline.getOptionValue("s");
+            }
+            if (cmdline.hasOption("t")) {
+                numThreads = Integer.parseInt(cmdline.getOptionValue("t"));
+            }
+            if (cmdline.hasOption("ft")) {
+                streamPrefix = cmdline.getOptionValue("ft");
+            }
+        }
+
+        @Override
+        protected String getUsage() {
+            return "delete_subscriber [options]";
+        }
+
+        @Override
+        protected int runCmd() throws Exception {
+            getConf().setZkAclId(getZkAclId());
+            return deleteSubscriber(getNamespace());
+        }
+
+        private int deleteSubscriber(final DistributedLogNamespace namespace) throws Exception {
+            Iterator<String> streamCollection = namespace.getLogs();
+            final List<String> streams = new ArrayList<String>();
+            while (streamCollection.hasNext()) {
+                String s = streamCollection.next();
+                if (null != streamPrefix) {
+                    if (s.startsWith(streamPrefix)) {
+                        streams.add(s);
+                    }
+                } else {
+                    streams.add(s);
+                }
+            }
+            if (0 == streams.size()) {
+                return 0;
+            }
+            System.out.println("Streams : " + streams);
+            if (!getForce() && !IOUtils.confirmPrompt("Do you want to delete subscriber "
+                + subscriberId + " for " + streams.size() + " streams ?")) {
+                return 0;
+            }
+            numThreads = Math.min(streams.size(), numThreads);
+            final int numStreamsPerThreads = streams.size() / numThreads + 1;
+            Thread[] threads = new Thread[numThreads];
+            for (int i = 0; i < numThreads; i++) {
+                final int tid = i;
+                threads[i] = new Thread("RemoveSubscriberThread-" + i) {
+                    @Override
+                    public void run() {
+                        try {
+                            deleteSubscriber(namespace, streams, tid, numStreamsPerThreads);
+                            System.out.println("Thread " + tid + " finished.");
+                        } catch (Exception e) {
+                            System.err.println("Thread " + tid + " quits with exception : " + e.getMessage());
+                        }
+                    }
+                };
+                threads[i].start();
+            }
+            for (int i = 0; i < numThreads; i++) {
+                threads[i].join();
+            }
+            return 0;
+        }
+
+        private void deleteSubscriber(DistributedLogNamespace namespace, List<String> streams,
+                                      int tid, int numStreamsPerThreads) throws Exception {
+            int startIdx = tid * numStreamsPerThreads;
+            int endIdx = Math.min(streams.size(), (tid + 1) * numStreamsPerThreads);
+            for (int i = startIdx; i < endIdx; i++) {
+                final String s = streams.get(i);
+                DistributedLogManager dlm = namespace.openLog(s);
+                final CountDownLatch countDownLatch = new CountDownLatch(1);
+                dlm.getSubscriptionsStore().deleteSubscriber(subscriberId)
+                    .addEventListener(new FutureEventListener<Boolean>() {
+                        @Override
+                        public void onFailure(Throwable cause) {
+                            System.out.println("Failed to delete subscriber for stream " + s);
+                            cause.printStackTrace();
+                            countDownLatch.countDown();
+                        }
+
+                        @Override
+                        public void onSuccess(Boolean value) {
+                            countDownLatch.countDown();
+                        }
+                    });
+                countDownLatch.await();
+                dlm.close();
+            }
+        }
+    }
+
+    public DistributedLogTool() {
+        super();
+        addCommand(new AuditBKSpaceCommand());
+        addCommand(new AuditLedgersCommand());
+        addCommand(new AuditDLSpaceCommand());
+        addCommand(new CreateCommand());
+        addCommand(new CountCommand());
+        addCommand(new DeleteCommand());
+        addCommand(new DeleteAllocatorPoolCommand());
+        addCommand(new DeleteLedgersCommand());
+        addCommand(new DumpCommand());
+        addCommand(new FindLedgerCommand());
+        addCommand(new InspectCommand());
+        addCommand(new InspectStreamCommand());
+        addCommand(new ListCommand());
+        addCommand(new ReadLastConfirmedCommand());
+        addCommand(new ReadEntriesCommand());
+        addCommand(new RecoverCommand());
+        addCommand(new RecoverLedgerCommand());
+        addCommand(new ShowCommand());
+        addCommand(new TruncateCommand());
+        addCommand(new TruncateStreamCommand());
+        addCommand(new DeserializeDLSNCommand());
+        addCommand(new SerializeDLSNCommand());
+        addCommand(new WatchNamespaceCommand());
+        addCommand(new DeleteSubscriberCommand());
+    }
+
+    @Override
+    protected String getName() {
+        return "dlog_tool";
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/tools/Tool.java b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/Tool.java
new file mode 100644
index 0000000..9045fa2
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/Tool.java
@@ -0,0 +1,243 @@
+/**
+ * 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.tools;
+
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.cli.BasicParser;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+
+import java.util.Map;
+import java.util.TreeMap;
+
+/**
+ * A Tool Framework
+ */
+public abstract class Tool {
+
+    /**
+     * Interface of a command to run in a tool.
+     */
+    protected interface Command {
+        String getName();
+        String getDescription();
+        int runCmd(String[] args) throws Exception;
+        void printUsage();
+    }
+
+    /**
+     * {@link org.apache.commons.cli.Options} based command.
+     */
+    protected abstract static class OptsCommand implements Command {
+
+        /**
+         * @return options used by this command.
+         */
+        protected abstract Options getOptions();
+
+        /**
+         * @return usage of this command.
+         */
+        protected String getUsage() {
+            return cmdName + " [options]";
+        }
+
+        /**
+         * Run given command line <i>commandLine</i>.
+         *
+         * @param commandLine
+         *          command line to run.
+         * @return return code of this command.
+         * @throws Exception
+         */
+        protected abstract int runCmd(CommandLine commandLine) throws Exception;
+
+        protected String cmdName;
+        protected String description;
+
+        protected OptsCommand(String name, String description) {
+            this.cmdName = name;
+            this.description = description;
+        }
+
+        @Override
+        public String getName() {
+            return cmdName;
+        }
+
+        @Override
+        public String getDescription() {
+            return description;
+        }
+
+        @Override
+        public int runCmd(String[] args) throws Exception {
+            try {
+                BasicParser parser = new BasicParser();
+                CommandLine cmdline = parser.parse(getOptions(), args);
+                return runCmd(cmdline);
+            } catch (ParseException e) {
+                printUsage();
+                return -1;
+            }
+        }
+
+        @Override
+        public void printUsage() {
+            HelpFormatter helpFormatter = new HelpFormatter();
+            println(cmdName + ": " + getDescription());
+            helpFormatter.printHelp(getUsage(), getOptions());
+        }
+    }
+
+    public class HelpCommand implements Command {
+
+        @Override
+        public String getName() {
+            return "help";
+        }
+
+        @Override
+        public String getDescription() {
+            return "describe the usage of this tool or its sub-commands.";
+        }
+
+        @Override
+        public int runCmd(String[] args) throws Exception {
+            if (args.length == 0) {
+                printToolUsage();
+                return -1;
+            }
+            String cmdName = args[0];
+            Command command = commands.get(cmdName);
+            if (null == command) {
+                System.err.println("Unknown command " + cmdName);
+                printToolUsage();
+                return -1;
+            }
+            command.printUsage();
+            println("");
+            return 0;
+        }
+
+        @Override
+        public void printUsage() {
+            println(getName() + ": " + getDescription());
+            println("");
+            println("usage: " + getName() + " <command>");
+        }
+    }
+
+    // Commands managed by a tool
+    protected final Map<String, Command> commands =
+            new TreeMap<String, Command>();
+
+    protected Tool() {
+        addCommand(new HelpCommand());
+    }
+
+    /**
+     * @return tool name.
+     */
+    protected abstract String getName();
+
+    /**
+     * Add a command in this tool.
+     *
+     * @param command
+     *          command to run in this tool.
+     */
+    protected void addCommand(Command command) {
+        commands.put(command.getName(), command);
+    }
+
+    /**
+     * Print a message in this tool.
+     *
+     * @param msg
+     *          message to print
+     */
+    protected static void println(String msg) {
+        System.out.println(msg);
+    }
+
+    /**
+     * print tool usage.
+     */
+    protected void printToolUsage() {
+        println("Usage: " + getName() + " <command>");
+        println("");
+        int maxKeyLength = 0;
+        for (String key : commands.keySet()) {
+            if (key.length() > maxKeyLength) {
+                maxKeyLength = key.length();
+            }
+        }
+        maxKeyLength += 2;
+        for (Map.Entry<String, Command> entry : commands.entrySet()) {
+            StringBuilder spacesBuilder = new StringBuilder();
+            int numSpaces = maxKeyLength - entry.getKey().length();
+            for (int i = 0; i < numSpaces; i++) {
+                spacesBuilder.append(" ");
+            }
+            println("\t"  + entry.getKey() + spacesBuilder.toString() + ": " + entry.getValue().getDescription());
+        }
+        println("");
+    }
+
+    public int run(String[] args) throws Exception {
+        if (args.length <= 0) {
+            printToolUsage();
+            return -1;
+        }
+        String cmdName = args[0];
+        Command cmd = commands.get(cmdName);
+        if (null == cmd) {
+            System.err.println("ERROR: Unknown command " + cmdName);
+            printToolUsage();
+            return -1;
+        }
+        // prepare new args
+        String[] newArgs = new String[args.length - 1];
+        System.arraycopy(args, 1, newArgs, 0, newArgs.length);
+        return cmd.runCmd(newArgs);
+    }
+
+    public static void main(String args[]) {
+        int rc = -1;
+        if (args.length <= 0) {
+            System.err.println("No tool to run.");
+            System.err.println("");
+            System.err.println("Usage : Tool <tool_class_name> <options>");
+            System.exit(-1);
+        }
+        String toolClass = args[0];
+        try {
+            Tool tool = ReflectionUtils.newInstance(toolClass, Tool.class);
+            String[] newArgs = new String[args.length - 1];
+            System.arraycopy(args, 1, newArgs, 0, newArgs.length);
+            rc = tool.run(newArgs);
+        } catch (Throwable t) {
+            System.err.println("Fail to run tool " + toolClass + " : ");
+            t.printStackTrace();
+        }
+        System.exit(rc);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/tools/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/package-info.java
new file mode 100644
index 0000000..b410188
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/tools/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Tools for distributedlog
+ */
+package org.apache.distributedlog.tools;
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
new file mode 100644
index 0000000..c0da29a
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Allocator.java
@@ -0,0 +1,102 @@
+/**
+ * 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.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>.
+ *
+ * <h3>Usage Example</h3>
+ *
+ * Here is an example on demonstrating how `Allocator` works.
+ *
+ * <pre> {@code
+ * Allocator<I, T, R> allocator = ...;
+ *
+ * // issue an allocate request
+ * try {
+ *   allocator.allocate();
+ * } catch (IOException ioe) {
+ *   // handle the exception
+ *   ...
+ *   return;
+ * }
+ *
+ * // Start a transaction
+ * final Transaction<T> txn = ...;
+ *
+ * // Try obtain object I
+ * Future<I> tryObtainFuture = allocator.tryObtain(txn, new OpListener<I>() {
+ *     public void onCommit(I resource) {
+ *         // the obtain succeed, process with the resource
+ *     }
+ *     public void onAbort() {
+ *         // the obtain failed.
+ *     }
+ * }).addFutureEventListener(new FutureEventListener() {
+ *     public void onSuccess(I resource) {
+ *         // the try obtain succeed. but the obtain has not been confirmed or aborted.
+ *         // execute the transaction to confirm if it could complete obtain
+ *         txn.execute();
+ *     }
+ *     public void onFailure(Throwable t) {
+ *         // handle the failure of try obtain
+ *     }
+ * });
+ *
+ * }</pre>
+ */
+public interface Allocator<I, T> extends AsyncCloseable, AsyncDeleteable {
+
+    /**
+     * Issue allocation request to allocate <i>I</i>.
+     * The implementation should be non-blocking call.
+     *
+     * @throws IOException
+     *          if fail to request allocating a <i>I</i>.
+     */
+    void allocate() throws IOException;
+
+    /**
+     * Try obtaining an <i>I</i> in a given transaction <i>T</i>. The object obtained is tentative.
+     * Whether the object is obtained or aborted is determined by the result of the execution. You could
+     * register a listener under this `tryObtain` operation to know whether the object is obtained or
+     * aborted.
+     *
+     * <p>
+     * It is a typical two-phases operation on obtaining a resource from allocator.
+     * The future returned by this method acts as a `prepare` operation, the resource is tentative obtained
+     * from the allocator. The execution of the txn acts as a `commit` operation, the resource is confirmed
+     * to be obtained by this transaction. <code>listener</code> is for the whole completion of the obtain.
+     * <p>
+     * <code>listener</code> is only triggered after `prepare` succeed. if `prepare` failed, no actions will
+     * happen to the listener.
+     *
+     * @param txn
+     *          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);
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/CommandLineUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/CommandLineUtils.java
new file mode 100644
index 0000000..5113e54
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/CommandLineUtils.java
@@ -0,0 +1,56 @@
+/**
+ * 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.Optional;
+import org.apache.commons.cli.CommandLine;
+
+/**
+ * Utils to commandline
+ */
+public class CommandLineUtils {
+
+    public static Optional<String> getOptionalStringArg(CommandLine cmdline, String arg) {
+        if (cmdline.hasOption(arg)) {
+            return Optional.of(cmdline.getOptionValue(arg));
+        } else {
+            return Optional.absent();
+        }
+    }
+
+    public static Optional<Boolean> getOptionalBooleanArg(CommandLine cmdline, String arg) {
+        if (cmdline.hasOption(arg)) {
+            return Optional.of(true);
+        } else {
+            return Optional.absent();
+        }
+    }
+
+    public static Optional<Integer> getOptionalIntegerArg(CommandLine cmdline, String arg) throws IllegalArgumentException {
+        try {
+            if (cmdline.hasOption(arg)) {
+                return Optional.of(Integer.parseInt(cmdline.getOptionValue(arg)));
+            } else {
+                return Optional.absent();
+            }
+        } catch (NumberFormatException ex) {
+            throw new IllegalArgumentException(arg + " is not a number");
+        }
+    }
+
+}
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
new file mode 100644
index 0000000..41c2be7
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/ConfUtils.java
@@ -0,0 +1,65 @@
+/**
+ * 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.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.config.ConcurrentConstConfiguration;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.commons.configuration.Configuration;
+
+import java.util.Iterator;
+
+public class ConfUtils {
+
+    /**
+     * Load configurations with prefixed <i>section</i> from source configuration <i>srcConf</i> into
+     * target configuration <i>targetConf</i>.
+     *
+     * @param targetConf
+     *          Target Configuration
+     * @param srcConf
+     *          Source Configuration
+     * @param section
+     *          Section Key
+     */
+    public static void loadConfiguration(Configuration targetConf, Configuration srcConf, String section) {
+        Iterator confKeys = srcConf.getKeys();
+        while (confKeys.hasNext()) {
+            Object keyObject = confKeys.next();
+            if (!(keyObject instanceof String)) {
+                continue;
+            }
+            String key = (String) keyObject;
+            if (key.startsWith(section)) {
+                targetConf.setProperty(key.substring(section.length()), srcConf.getProperty(key));
+            }
+        }
+    }
+
+    /**
+     * Create const dynamic configuration based on distributedlog configuration.
+     *
+     * @param conf
+     *          static distributedlog configuration.
+     * @return dynamic configuration
+     */
+    public static DynamicDistributedLogConfiguration getConstDynConf(DistributedLogConfiguration conf) {
+        ConcurrentConstConfiguration constConf = new ConcurrentConstConfiguration(conf);
+        return new DynamicDistributedLogConfiguration(constConf);
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/DLUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/DLUtils.java
new file mode 100644
index 0000000..7b7e0f7
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/DLUtils.java
@@ -0,0 +1,321 @@
+/**
+ * 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 org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.exceptions.InvalidStreamNameException;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.commons.lang.StringUtils;
+
+import java.net.InetAddress;
+import java.net.URI;
+import java.net.URISyntaxException;
+import java.util.List;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+/**
+ * Utilities about DL implementations like uri, log segments, metadata serialization and deserialization.
+ */
+public class DLUtils {
+
+    /**
+     * Find the log segment whose transaction ids are not less than provided <code>transactionId</code>.
+     *
+     * @param segments
+     *          segments to search
+     * @param transactionId
+     *          transaction id to find
+     * @return the first log segment whose transaction ids are not less than <code>transactionId</code>.
+     */
+    public static int findLogSegmentNotLessThanTxnId(List<LogSegmentMetadata> segments,
+                                                     long transactionId) {
+        int found = -1;
+        for (int i = segments.size() - 1; i >= 0; i--) {
+            LogSegmentMetadata segment = segments.get(i);
+            if (segment.getFirstTxId() <= transactionId) {
+                found = i;
+                break;
+            }
+        }
+        if (found <= -1) {
+            return -1;
+        }
+        if (found == 0 && segments.get(0).getFirstTxId() == transactionId) {
+            return 0;
+        }
+        LogSegmentMetadata foundSegment = segments.get(found);
+        if (foundSegment.getFirstTxId() == transactionId) {
+            for (int i = found - 1; i >= 0; i--) {
+                LogSegmentMetadata segment = segments.get(i);
+                if (segment.isInProgress()) {
+                    break;
+                }
+                if (segment.getLastTxId() < transactionId) {
+                    break;
+                }
+                found = i;
+            }
+            return found;
+        } else {
+            if (foundSegment.isInProgress()
+                    || found == segments.size() - 1) {
+                return found;
+            }
+            if (foundSegment.getLastTxId() >= transactionId) {
+                return found;
+            }
+            return found + 1;
+        }
+    }
+
+    /**
+     * Assign next log segment sequence number based on a decreasing list of log segments.
+     *
+     * @param segmentListDesc
+     *          a decreasing list of log segments
+     * @return null if no log segments was assigned a sequence number in <code>segmentListDesc</code>.
+     *         otherwise, return next log segment sequence number
+     */
+    public static Long nextLogSegmentSequenceNumber(List<LogSegmentMetadata> segmentListDesc) {
+        int lastAssignedLogSegmentIdx = -1;
+        Long lastAssignedLogSegmentSeqNo = null;
+        Long nextLogSegmentSeqNo = null;
+
+        for (int i = 0; i < segmentListDesc.size(); i++) {
+            LogSegmentMetadata metadata = segmentListDesc.get(i);
+            if (LogSegmentMetadata.supportsLogSegmentSequenceNo(metadata.getVersion())) {
+                lastAssignedLogSegmentSeqNo = metadata.getLogSegmentSequenceNumber();
+                lastAssignedLogSegmentIdx = i;
+                break;
+            }
+        }
+
+        if (null != lastAssignedLogSegmentSeqNo) {
+            // latest log segment is assigned with a sequence number, start with next sequence number
+            nextLogSegmentSeqNo = lastAssignedLogSegmentSeqNo + lastAssignedLogSegmentIdx + 1;
+        }
+        return nextLogSegmentSeqNo;
+    }
+
+    /**
+     * Compute the start sequence id for <code>segment</code>, based on previous segment list
+     * <code>segmentListDesc</code>.
+     *
+     * @param logSegmentDescList
+     *          list of segments in descending order
+     * @param segment
+     *          segment to compute start sequence id for
+     * @return start sequence id
+     */
+    public static long computeStartSequenceId(List<LogSegmentMetadata> logSegmentDescList,
+                                              LogSegmentMetadata segment)
+            throws UnexpectedException {
+        long startSequenceId = 0L;
+        for (LogSegmentMetadata metadata : logSegmentDescList) {
+            if (metadata.getLogSegmentSequenceNumber() >= segment.getLogSegmentSequenceNumber()) {
+                continue;
+            } else if (metadata.getLogSegmentSequenceNumber() < (segment.getLogSegmentSequenceNumber() - 1)) {
+                break;
+            }
+            if (metadata.isInProgress()) {
+                throw new UnexpectedException("Should not complete log segment " + segment.getLogSegmentSequenceNumber()
+                        + " since it's previous log segment is still inprogress : " + logSegmentDescList);
+            }
+            if (metadata.supportsSequenceId()) {
+                startSequenceId = metadata.getStartSequenceId() + metadata.getRecordCount();
+            }
+        }
+        return startSequenceId;
+    }
+
+    /**
+     * Deserialize log segment sequence number for bytes <code>data</code>.
+     *
+     * @param data
+     *          byte representation of log segment sequence number
+     * @return log segment sequence number
+     * @throws NumberFormatException if the bytes aren't valid
+     */
+    public static long deserializeLogSegmentSequenceNumber(byte[] data) {
+        String seqNoStr = new String(data, UTF_8);
+        return Long.parseLong(seqNoStr);
+    }
+
+    /**
+     * Serilize log segment sequence number <code>logSegmentSeqNo</code> into bytes.
+     *
+     * @param logSegmentSeqNo
+     *          log segment sequence number
+     * @return byte representation of log segment sequence number
+     */
+    public static byte[] serializeLogSegmentSequenceNumber(long logSegmentSeqNo) {
+        return Long.toString(logSegmentSeqNo).getBytes(UTF_8);
+    }
+
+    /**
+     * Deserialize log record transaction id for bytes <code>data</code>.
+     *
+     * @param data
+     *          byte representation of log record transaction id
+     * @return log record transaction id
+     * @throws NumberFormatException if the bytes aren't valid
+     */
+    public static long deserializeTransactionId(byte[] data) {
+        String seqNoStr = new String(data, UTF_8);
+        return Long.parseLong(seqNoStr);
+    }
+
+    /**
+     * Serilize log record transaction id <code>transactionId</code> into bytes.
+     *
+     * @param transactionId
+     *          log record transaction id
+     * @return byte representation of log record transaction id.
+     */
+    public static byte[] serializeTransactionId(long transactionId) {
+        return Long.toString(transactionId).getBytes(UTF_8);
+    }
+
+    /**
+     * Serialize log segment id into bytes.
+     *
+     * @param logSegmentId
+     *          log segment id
+     * @return bytes representation of log segment id
+     */
+    public static byte[] logSegmentId2Bytes(long logSegmentId) {
+        return Long.toString(logSegmentId).getBytes(UTF_8);
+    }
+
+    /**
+     * Deserialize bytes into log segment id.
+     *
+     * @param data
+     *          bytes representation of log segment id
+     * @return log segment id
+     */
+    public static long bytes2LogSegmentId(byte[] data) {
+        return Long.parseLong(new String(data, UTF_8));
+    }
+
+    /**
+     * Normalize the uri.
+     *
+     * @param uri the distributedlog uri.
+     * @return the normalized uri
+     */
+    public static URI normalizeURI(URI uri) {
+        checkNotNull(uri, "DistributedLog uri is null");
+        String scheme = uri.getScheme();
+        checkNotNull(scheme, "Invalid distributedlog uri : " + uri);
+        scheme = scheme.toLowerCase();
+        String[] schemeParts = StringUtils.split(scheme, '-');
+        checkArgument(Objects.equal(DistributedLogConstants.SCHEME_PREFIX, schemeParts[0].toLowerCase()),
+                "Unknown distributedlog scheme found : " + uri);
+        URI normalizedUri;
+        try {
+            normalizedUri = new URI(
+                    schemeParts[0],     // remove backend info
+                    uri.getAuthority(),
+                    uri.getPath(),
+                    uri.getQuery(),
+                    uri.getFragment());
+        } catch (URISyntaxException e) {
+            throw new IllegalArgumentException("Invalid distributedlog uri found : " + uri, e);
+        }
+        return normalizedUri;
+    }
+
+    private static String getHostIpLockClientId() {
+        try {
+            return InetAddress.getLocalHost().toString();
+        } catch(Exception ex) {
+            return DistributedLogConstants.UNKNOWN_CLIENT_ID;
+        }
+    }
+
+    /**
+     * Normalize the client id.
+     *
+     * @return the normalized client id.
+     */
+    public static String normalizeClientId(String clientId) {
+        String normalizedClientId;
+        if (clientId.equals(DistributedLogConstants.UNKNOWN_CLIENT_ID)) {
+            normalizedClientId = getHostIpLockClientId();
+        } else {
+            normalizedClientId = clientId;
+        }
+        return normalizedClientId;
+    }
+
+    /**
+     * Is it a reserved stream name in bkdl namespace?
+     *
+     * @param name
+     *          stream name
+     * @return true if it is reserved name, otherwise false.
+     */
+    public static boolean isReservedStreamName(String name) {
+        return name.startsWith(".");
+    }
+
+    /**
+     * Validate the stream name.
+     *
+     * @param nameOfStream
+     *          name of stream
+     * @throws InvalidStreamNameException
+     */
+    public static void validateName(String nameOfStream)
+            throws InvalidStreamNameException {
+        String reason = null;
+        char chars[] = nameOfStream.toCharArray();
+        char c;
+        // validate the stream to see if meet zookeeper path's requirement
+        for (int i = 0; i < chars.length; i++) {
+            c = chars[i];
+
+            if (c == 0) {
+                reason = "null character not allowed @" + i;
+                break;
+            } else if (c == '/') {
+                reason = "'/' not allowed @" + i;
+                break;
+            } else if (c > '\u0000' && c < '\u001f'
+                    || c > '\u007f' && c < '\u009F'
+                    || c > '\ud800' && c < '\uf8ff'
+                    || c > '\ufff0' && c < '\uffff') {
+                reason = "invalid charater @" + i;
+                break;
+            }
+        }
+        if (null != reason) {
+            throw new InvalidStreamNameException(nameOfStream, reason);
+        }
+        if (isReservedStreamName(nameOfStream)) {
+            throw new InvalidStreamNameException(nameOfStream,
+                    "Stream Name is reserved");
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java
new file mode 100644
index 0000000..8b5cc65
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/FailpointUtils.java
@@ -0,0 +1,134 @@
+/**
+ * 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.io.IOException;
+import java.util.concurrent.ConcurrentHashMap;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class FailpointUtils {
+    static final Logger logger = LoggerFactory.getLogger(FailpointUtils.class);
+
+    public enum FailPointName {
+        FP_StartLogSegmentBeforeLedgerCreate,
+        FP_StartLogSegmentAfterLedgerCreate,
+        FP_StartLogSegmentAfterInProgressCreate,
+        FP_StartLogSegmentOnAssignLogSegmentSequenceNumber,
+        FP_FinalizeLedgerBeforeDelete,
+        FP_TransmitBeforeAddEntry,
+        FP_TransmitComplete,
+        FP_WriteInternalLostLock,
+        FP_TransmitFailGetBuffer,
+        FP_LockUnlockCleanup,
+        FP_LockTryCloseRaceCondition,
+        FP_LockTryAcquire,
+        FP_ZooKeeperConnectionLoss,
+        FP_RecoverIncompleteLogSegments,
+        FP_LogWriterIssuePending,
+    }
+
+    public static interface FailPointAction {
+        boolean checkFailPoint() throws IOException;
+        boolean checkFailPointNoThrow();
+    }
+
+    public static abstract class AbstractFailPointAction implements FailPointAction {
+        @Override
+        public boolean checkFailPointNoThrow() {
+            try {
+                return checkFailPoint();
+            } catch (IOException ex) {
+                logger.error("failpoint action raised unexpected exception");
+                return true;
+            }
+        }
+    }
+
+    public static final FailPointAction DEFAULT_ACTION = new AbstractFailPointAction() {
+        @Override
+        public boolean checkFailPoint() throws IOException {
+            return true;
+        }
+    };
+
+    public static final FailPointAction THROW_ACTION = new AbstractFailPointAction() {
+        @Override
+        public boolean checkFailPoint() throws IOException {
+            throw new IOException("Throw ioexception for failure point");
+        }
+    };
+
+    public enum FailPointActions {
+        FailPointAction_Default,
+        FailPointAction_Throw
+    }
+
+    static ConcurrentHashMap<FailPointName, FailPointAction> failPointState =
+            new ConcurrentHashMap<FailPointName, FailPointAction>();
+
+    public static void setFailpoint(FailPointName failpoint, FailPointActions action) {
+        FailPointAction fpAction = null;
+        switch (action) {
+        case FailPointAction_Default:
+            fpAction = DEFAULT_ACTION;
+            break;
+        case FailPointAction_Throw:
+            fpAction = THROW_ACTION;
+            break;
+        default:
+            break;
+        }
+        setFailpoint(failpoint, fpAction);
+    }
+
+    public static void setFailpoint(FailPointName failpoint, FailPointAction action) {
+        if (null != action) {
+            failPointState.put(failpoint, action);
+        }
+    }
+
+    public static void removeFailpoint(FailPointName failpoint) {
+        failPointState.remove(failpoint);
+    }
+
+    public static boolean checkFailPoint(FailPointName failPoint) throws IOException {
+        FailPointAction action = failPointState.get(failPoint);
+
+        if (action == null) {
+            return false;
+        }
+
+        try {
+            return action.checkFailPoint();
+        } catch (IOException ioe) {
+            throw new IOException("Induced Exception at:" + failPoint, ioe);
+        }
+    }
+
+    public static boolean checkFailPointNoThrow(FailPointName failPoint) {
+        FailPointAction action = failPointState.get(failPoint);
+
+        if (action == null) {
+            return false;
+        }
+
+        return action.checkFailPointNoThrow();
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/FutureUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/FutureUtils.java
new file mode 100644
index 0000000..8e4a8be
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/FutureUtils.java
@@ -0,0 +1,534 @@
+/**
+ * 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 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.twitter.util.Await;
+import com.twitter.util.Duration;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+import com.twitter.util.FutureCancelledException;
+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.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 javax.annotation.Nullable;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Utilities to process future
+ */
+public class FutureUtils {
+
+    private static final Logger logger = LoggerFactory.getLogger(FutureUtils.class);
+
+    public static class OrderedFutureEventListener<R>
+            implements FutureEventListener<R> {
+
+        public static <R> OrderedFutureEventListener<R> of(
+                FutureEventListener<R> listener,
+                OrderedScheduler scheduler,
+                Object key) {
+            return new OrderedFutureEventListener<R>(scheduler, key, listener);
+        }
+
+        private final OrderedScheduler scheduler;
+        private final Object key;
+        private final FutureEventListener<R> listener;
+
+        private OrderedFutureEventListener(OrderedScheduler scheduler,
+                                           Object key,
+                                           FutureEventListener<R> listener) {
+            this.scheduler = scheduler;
+            this.key = key;
+            this.listener = listener;
+        }
+
+        @Override
+        public void onSuccess(final R value) {
+            scheduler.submit(key, new Runnable() {
+                @Override
+                public void run() {
+                    listener.onSuccess(value);
+                }
+            });
+        }
+
+        @Override
+        public void onFailure(final Throwable cause) {
+            scheduler.submit(key, new Runnable() {
+                @Override
+                public void run() {
+                    listener.onFailure(cause);
+                }
+            });
+        }
+    }
+
+    public static class FutureEventListenerRunnable<R>
+            implements FutureEventListener<R> {
+
+        public static <R> FutureEventListenerRunnable<R> of(
+                FutureEventListener<R> listener,
+                ExecutorService executorService) {
+            return new FutureEventListenerRunnable<R>(executorService, listener);
+        }
+
+        private final ExecutorService executorService;
+        private final FutureEventListener<R> listener;
+
+        private FutureEventListenerRunnable(ExecutorService executorService,
+                                            FutureEventListener<R> listener) {
+            this.executorService = executorService;
+            this.listener = listener;
+        }
+
+        @Override
+        public void onSuccess(final R value) {
+            executorService.submit(new Runnable() {
+                @Override
+                public void run() {
+                    listener.onSuccess(value);
+                }
+            });
+        }
+
+        @Override
+        public void onFailure(final Throwable cause) {
+            executorService.submit(new Runnable() {
+                @Override
+                public void run() {
+                    listener.onFailure(cause);
+                }
+            });
+        }
+    }
+
+    private static class ListFutureProcessor<T, R>
+            extends Function<Throwable, BoxedUnit>
+            implements FutureEventListener<R>, Runnable {
+
+        private volatile boolean interrupted = false;
+        private final Iterator<T> itemsIter;
+        private final Function<T, Future<R>> processFunc;
+        private final Promise<List<R>> promise;
+        private final List<R> results;
+        private final ExecutorService callbackExecutor;
+
+        ListFutureProcessor(List<T> items,
+                            Function<T, Future<R>> processFunc,
+                            ExecutorService callbackExecutor) {
+            this.itemsIter = items.iterator();
+            this.processFunc = processFunc;
+            this.promise = new Promise<List<R>>();
+            this.promise.setInterruptHandler(this);
+            this.results = new ArrayList<R>();
+            this.callbackExecutor = callbackExecutor;
+        }
+
+        @Override
+        public BoxedUnit apply(Throwable cause) {
+            interrupted = true;
+            return BoxedUnit.UNIT;
+        }
+
+        @Override
+        public void onSuccess(R value) {
+            results.add(value);
+            if (null == callbackExecutor) {
+                run();
+            } else {
+                callbackExecutor.submit(this);
+            }
+        }
+
+        @Override
+        public void onFailure(final Throwable cause) {
+            if (null == callbackExecutor) {
+                promise.setException(cause);
+            } else {
+                callbackExecutor.submit(new Runnable() {
+                    @Override
+                    public void run() {
+                        promise.setException(cause);
+                    }
+                });
+            }
+        }
+
+        @Override
+        public void run() {
+            if (interrupted) {
+                logger.debug("ListFutureProcessor is interrupted.");
+                return;
+            }
+            if (!itemsIter.hasNext()) {
+                promise.setValue(results);
+                return;
+            }
+            processFunc.apply(itemsIter.next()).addEventListener(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> Future<List<R>> processList(List<T> collection,
+                                                     Function<T, Future<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;
+    }
+
+    /**
+     * 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> Future<T> stats(Future<T> result,
+                                      OpStatsLogger opStatsLogger,
+                                      Stopwatch stopwatch) {
+        return result.addEventListener(new OpStatsListener<T>(opStatsLogger, stopwatch));
+    }
+
+    /**
+     * Await for the result of the future and thrown bk related exceptions.
+     *
+     * @param result future to wait for
+     * @return the result of future
+     * @throws BKException when exceptions are thrown by the future. If there is unkown exceptions
+     *         thrown from the future, the exceptions will be wrapped into
+     *         {@link org.apache.bookkeeper.client.BKException.BKUnexpectedConditionException}.
+     */
+    public static <T> T bkResult(Future<T> result) throws BKException {
+        try {
+            return Await.result(result);
+        } catch (BKException bke) {
+            throw bke;
+        } catch (InterruptedException ie) {
+            throw BKException.create(BKException.Code.InterruptedException);
+        } catch (Exception e) {
+            logger.warn("Encountered unexpected exception on waiting bookkeeper results : ", e);
+            throw BKException.create(BKException.Code.UnexpectedConditionException);
+        }
+    }
+
+    /**
+     * Return the bk exception return code for a <i>throwable</i>.
+     *
+     * @param throwable the cause of the exception
+     * @return the bk exception return code. if the exception isn't bk exceptions,
+     *         it would return {@link BKException.Code#UnexpectedConditionException}.
+     */
+    public static int bkResultCode(Throwable throwable) {
+        if (throwable instanceof BKException) {
+            return ((BKException)throwable).getCode();
+        }
+        return BKException.Code.UnexpectedConditionException;
+    }
+
+    /**
+     * Wait for the result until it completes.
+     *
+     * @param result result to wait
+     * @return the result
+     * @throws IOException when encountered exceptions on the result
+     */
+    public static <T> T result(Future<T> result) throws IOException {
+        return result(result, Duration.Top());
+    }
+
+    /**
+     * 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}.
+     *
+     * @param result result to wait
+     * @param duration duration to wait
+     * @return the result
+     * @throws IOException when encountered exceptions on the result or waiting for the result.
+     */
+    public static <T> T result(Future<T> result, Duration duration)
+            throws IOException {
+        try {
+            return Await.result(result, duration);
+        } catch (KeeperException ke) {
+            throw new ZKException("Encountered zookeeper exception on waiting result", ke);
+        } catch (BKException bke) {
+            throw new BKTransmitException("Encountered bookkeeper exception on waiting result", bke.getCode());
+        } catch (IOException ioe) {
+            throw ioe;
+        } catch (InterruptedException ie) {
+            throw new DLInterruptedException("Interrupted on waiting result", ie);
+        } catch (Exception e) {
+            throw new IOException("Encountered exception on waiting result", e);
+        }
+    }
+
+    /**
+     * Wait for the result of a lock operation.
+     *
+     * @param result result to wait
+     * @param lockPath path of the lock
+     * @return the result
+     * @throws LockingException when encountered exceptions on the result of lock operation
+     */
+    public static <T> T lockResult(Future<T> result, String lockPath) throws LockingException {
+        try {
+            return Await.result(result);
+        } catch (LockingException le) {
+            throw le;
+        } catch (Exception e) {
+            throw new LockingException(lockPath, "Encountered exception on locking ", e);
+        }
+    }
+
+    /**
+     * 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);
+        }
+    }
+
+    /**
+     * Cancel the future. It would interrupt the future.
+     *
+     * @param future future to cancel
+     */
+    public static <T> void cancel(Future<T> future) {
+        future.raise(new FutureCancelledException());
+    }
+
+    /**
+     * 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> Promise<T> within(final Promise<T> promise,
+                                        final long timeout,
+                                        final TimeUnit unit,
+                                        final Throwable cause,
+                                        final OrderedScheduler scheduler,
+                                        final Object key) {
+        if (timeout < DistributedLogConstants.FUTURE_TIMEOUT_IMMEDIATE || promise.isDefined()) {
+            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.isDefined() && FutureUtils.setException(promise, cause)) {
+                    logger.info("Raise exception", cause);
+                }
+            }
+        }, timeout, unit);
+        // when the promise is satisfied, cancel the timeout task
+        promise.respond(new AbstractFunction1<Try<T>, BoxedUnit>() {
+            @Override
+            public BoxedUnit apply(Try<T> value) {
+                if (!task.cancel(true)) {
+                    logger.debug("Failed to cancel the timeout task");
+                }
+                return BoxedUnit.UNIT;
+            }
+        });
+        return promise;
+    }
+
+    /**
+     * 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
+     * @param value value to satisfy
+     * @param scheduler scheduler to satisfy the promise with provided value
+     * @param key the submit key of the ordered scheduler
+     */
+    public static <T> void setValue(final Promise<T> promise,
+                                    final T value,
+                                    OrderedScheduler scheduler,
+                                    Object key) {
+        scheduler.submit(key, new Runnable() {
+            @Override
+            public void run() {
+                setValue(promise, value);
+            }
+        });
+    }
+
+    /**
+     * Satisfy the <i>promise</i> with provide value.
+     * <p>If the promise was already satisfied, nothing will be changed.
+     *
+     * @param promise promise to satisfy
+     * @param value value to satisfy
+     * @return true if successfully satisfy the future. false if the promise has been satisfied.
+     */
+    public static <T> boolean setValue(Promise<T> promise, T value) {
+        boolean success = promise.updateIfEmpty(new Return<T>(value));
+        if (!success) {
+            logger.info("Result set multiple times. Value = '{}', New = 'Return({})'",
+                    promise.poll(), value);
+        }
+        return success;
+    }
+
+    /**
+     * 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 scheduler the scheduler to satisfy the promise
+     * @param key submit key of the ordered scheduler
+     */
+    public static <T> void setException(final Promise<T> promise,
+                                        final Throwable cause,
+                                        OrderedScheduler scheduler,
+                                        Object key) {
+        scheduler.submit(key, new Runnable() {
+            @Override
+            public void run() {
+                setException(promise, cause);
+            }
+        });
+    }
+
+    /**
+     * Satisfy the <i>promise</i> with provided <i>cause</i>.
+     *
+     * @param promise promise to satisfy
+     * @param cause cause to satisfy
+     * @return true if successfully satisfy the future. false if the promise has been satisfied.
+     */
+    public static <T> boolean setException(Promise<T> promise, Throwable cause) {
+        boolean success = promise.updateIfEmpty(new Throw<T>(cause));
+        if (!success) {
+            logger.info("Result set multiple times. Value = '{}', New = 'Throw({})'",
+                    promise.poll(), cause);
+        }
+        return success;
+    }
+
+    /**
+     * Ignore exception from the <i>future</i>.
+     *
+     * @param future the original future
+     * @return a transformed future ignores exceptions
+     */
+    public static <T> Promise<Void> ignore(Future<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> Promise<Void> ignore(Future<T> future, final String errorMsg) {
+        final Promise<Void> promise = new Promise<Void>();
+        future.addEventListener(new FutureEventListener<T>() {
+            @Override
+            public void onSuccess(T value) {
+                setValue(promise, null);
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                if (null != errorMsg) {
+                    logger.error(errorMsg, cause);
+                }
+                setValue(promise, null);
+            }
+        });
+        return promise;
+    }
+
+    /**
+     * 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);
+    }
+
+}
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
new file mode 100644
index 0000000..3372476
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/MonitoredFuturePool.java
@@ -0,0 +1,131 @@
+/**
+ * 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
new file mode 100644
index 0000000..3121a19
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/MonitoredScheduledThreadPoolExecutor.java
@@ -0,0 +1,257 @@
+/**
+ * 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
new file mode 100644
index 0000000..ad1ba4e
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/OrderedScheduler.java
@@ -0,0 +1,490 @@
+/**
+ * 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/PermitLimiter.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/PermitLimiter.java
new file mode 100644
index 0000000..15394dc
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/PermitLimiter.java
@@ -0,0 +1,57 @@
+/**
+ * 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;
+
+/**
+ * A simple limiter interface which tracks acquire/release of permits, for
+ * example for tracking outstanding writes.
+ */
+public interface PermitLimiter {
+
+    public static PermitLimiter NULL_PERMIT_LIMITER = new PermitLimiter() {
+        @Override
+        public boolean acquire() {
+            return true;
+        }
+        @Override
+        public void release(int permits) {
+        }
+
+        @Override
+        public void close() {
+
+        }
+    };
+
+    /**
+     * Acquire a permit.
+     *
+     * @return true if successfully acquire a permit, otherwise false.
+     */
+    boolean acquire();
+
+    /**
+     * Release a permit.
+     */
+    void release(int permits);
+
+    /**
+     * 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-core/src/main/java/org/apache/distributedlog/util/PermitManager.java
new file mode 100644
index 0000000..24c7860
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/PermitManager.java
@@ -0,0 +1,93 @@
+/**
+ * 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;
+
+public interface PermitManager {
+
+    public static interface Permit {
+        static final Permit ALLOWED = new Permit() {
+            @Override
+            public boolean isAllowed() {
+                return true;
+            }
+        };
+        boolean isAllowed();
+    }
+
+    public static PermitManager UNLIMITED_PERMIT_MANAGER = new PermitManager() {
+        @Override
+        public Permit acquirePermit() {
+            return Permit.ALLOWED;
+        }
+
+        @Override
+        public void releasePermit(Permit permit) {
+            // nop
+        }
+
+        @Override
+        public boolean allowObtainPermits() {
+            return true;
+        }
+
+        @Override
+        public boolean disallowObtainPermits(Permit permit) {
+            return false;
+        }
+
+        @Override
+        public void close() {
+            // nop
+        }
+
+    };
+
+    /**
+     * Obetain a permit from permit manager.
+     *
+     * @return permit.
+     */
+    Permit acquirePermit();
+
+    /**
+     * Release a given permit.
+     *
+     * @param permit
+     *          permit to release
+     */
+    void releasePermit(Permit permit);
+
+    /**
+     * Allow obtaining permits.
+     */
+    boolean allowObtainPermits();
+
+    /**
+     * Disallow obtaining permits. Disallow needs to be performed under the context
+     * of <i>permit</i>.
+     *
+     * @param permit
+     *          permit context to disallow
+     */
+    boolean disallowObtainPermits(Permit permit);
+
+    /**
+     * Release the resources
+     */
+    void close();
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/RetryPolicyUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/RetryPolicyUtils.java
new file mode 100644
index 0000000..87d4d61
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/RetryPolicyUtils.java
@@ -0,0 +1,45 @@
+/**
+ * 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.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.bookkeeper.zookeeper.RetryPolicy;
+
+/**
+ * Utils for {@link org.apache.bookkeeper.zookeeper.RetryPolicy}
+ */
+public class RetryPolicyUtils {
+
+    /**
+     * Infinite retry policy
+     */
+    public static final RetryPolicy DEFAULT_INFINITE_RETRY_POLICY = infiniteRetry(200, 2000);
+
+    /**
+     * Create an infinite retry policy with backoff time between <i>baseBackOffTimeMs</i> and
+     * <i>maxBackoffTimeMs</i>.
+     *
+     * @param baseBackoffTimeMs base backoff time in milliseconds
+     * @param maxBackoffTimeMs maximum backoff time in milliseconds
+     * @return an infinite retry policy
+     */
+    public static RetryPolicy infiniteRetry(long baseBackoffTimeMs, long maxBackoffTimeMs) {
+        return new BoundExponentialBackoffRetryPolicy(baseBackoffTimeMs, maxBackoffTimeMs, Integer.MAX_VALUE);
+    }
+
+}
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
new file mode 100644
index 0000000..a467d26
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/SafeQueueingFuturePool.java
@@ -0,0 +1,115 @@
+/**
+ * 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/SchedulerUtils.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/SchedulerUtils.java
new file mode 100644
index 0000000..66e382c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/SchedulerUtils.java
@@ -0,0 +1,56 @@
+/**
+ * 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.util.OrderedSafeExecutor;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+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;
+        }
+        service.shutdown();
+        try {
+            service.awaitTermination(timeout, timeUnit);
+        } catch (InterruptedException e) {
+            logger.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-core/src/main/java/org/apache/distributedlog/util/Sequencer.java
new file mode 100644
index 0000000..ab8de35
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Sequencer.java
@@ -0,0 +1,31 @@
+/**
+ * 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;
+
+/**
+ * Sequencer generating transaction id.
+ */
+public interface Sequencer {
+
+    /**
+     * Return next transaction id generated by the sequencer.
+     *
+     * @return next transaction id generated by the sequencer.
+     */
+    long nextId();
+}
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
new file mode 100644
index 0000000..767ddf6
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/SimplePermitLimiter.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.util;
+
+import com.google.common.annotations.VisibleForTesting;
+
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+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;
+
+/**
+ * Simple counter based {@link PermitLimiter}.
+ *
+ * <h3>Metrics</h3>
+ * <ul>
+ * <li> `permits`: gauge. how many permits are acquired right now?
+ * <li> `permits`/*: opstats. the characteristics about number of permits already acquired on each acquires.
+ * <li> `acquireFailure`: counter. how many acquires failed? failure means it already reached maximum permits
+ * when trying to acquire.
+ * </ul>
+ */
+public class SimplePermitLimiter implements PermitLimiter {
+
+    final Counter acquireFailureCounter;
+    final OpStatsLogger permitsMetric;
+    final AtomicInteger permits;
+    final int permitsMax;
+    final boolean darkmode;
+    final Feature disableWriteLimitFeature;
+    private StatsLogger statsLogger = null;
+    private Gauge<Number> permitsGauge = null;
+    private String permitsGaugeLabel = "";
+
+    public SimplePermitLimiter(boolean darkmode, int permitsMax, StatsLogger statsLogger,
+                               boolean singleton, Feature disableWriteLimitFeature) {
+        this.permits = new AtomicInteger(0);
+        this.permitsMax = permitsMax;
+        this.darkmode = darkmode;
+        this.disableWriteLimitFeature = disableWriteLimitFeature;
+
+        // stats
+        if (singleton) {
+            this.statsLogger = statsLogger;
+            this.permitsGauge = new Gauge<Number>() {
+                @Override
+                public Number getDefaultValue() {
+                    return 0;
+                }
+                @Override
+                public Number getSample() {
+                    return permits.get();
+                }
+            };
+            this.permitsGaugeLabel = "permits";
+            statsLogger.registerGauge(permitsGaugeLabel, permitsGauge);
+        }
+        acquireFailureCounter = statsLogger.getCounter("acquireFailure");
+        permitsMetric = statsLogger.getOpStatsLogger("permits");
+    }
+
+    public boolean isDarkmode() {
+        return darkmode || disableWriteLimitFeature.isAvailable();
+    }
+
+    @Override
+    public boolean acquire() {
+        permitsMetric.registerSuccessfulEvent(permits.get());
+        if (permits.incrementAndGet() <= permitsMax || isDarkmode()) {
+            return true;
+        } else {
+            acquireFailureCounter.inc();
+            permits.decrementAndGet();
+            return false;
+        }
+    }
+
+    @Override
+    public void release(int permitsToRelease) {
+        permits.addAndGet(-permitsToRelease);
+    }
+
+    @Override
+    public void close() {
+        unregisterGauge();
+    }
+
+    @VisibleForTesting
+    public int getPermits() {
+        return permits.get();
+    }
+
+    public void unregisterGauge() {
+        if (this.statsLogger != null && this.permitsGauge != null) {
+            this.statsLogger.unregisterGauge(permitsGaugeLabel, permitsGauge);
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/Sizable.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Sizable.java
new file mode 100644
index 0000000..2f606e2
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Sizable.java
@@ -0,0 +1,31 @@
+/**
+ * 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;
+
+/**
+ * The {@code Sizable} interface is to provide the capability of calculating size
+ * of any objects.
+ */
+public interface Sizable {
+    /**
+     * Calculate the size for this instance.
+     *
+     * @return size of the instance.
+     */
+    long size();
+}
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
new file mode 100644
index 0000000..69dfdbe
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/TimeSequencer.java
@@ -0,0 +1,39 @@
+/**
+ * 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.distributedlog.DistributedLogConstants;
+
+/**
+ * Time based sequencer. It generated non-decreasing transaction id using milliseconds.
+ * It isn't thread-safe. The caller takes the responsibility on synchronization.
+ */
+public class TimeSequencer implements Sequencer {
+
+    private long lastId = DistributedLogConstants.INVALID_TXID;
+
+    public void setLastId(long lastId) {
+        this.lastId = lastId;
+    }
+
+    @Override
+    public long nextId() {
+        lastId = Math.max(lastId, System.currentTimeMillis());
+        return lastId;
+    }
+}
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
new file mode 100644
index 0000000..3a623dc
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Transaction.java
@@ -0,0 +1,97 @@
+/**
+ * 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.annotations.Beta;
+import com.twitter.util.Future;
+
+/**
+ * Util class represents a transaction
+ */
+@Beta
+public interface Transaction<OpResult> {
+
+    /**
+     * An operation executed in a transaction.
+     */
+    interface Op<OpResult> {
+
+        /**
+         * Execute after the transaction succeeds
+         */
+        void commit(OpResult r);
+
+        /**
+         * Execute after the transaction fails
+         */
+        void abort(Throwable t, OpResult r);
+
+    }
+
+    /**
+     * Listener on the result of an {@link org.apache.distributedlog.util.Transaction.Op}.
+     *
+     * @param <OpResult>
+     */
+    interface OpListener<OpResult> {
+
+        /**
+         * Trigger on operation committed.
+         *
+         * @param r
+         *          result to return
+         */
+        void onCommit(OpResult r);
+
+        /**
+         * Trigger on operation aborted.
+         *
+         * @param t
+         *          reason to abort
+         */
+        void onAbort(Throwable t);
+    }
+
+    /**
+     * Add the operation to current transaction.
+     *
+     * @param operation
+     *          operation to execute under current transaction
+     */
+    void addOp(Op<OpResult> operation);
+
+    /**
+     * Execute the current transaction. If the transaction succeed, all operations will be
+     * committed (via {@link org.apache.distributedlog.util.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();
+
+    /**
+     * Abort current transaction. If this is called and the transaction haven't been executed by
+     * {@link #execute()}, it would abort all operations. If the transaction has been executed,
+     * the behavior is left up to implementation - if transaction is cancellable, the {@link #abort(Throwable)}
+     * could attempt to cancel it.
+     *
+     * @param reason reason to abort the transaction
+     */
+    void abort(Throwable reason);
+
+}
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
new file mode 100644
index 0000000..17eb8e3
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/Utils.java
@@ -0,0 +1,607 @@
+/**
+ * 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.io.Closeable;
+import java.io.IOException;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+import javax.annotation.Nullable;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Optional;
+import com.google.common.collect.Lists;
+import com.google.common.io.Closeables;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.exceptions.ZKException;
+import org.apache.distributedlog.function.VoidFunctions;
+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;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+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.
+ */
+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
+     * changes. This is appropriate to use when computing how much longer to
+     * wait for an interval to expire.
+     *
+     * @return current time in nanoseconds.
+     */
+    public static long nowInNanos() {
+        return System.nanoTime();
+    }
+
+    /**
+     * Current time from some fixed base time - so useful for cross machine
+     * comparison
+     *
+     * @return current time in milliseconds.
+     */
+    public static long nowInMillis() {
+        return System.currentTimeMillis();
+    }
+
+    /**
+     * Milliseconds elapsed since the time specified, the input is nanoTime
+     * the only conversion happens when computing the elapsed time
+     *
+     * @param startMsecTime the start of the interval that we are measuring
+     * @return elapsed time in milliseconds.
+     */
+    public static long elapsedMSec(long startMsecTime) {
+        return (System.currentTimeMillis() - startMsecTime);
+    }
+
+    public static boolean randomPercent(double percent) {
+        return (Math.random() * 100.0) <= percent;
+    }
+
+    /**
+     * Synchronously create zookeeper path recursively and optimistically.
+     *
+     * @see #zkAsyncCreateFullPathOptimistic(ZooKeeperClient, String, byte[], List, CreateMode)
+     * @param zkc Zookeeper client
+     * @param path Zookeeper full path
+     * @param data Zookeeper data
+     * @param acl Acl of the zk path
+     * @param createMode Create mode of zk path
+     * @throws ZooKeeperClient.ZooKeeperConnectionException
+     * @throws KeeperException
+     * @throws InterruptedException
+     */
+    public static void zkCreateFullPathOptimistic(
+        ZooKeeperClient zkc,
+        String path,
+        byte[] data,
+        final List<ACL> acl,
+        final CreateMode createMode)
+        throws ZooKeeperClient.ZooKeeperConnectionException, KeeperException, InterruptedException {
+        try {
+            Await.result(zkAsyncCreateFullPathOptimistic(zkc, path, data, acl, createMode));
+        } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
+            throw zkce;
+        } catch (KeeperException ke) {
+            throw ke;
+        } catch (InterruptedException ie) {
+            throw ie;
+        } catch (RuntimeException rte) {
+            throw rte;
+        } catch (Exception exc) {
+            throw new RuntimeException("Unexpected Exception", exc);
+        }
+    }
+
+    /**
+     * Asynchronously create zookeeper path recursively and optimistically.
+     *
+     * @param zkc Zookeeper client
+     * @param pathToCreate  Zookeeper full path
+     * @param parentPathShouldNotCreate The recursive creation should stop if this path doesn't exist
+     * @param data Zookeeper data
+     * @param acl Acl of the zk path
+     * @param createMode Create mode of zk path
+     * @param callback Callback
+     * @param ctx Context object
+     */
+    public static void zkAsyncCreateFullPathOptimisticRecursive(
+        final ZooKeeperClient zkc,
+        final String pathToCreate,
+        final Optional<String> parentPathShouldNotCreate,
+        final byte[] data,
+        final List<ACL> acl,
+        final CreateMode createMode,
+        final AsyncCallback.StringCallback callback,
+        final Object ctx) {
+        try {
+            zkc.get().create(pathToCreate, data, acl, createMode, new AsyncCallback.StringCallback() {
+                @Override
+                public void processResult(int rc, String path, Object ctx, String name) {
+
+                    if (rc != KeeperException.Code.NONODE.intValue()) {
+                        callback.processResult(rc, path, ctx, name);
+                        return;
+                    }
+
+                    // Since we got a nonode, it means that my parents may not exist
+                    // ephemeral nodes can't have children so Create mode is always
+                    // persistent parents
+                    int lastSlash = pathToCreate.lastIndexOf('/');
+                    if (lastSlash <= 0) {
+                        callback.processResult(rc, path, ctx, name);
+                        return;
+                    }
+                    String parent = pathToCreate.substring(0, lastSlash);
+                    if (parentPathShouldNotCreate.isPresent() && Objects.equal(parentPathShouldNotCreate.get(), parent)) {
+                        // we should stop here
+                        callback.processResult(rc, path, ctx, name);
+                        return;
+                    }
+                    zkAsyncCreateFullPathOptimisticRecursive(zkc, parent, parentPathShouldNotCreate, new byte[0], acl,
+                            CreateMode.PERSISTENT, new AsyncCallback.StringCallback() {
+                                @Override
+                                public void processResult(int rc, String path, Object ctx, String name) {
+                                    if (rc == KeeperException.Code.OK.intValue() || rc == KeeperException.Code.NODEEXISTS.intValue()) {
+                                        // succeeded in creating the parent, now create the original path
+                                        zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate, parentPathShouldNotCreate,
+                                                data, acl, createMode, callback, ctx);
+                                    } else {
+                                        callback.processResult(rc, path, ctx, name);
+                                    }
+                                }
+                            }, ctx);
+                }
+            }, ctx);
+        } catch (ZooKeeperClient.ZooKeeperConnectionException zkce) {
+            callback.processResult(DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE, zkce.getMessage(), ctx, pathToCreate);
+        } catch (InterruptedException ie) {
+            callback.processResult(DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE, ie.getMessage(), ctx, pathToCreate);
+        }
+    }
+
+    /**
+     * Asynchronously create zookeeper path recursively and optimistically.
+     *
+     * @param zkc Zookeeper client
+     * @param pathToCreate  Zookeeper full path
+     * @param data Zookeeper data
+     * @param acl Acl of the zk path
+     * @param createMode Create mode of zk path
+     */
+    public static Future<BoxedUnit> zkAsyncCreateFullPathOptimistic(
+        final ZooKeeperClient zkc,
+        final String pathToCreate,
+        final byte[] data,
+        final List<ACL> acl,
+        final CreateMode createMode) {
+        Optional<String> parentPathShouldNotCreate = Optional.absent();
+        return zkAsyncCreateFullPathOptimistic(
+                zkc,
+                pathToCreate,
+                parentPathShouldNotCreate,
+                data,
+                acl,
+                createMode);
+    }
+
+    /**
+     * Asynchronously create zookeeper path recursively and optimistically
+     *
+     * @param zkc Zookeeper client
+     * @param pathToCreate  Zookeeper full path
+     * @param parentPathShouldNotCreate zookeeper parent path should not be created
+     * @param data Zookeeper data
+     * @param acl Acl of the zk path
+     * @param createMode Create mode of zk path
+     */
+    public static Future<BoxedUnit> 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>();
+
+        zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate, parentPathShouldNotCreate,
+                data, acl, createMode, new AsyncCallback.StringCallback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx, String name) {
+                handleKeeperExceptionCode(rc, path, result);
+            }
+        }, result);
+
+        return result;
+    }
+
+    /**
+     * Asynchronously create zookeeper path recursively and optimistically.
+     *
+     * @param zkc Zookeeper client
+     * @param pathToCreate  Zookeeper full path
+     * @param data Zookeeper data
+     * @param acl Acl of the zk path
+     * @param createMode Create mode of zk path
+     */
+    public static Future<BoxedUnit> zkAsyncCreateFullPathOptimisticAndSetData(
+        final ZooKeeperClient zkc,
+        final String pathToCreate,
+        final byte[] data,
+        final List<ACL> acl,
+        final CreateMode createMode) {
+        final Promise<BoxedUnit> result = new Promise<BoxedUnit>();
+
+        try {
+            zkc.get().setData(pathToCreate, data, -1, new AsyncCallback.StatCallback() {
+                @Override
+                public void processResult(int rc, String path, Object ctx, Stat stat) {
+                    if (rc != KeeperException.Code.NONODE.intValue()) {
+                        handleKeeperExceptionCode(rc, path, result);
+                        return;
+                    }
+
+                    Optional<String> parentPathShouldNotCreate = Optional.absent();
+                    zkAsyncCreateFullPathOptimisticRecursive(zkc, pathToCreate, parentPathShouldNotCreate,
+                            data, acl, createMode, new AsyncCallback.StringCallback() {
+                        @Override
+                        public void processResult(int rc, String path, Object ctx, String name) {
+                            handleKeeperExceptionCode(rc, path, result);
+                        }
+                    }, result);
+                }
+            }, result);
+        } catch (Exception exc) {
+            result.setException(exc);
+        }
+
+        return result;
+    }
+
+    private static void handleKeeperExceptionCode(int rc, String pathOrMessage, Promise<BoxedUnit> result) {
+        if (KeeperException.Code.OK.intValue() == rc) {
+            result.setValue(BoxedUnit.UNIT);
+        } else if (DistributedLogConstants.ZK_CONNECTION_EXCEPTION_RESULT_CODE == rc) {
+            result.setException(new ZooKeeperClient.ZooKeeperConnectionException(pathOrMessage));
+        } else if (DistributedLogConstants.DL_INTERRUPTED_EXCEPTION_RESULT_CODE == rc) {
+            result.setException(new DLInterruptedException(pathOrMessage));
+        } else {
+            result.setException(KeeperException.create(KeeperException.Code.get(rc), pathOrMessage));
+        }
+    }
+
+    public static Future<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));
+        } catch (InterruptedException e) {
+            return Future.exception(FutureUtils.zkException(e, path));
+        }
+        return zkGetData(zk, path, watch);
+    }
+
+    /**
+     * Retrieve data from zookeeper <code>path</code>.
+     *
+     * @param path
+     *          zookeeper path to retrieve data
+     * @param watch
+     *          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[]>>();
+        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));
+                    } else {
+                        promise.setValue(new Versioned<byte[]>(data, new ZkVersion(stat.getVersion())));
+                    }
+                } else if (KeeperException.Code.NONODE.intValue() == rc) {
+                    promise.setValue(new Versioned<byte[]>(null, null));
+                } else {
+                    promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                }
+            }
+        }, null);
+        return promise;
+    }
+
+    public static Future<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));
+        } catch (InterruptedException e) {
+            return Future.exception(FutureUtils.zkException(e, path));
+        }
+        return zkSetData(zk, path, data, version);
+    }
+
+    /**
+     * Set <code>data</code> to zookeeper <code>path</code>.
+     *
+     * @param zk
+     *          zookeeper client
+     * @param path
+     *          path to set data
+     * @param data
+     *          data to set
+     * @param version
+     *          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>();
+        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())));
+                    return;
+                }
+                promise.updateIfEmpty(new Throw<ZkVersion>(
+                        KeeperException.create(KeeperException.Code.get(rc))));
+                return;
+            }
+        }, null);
+        return promise;
+    }
+
+    public static Future<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));
+        } catch (InterruptedException e) {
+            return Future.exception(FutureUtils.zkException(e, path));
+        }
+        return zkDelete(zk, path, version);
+    }
+
+    /**
+     * Delete the given <i>path</i> from zookeeper.
+     *
+     * @param zk
+     *          zookeeper client
+     * @param path
+     *          path to delete
+     * @param version
+     *          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>();
+        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));
+                    return;
+                }
+                promise.updateIfEmpty(new Throw<Void>(
+                        KeeperException.create(KeeperException.Code.get(rc))));
+                return;
+            }
+        }, null);
+        return promise;
+    }
+
+    /**
+     * Delete the given <i>path</i> from zookeeper.
+     *
+     * @param zkc
+     *          zookeeper client
+     * @param path
+     *          path to delete
+     * @param version
+     *          version used to set data
+     * @return future representing if the delete is successful. Return true if the node is deleted,
+     * false if the node doesn't exist, otherwise future will throw exception
+     *
+     */
+    public static Future<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));
+        } catch (InterruptedException e) {
+            return Future.exception(FutureUtils.zkException(e, path));
+        }
+        final Promise<Boolean> promise = new Promise<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);
+                } else if (KeeperException.Code.NONODE.intValue() == rc) {
+                    promise.setValue(false);
+                } else {
+                    promise.setException(KeeperException.create(KeeperException.Code.get(rc)));
+                }
+            }
+        }, null);
+        return promise;
+    }
+
+    public static Future<Void> asyncClose(@Nullable AsyncCloseable closeable,
+                                          boolean swallowIOException) {
+        if (null == closeable) {
+            return Future.Void();
+        } else if (swallowIOException) {
+            return FutureUtils.ignore(closeable.asyncClose());
+        } else {
+            return closeable.asyncClose();
+        }
+    }
+
+    /**
+     * Sync zookeeper client on given <i>path</i>.
+     *
+     * @param zkc
+     *          zookeeper client
+     * @param path
+     *          path to sync
+     * @return zookeeper client after sync
+     * @throws IOException
+     */
+    public static ZooKeeper sync(ZooKeeperClient zkc, String path) throws IOException {
+        ZooKeeper zk;
+        try {
+            zk = zkc.get();
+        } catch (InterruptedException e) {
+            throw new DLInterruptedException("Interrupted on checking if log " + path + " exists", e);
+        }
+        final CountDownLatch syncLatch = new CountDownLatch(1);
+        final AtomicInteger syncResult = new AtomicInteger(0);
+        zk.sync(path, new AsyncCallback.VoidCallback() {
+            @Override
+            public void processResult(int rc, String path, Object ctx) {
+                syncResult.set(rc);
+                syncLatch.countDown();
+            }
+        }, null);
+        try {
+            syncLatch.await();
+        } catch (InterruptedException e) {
+            throw new DLInterruptedException("Interrupted on syncing zookeeper connection", e);
+        }
+        if (KeeperException.Code.OK.intValue() != syncResult.get()) {
+            throw new ZKException("Error syncing zookeeper connection ",
+                    KeeperException.Code.get(syncResult.get()));
+        }
+        return zk;
+    }
+
+    /**
+     * Close a closeable.
+     *
+     * @param closeable
+     *          closeable to close
+     */
+    public static void close(@Nullable Closeable closeable) {
+        if (null == closeable) {
+            return;
+        }
+        try {
+            Closeables.close(closeable, true);
+        } catch (IOException e) {
+            // no-op. the exception is swallowed.
+        }
+    }
+
+    /**
+     * Close an async closeable.
+     *
+     * @param closeable
+     *          closeable to close
+     */
+    public static void close(@Nullable AsyncCloseable closeable)
+            throws IOException {
+        if (null == closeable) {
+            return;
+        }
+        FutureUtils.result(closeable.asyncClose());
+    }
+
+    /**
+     * Close an async closeable.
+     *
+     * @param closeable
+     *          closeable to close
+     */
+    public static void closeQuietly(@Nullable AsyncCloseable closeable) {
+        if (null == closeable) {
+            return;
+        }
+        try {
+            FutureUtils.result(closeable.asyncClose());
+        } catch (IOException e) {
+            // no-op. the exception is swallowed.
+        }
+    }
+
+    /**
+     * Close the closeables in sequence.
+     *
+     * @param closeables
+     *          closeables to close
+     * @return future represents the close future
+     */
+    public static Future<Void> closeSequence(ExecutorService executorService,
+                                             AsyncCloseable... closeables) {
+        return closeSequence(executorService, false, closeables);
+    }
+
+    /**
+     * Close the closeables in sequence and ignore errors during closing.
+     *
+     * @param executorService executor to execute closeable
+     * @param ignoreCloseError whether to ignore errors during closing
+     * @param closeables list of closeables
+     * @return future represents the close future.
+     */
+    public static Future<Void> closeSequence(ExecutorService executorService,
+                                             boolean ignoreCloseError,
+                                             AsyncCloseable... closeables) {
+        List<AsyncCloseable> closeableList = Lists.newArrayListWithExpectedSize(closeables.length);
+        for (AsyncCloseable closeable : closeables) {
+            if (null == closeable) {
+                closeableList.add(AsyncCloseable.NULL);
+            } else {
+                closeableList.add(closeable);
+            }
+        }
+        return FutureUtils.processList(
+                closeableList,
+                ignoreCloseError ? AsyncCloseable.CLOSE_FUNC_IGNORE_ERRORS : AsyncCloseable.CLOSE_FUNC,
+                executorService).map(VoidFunctions.LIST_TO_VOID_FUNC);
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/util/package-info.java b/distributedlog-core/src/main/java/org/apache/distributedlog/util/package-info.java
new file mode 100644
index 0000000..7371ef4
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/util/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * DistributedLog Utils
+ */
+package org.apache.distributedlog.util;
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/DefaultZKOp.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/DefaultZKOp.java
new file mode 100644
index 0000000..45120d4
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/DefaultZKOp.java
@@ -0,0 +1,55 @@
+/**
+ * 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.zk;
+
+import org.apache.distributedlog.util.Transaction.OpListener;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
+
+import javax.annotation.Nullable;
+
+/**
+ * Default zookeeper operation. No action on commiting or aborting.
+ */
+public class DefaultZKOp extends ZKOp {
+
+    public static DefaultZKOp of(Op op, OpListener<Void> listener) {
+        return new DefaultZKOp(op, listener);
+    }
+
+    private final OpListener<Void> listener;
+
+    private DefaultZKOp(Op op, @Nullable OpListener<Void> opListener) {
+        super(op);
+        this.listener = opListener;
+    }
+
+    @Override
+    protected void commitOpResult(OpResult opResult) {
+        if (null != listener) {
+            listener.onCommit(null);
+        }
+    }
+
+    @Override
+    protected void abortOpResult(Throwable t, OpResult opResult) {
+        if (null != listener) {
+            listener.onAbort(t);
+        }
+    }
+}
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
new file mode 100644
index 0000000..9a61c1c
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/LimitedPermitManager.java
@@ -0,0 +1,195 @@
+/**
+ * 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.zk;
+
+import org.apache.distributedlog.util.PermitManager;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.Semaphore;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Manager to control all the log segments rolling.
+ */
+public class LimitedPermitManager implements PermitManager, Runnable, Watcher {
+
+    static final Logger LOG = LoggerFactory.getLogger(LimitedPermitManager.class);
+
+    static enum PermitState {
+        ALLOWED, DISALLOWED, DISABLED
+    }
+
+    class EpochPermit implements Permit {
+
+        final PermitState state;
+        final int epoch;
+
+        EpochPermit(PermitState state) {
+            this.state = state;
+            this.epoch = LimitedPermitManager.this.epoch.get();
+        }
+
+        int getEpoch() {
+            return epoch;
+        }
+
+        @Override
+        public boolean isAllowed() {
+            return PermitState.ALLOWED == state;
+        }
+    }
+
+    boolean enablePermits = true;
+    final Semaphore semaphore;
+    final int period;
+    final TimeUnit timeUnit;
+    final ScheduledExecutorService executorService;
+    final AtomicInteger epoch = new AtomicInteger(0);
+    private StatsLogger statsLogger = null;
+    private Gauge<Number> outstandingGauge = null;
+
+    public LimitedPermitManager(int concurrency, int period, TimeUnit timeUnit,
+                                ScheduledExecutorService executorService) {
+        this(concurrency, period, timeUnit, executorService, NullStatsLogger.INSTANCE);
+    }
+
+    public LimitedPermitManager(final int concurrency, int period, TimeUnit timeUnit,
+            ScheduledExecutorService executorService, StatsLogger statsLogger) {
+        if (concurrency > 0) {
+            this.semaphore = new Semaphore(concurrency);
+        } else {
+            this.semaphore = null;
+        }
+        this.period = period;
+        this.timeUnit = timeUnit;
+        this.executorService = executorService;
+        this.statsLogger = statsLogger;
+        this.outstandingGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return null == semaphore ? 0 : concurrency - semaphore.availablePermits();
+            }
+        };
+        this.statsLogger.scope("permits").registerGauge("outstanding", this.outstandingGauge);
+    }
+
+    @Override
+    synchronized public Permit acquirePermit() {
+        if (!enablePermits) {
+            return new EpochPermit(PermitState.DISABLED);
+        }
+        if (null != semaphore) {
+            return semaphore.tryAcquire() ? new EpochPermit(PermitState.ALLOWED) :
+                    new EpochPermit(PermitState.DISALLOWED);
+        } else {
+            return new EpochPermit(PermitState.ALLOWED);
+        }
+    }
+
+    @Override
+    synchronized public void releasePermit(Permit permit) {
+        if (null != semaphore && permit.isAllowed()) {
+            if (period <= 0) {
+                semaphore.release();
+            } else {
+                try {
+                    executorService.schedule(this, period, timeUnit);
+                } catch (RejectedExecutionException ree) {
+                    LOG.warn("Failed on scheduling releasing permit in given period ({}ms)." +
+                            " Release it immediately : ", timeUnit.toMillis(period), ree);
+                    semaphore.release();
+                }
+            }
+        }
+    }
+
+    @Override
+    synchronized public boolean disallowObtainPermits(Permit permit) {
+        if (!(permit instanceof EpochPermit)) {
+            return false;
+        }
+        if (epoch.getAndIncrement() == ((EpochPermit)permit).getEpoch()) {
+            this.enablePermits = false;
+            LOG.info("EnablePermits = {}, Epoch = {}.", this.enablePermits, epoch.get());
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public void close() {
+        unregisterGauge();
+    }
+
+    @Override
+    synchronized public boolean allowObtainPermits() {
+        forceSetAllowPermits(true);
+        return true;
+    }
+
+    synchronized void forceSetAllowPermits(boolean allowPermits) {
+        epoch.getAndIncrement();
+        this.enablePermits = allowPermits;
+        LOG.info("EnablePermits = {}, Epoch = {}.", this.enablePermits, epoch.get());
+    }
+
+    @Override
+    public void run() {
+        semaphore.release();
+    }
+
+    @Override
+    public void process(WatchedEvent event) {
+        if (event.getType().equals(Event.EventType.None)) {
+            switch (event.getState()) {
+            case SyncConnected:
+                forceSetAllowPermits(true);
+                break;
+            case Disconnected:
+                forceSetAllowPermits(false);
+                break;
+            case Expired:
+                forceSetAllowPermits(false);
+                break;
+            default:
+                break;
+            }
+        }
+    }
+
+    public void unregisterGauge() {
+        if(this.statsLogger != null && this.outstandingGauge != null) {
+            this.statsLogger.scope("permits").unregisterGauge("outstanding", this.outstandingGauge);
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKOp.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKOp.java
new file mode 100644
index 0000000..39e4c30
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKOp.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.zk;
+
+import org.apache.distributedlog.util.Transaction;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
+
+import javax.annotation.Nullable;
+
+/**
+ * ZooKeeper Transaction Operation
+ */
+public abstract class ZKOp implements Transaction.Op<Object> {
+
+    protected final Op op;
+
+    protected ZKOp(Op op) {
+        this.op = op;
+    }
+
+    public Op getOp() {
+        return op;
+    }
+
+    @Override
+    public void commit(Object r) {
+        assert(r instanceof OpResult);
+        commitOpResult((OpResult) r);
+    }
+
+    protected abstract void commitOpResult(OpResult opResult);
+
+    @Override
+    public void abort(Throwable t, Object r) {
+        assert(r instanceof OpResult);
+        abortOpResult(t, (OpResult) r);
+    }
+
+    /**
+     * Abort the operation with exception <i>t</i> and result <i>opResult</i>.
+     *
+     * @param t the reason to abort the operation
+     * @param opResult the result of operation
+     */
+    protected abstract void abortOpResult(Throwable t,
+                                          @Nullable OpResult opResult);
+}
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
new file mode 100644
index 0000000..a5da9c0
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKTransaction.java
@@ -0,0 +1,103 @@
+/**
+ * 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.zk;
+
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.Transaction;
+import com.twitter.util.Future;
+import com.twitter.util.Promise;
+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
+ */
+public class ZKTransaction implements Transaction<Object>, AsyncCallback.MultiCallback {
+
+    private final ZooKeeperClient zkc;
+    private final List<ZKOp> ops;
+    private final List<org.apache.zookeeper.Op> zkOps;
+    private final Promise<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>();
+    }
+
+    @Override
+    public void addOp(Op<Object> operation) {
+        if (done.get()) {
+            throw new IllegalStateException("Add an operation to a finished transaction");
+        }
+        assert(operation instanceof ZKOp);
+        ZKOp zkOp = (ZKOp) operation;
+        this.ops.add(zkOp);
+        this.zkOps.add(zkOp.getOp());
+    }
+
+    @Override
+    public Future<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, ""));
+        } catch (InterruptedException e) {
+            result.setException(FutureUtils.zkException(e, ""));
+        }
+        return result;
+    }
+
+    @Override
+    public void abort(Throwable cause) {
+        if (!done.compareAndSet(false, true)) {
+            return;
+        }
+        for (int i = 0; i < ops.size(); i++) {
+            ops.get(i).abortOpResult(cause, null);
+        }
+        FutureUtils.setException(result, cause);
+    }
+
+    @Override
+    public void processResult(int rc, String path, Object ctx, List<OpResult> results) {
+        if (KeeperException.Code.OK.intValue() == rc) { // transaction succeed
+            for (int i = 0; i < ops.size(); i++) {
+                ops.get(i).commitOpResult(results.get(i));
+            }
+            FutureUtils.setValue(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);
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKVersionedSetOp.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKVersionedSetOp.java
new file mode 100644
index 0000000..6ab5973
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKVersionedSetOp.java
@@ -0,0 +1,71 @@
+/**
+ * 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.zk;
+
+import org.apache.distributedlog.util.Transaction.OpListener;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
+
+import javax.annotation.Nullable;
+
+/**
+ * ZooKeeper Operation that plays with {@link org.apache.bookkeeper.versioning.Version}
+ */
+public class ZKVersionedSetOp extends ZKOp {
+
+    private final OpListener<Version> listener;
+
+    public ZKVersionedSetOp(Op op,
+                            @Nullable OpListener<Version> opListener) {
+        super(op);
+        this.listener = opListener;
+    }
+
+    @Override
+    protected void commitOpResult(OpResult opResult) {
+        assert(opResult instanceof OpResult.SetDataResult);
+        OpResult.SetDataResult setDataResult = (OpResult.SetDataResult) opResult;
+        if (null != listener) {
+            listener.onCommit(new ZkVersion(setDataResult.getStat().getVersion()));
+        }
+    }
+
+    @Override
+    protected void abortOpResult(Throwable t,
+                                 @Nullable OpResult opResult) {
+        Throwable cause;
+        if (null == opResult) {
+            cause = t;
+        } else {
+            assert (opResult instanceof OpResult.ErrorResult);
+            OpResult.ErrorResult errorResult = (OpResult.ErrorResult) opResult;
+            if (KeeperException.Code.OK.intValue() == errorResult.getErr()) {
+                cause = t;
+            } else {
+                cause = KeeperException.create(KeeperException.Code.get(errorResult.getErr()));
+            }
+        }
+        if (null != listener) {
+            listener.onAbort(cause);
+        }
+    }
+
+}
diff --git a/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKWatcherManager.java b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKWatcherManager.java
new file mode 100644
index 0000000..670b5d2
--- /dev/null
+++ b/distributedlog-core/src/main/java/org/apache/distributedlog/zk/ZKWatcherManager.java
@@ -0,0 +1,239 @@
+/**
+ * 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.zk;
+
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.HashSet;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.atomic.AtomicInteger;
+
+/**
+ * Watcher Manager to manage watchers.
+ * <h3>Metrics</h3>
+ * <ul>
+ * <li> `total_watches`: total number of watches that managed by this watcher manager.
+ * <li> `num_child_watches`: number of paths that are watched for children changes by this watcher manager.
+ * </ul>
+ */
+public class ZKWatcherManager implements Watcher {
+
+    static final Logger logger = LoggerFactory.getLogger(ZKWatcherManager.class);
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    public static class Builder {
+
+        private String _name;
+        private StatsLogger _statsLogger;
+        private ZooKeeperClient _zkc;
+
+        public Builder name(String name) {
+            this._name = name;
+            return this;
+        }
+
+        public Builder zkc(ZooKeeperClient zkc) {
+            this._zkc = zkc;
+            return this;
+        }
+
+        public Builder statsLogger(StatsLogger statsLogger) {
+            this._statsLogger = statsLogger;
+            return this;
+        }
+
+        public ZKWatcherManager build() {
+            return new ZKWatcherManager(_name, _zkc, _statsLogger);
+        }
+    }
+
+    private final String name;
+    private final ZooKeeperClient zkc;
+    private final StatsLogger statsLogger;
+    // Gauges and their labels
+    private final Gauge<Number> totalWatchesGauge;
+    private static final String totalWatchesGauageLabel = "total_watches";
+    private final Gauge<Number> numChildWatchesGauge;
+    private static final String numChildWatchesGauageLabel = "num_child_watches";
+
+    protected final ConcurrentMap<String, Set<Watcher>> childWatches;
+    protected final AtomicInteger allWatchesGauge;
+
+    private ZKWatcherManager(String name,
+                             ZooKeeperClient zkc,
+                             StatsLogger statsLogger) {
+        this.name = name;
+        this.zkc = zkc;
+        this.statsLogger = statsLogger;
+
+        // watches
+        this.childWatches = new ConcurrentHashMap<String, Set<Watcher>>();
+        this.allWatchesGauge = new AtomicInteger(0);
+
+        // stats
+        totalWatchesGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return allWatchesGauge.get();
+            }
+        };
+        this.statsLogger.registerGauge(totalWatchesGauageLabel, totalWatchesGauge);
+
+        numChildWatchesGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return childWatches.size();
+            }
+        };
+
+        this.statsLogger.registerGauge(numChildWatchesGauageLabel, numChildWatchesGauge);
+    }
+
+    public Watcher registerChildWatcher(String path, Watcher watcher) {
+        Set<Watcher> watchers = childWatches.get(path);
+        if (null == watchers) {
+            Set<Watcher> newWatchers = new HashSet<Watcher>();
+            Set<Watcher> oldWatchers = childWatches.putIfAbsent(path, newWatchers);
+            watchers = (null == oldWatchers) ? newWatchers : oldWatchers;
+        }
+        synchronized (watchers) {
+            if (childWatches.get(path) == watchers) {
+                if (watchers.add(watcher)) {
+                    allWatchesGauge.incrementAndGet();
+                }
+            } else {
+                logger.warn("Watcher set for path {} has been changed while registering child watcher {}.",
+                        path, watcher);
+            }
+        }
+        return this;
+    }
+
+    public void unregisterChildWatcher(String path, Watcher watcher, boolean removeFromServer) {
+        Set<Watcher> watchers = childWatches.get(path);
+        if (null == watchers) {
+            logger.warn("No watchers found on path {} while unregistering child watcher {}.",
+                    path, watcher);
+            return;
+        }
+        synchronized (watchers) {
+            if (watchers.remove(watcher)) {
+                allWatchesGauge.decrementAndGet();
+            } else {
+                logger.warn("Remove a non-registered child watcher {} from path {}", watcher, path);
+            }
+            if (watchers.isEmpty()) {
+                // best-efforts to remove watches
+                try {
+                    if (null != zkc && removeFromServer) {
+                        zkc.get().removeWatches(path, this, WatcherType.Children, true, new AsyncCallback.VoidCallback() {
+                            @Override
+                            public void processResult(int rc, String path, Object ctx) {
+                                if (KeeperException.Code.OK.intValue() == rc) {
+                                    logger.debug("Successfully removed children watches from {}", path);
+                                } else {
+                                    logger.debug("Encountered exception on removing children watches from {}",
+                                            path, KeeperException.create(KeeperException.Code.get(rc)));
+                                }
+                            }
+                        }, null);
+                    }
+                } catch (InterruptedException e) {
+                    logger.debug("Encountered exception on removing watches from {}", path, e);
+                } catch (ZooKeeperClient.ZooKeeperConnectionException e) {
+                    logger.debug("Encountered exception on removing watches from {}", path, e);
+                }
+                childWatches.remove(path, watchers);
+            }
+        }
+    }
+
+    public void unregisterGauges() {
+        this.statsLogger.unregisterGauge(totalWatchesGauageLabel, totalWatchesGauge);
+        this.statsLogger.unregisterGauge(numChildWatchesGauageLabel, numChildWatchesGauge);
+    }
+
+    @Override
+    public void process(WatchedEvent event) {
+        switch (event.getType()) {
+            case None:
+                handleKeeperStateEvent(event);
+                break;
+            case NodeChildrenChanged:
+                handleChildWatchEvent(event);
+                break;
+            default:
+                break;
+        }
+    }
+
+    private void handleKeeperStateEvent(WatchedEvent event) {
+        Set<Watcher> savedAllWatches = new HashSet<Watcher>(allWatchesGauge.get());
+        for (Set<Watcher> watcherSet : childWatches.values()) {
+            synchronized (watcherSet) {
+                savedAllWatches.addAll(watcherSet);
+            }
+        }
+        for (Watcher watcher : savedAllWatches) {
+            watcher.process(event);
+        }
+    }
+
+    private void handleChildWatchEvent(WatchedEvent event) {
+        String path = event.getPath();
+        if (null == path) {
+            logger.warn("Received zookeeper watch event with null path : {}", event);
+            return;
+        }
+        Set<Watcher> watchers = childWatches.get(path);
+        if (null == watchers) {
+            return;
+        }
+        Set<Watcher> watchersToFire;
+        synchronized (watchers) {
+            watchersToFire = new HashSet<Watcher>(watchers.size());
+            watchersToFire.addAll(watchers);
+        }
+        for (Watcher watcher : watchersToFire) {
+            watcher.process(event);
+        }
+    }
+}
diff --git a/distributedlog-core/src/main/resources/findbugsExclude.xml b/distributedlog-core/src/main/resources/findbugsExclude.xml
index c07fad9..80adec8 100644
--- a/distributedlog-core/src/main/resources/findbugsExclude.xml
+++ b/distributedlog-core/src/main/resources/findbugsExclude.xml
@@ -18,22 +18,22 @@
 <FindBugsFilter>
   <Match>
     <!-- generated code, we can't be held responsible for findbugs in it //-->
-    <Class name="~com\.twitter\.distributedlog\.thrift.*" />
+    <Class name="~org\.apache\.distributedlog\.thrift.*" />
   </Match>
   <Match>
     <!-- it is safe to store external bytes reference here. //-->
-    <Class name="com.twitter.distributedlog.Entry$Builder" />
+    <Class name="org.apache.distributedlog.Entry$Builder" />
     <Method name="setData" />
     <Bug pattern="EI_EXPOSE_REP2" />
   </Match>
   <Match>
     <!-- it is safe to store external bytes reference here. //-->
-    <Class name="com.twitter.distributedlog.Entry" />
+    <Class name="org.apache.distributedlog.Entry" />
     <Method name="getRawData" />
     <Bug pattern="EI_EXPOSE_REP" />
   </Match>
   <Match>
-    <Class name="com.twitter.distributedlog.BKAsyncLogReader" />
+    <Class name="org.apache.distributedlog.BKAsyncLogReader" />
     <Method name="run" />
     <Bug pattern="JLM_JSR166_UTILCONCURRENT_MONITORENTER" />
   </Match>
diff --git a/distributedlog-core/src/main/thrift/metadata.thrift b/distributedlog-core/src/main/thrift/metadata.thrift
index e424ca1..4c9918b 100644
--- a/distributedlog-core/src/main/thrift/metadata.thrift
+++ b/distributedlog-core/src/main/thrift/metadata.thrift
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-namespace java com.twitter.distributedlog.thrift
+namespace java org.apache.distributedlog.thrift
 
 struct BKDLConfigFormat {
     1: optional string bkZkServers
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/DLMTestUtil.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/DLMTestUtil.java
deleted file mode 100644
index c403e26..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/DLMTestUtil.java
+++ /dev/null
@@ -1,489 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.namespace.NamespaceDriver;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.PermitLimiter;
-import com.twitter.distributedlog.util.RetryPolicyUtils;
-import com.twitter.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;
-import org.apache.bookkeeper.feature.SettableFeatureProvider;
-import org.apache.bookkeeper.versioning.Version;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.net.URL;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import static org.junit.Assert.assertArrayEquals;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertNotNull;
-
-/**
- * Utility class for setting up bookkeeper ensembles
- * and bringing individual bookies up and down
- */
-public class DLMTestUtil {
-    protected static final Logger LOG = LoggerFactory.getLogger(DLMTestUtil.class);
-    private final static byte[] payloadStatic = repeatString("abc", 512).getBytes();
-
-    static String repeatString(String s, int n) {
-        String ret = s;
-        for(int i = 1; i < n; i++) {
-            ret += s;
-        }
-        return ret;
-    }
-
-    public static Map<Long, LogSegmentMetadata> readLogSegments(ZooKeeperClient zkc, String ledgerPath) throws Exception {
-        List<String> children = zkc.get().getChildren(ledgerPath, false);
-        LOG.info("Children under {} : {}", ledgerPath, children);
-        Map<Long, LogSegmentMetadata> segments =
-            new HashMap<Long, LogSegmentMetadata>(children.size());
-        for (String child : children) {
-            LogSegmentMetadata segment =
-                    FutureUtils.result(LogSegmentMetadata.read(zkc, ledgerPath + "/" + child));
-            LOG.info("Read segment {} : {}", child, segment);
-            segments.put(segment.getLogSegmentSequenceNumber(), segment);
-        }
-        return segments;
-    }
-
-    public static URI createDLMURI(int port, String path) throws Exception {
-        return LocalDLMEmulator.createDLMURI("127.0.0.1:" + port, path);
-    }
-
-    public static DistributedLogManager createNewDLM(String name,
-                                                     DistributedLogConfiguration conf,
-                                                     URI uri) throws Exception {
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf).uri(uri).build();
-        return namespace.openLog(name);
-    }
-
-    static MetadataAccessor createNewMetadataAccessor(DistributedLogConfiguration conf,
-                                                      String name,
-                                                      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()
-                .conf(conf).uri(uri).build();
-        return namespace.getNamespaceDriver().getMetadataAccessor(name);
-    }
-
-    public static void fenceStream(DistributedLogConfiguration conf, URI uri, String name) throws Exception {
-        DistributedLogManager dlm = createNewDLM(name, conf, uri);
-        try {
-            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)));
-        } finally {
-            dlm.close();
-        }
-    }
-
-    static long getNumberofLogRecords(DistributedLogManager bkdlm, long startTxId) throws IOException {
-        long numLogRecs = 0;
-        LogReader reader = bkdlm.getInputStream(startTxId);
-        LogRecord record = reader.readNext(false);
-        while (null != record) {
-            numLogRecs++;
-            verifyLogRecord(record);
-            record = reader.readNext(false);
-        }
-        reader.close();
-        return numLogRecs;
-    }
-
-    public static LogRecord getLogRecordInstance(long txId) {
-        return new LogRecord(txId, generatePayload(txId));
-    }
-
-    public static LogRecord getLogRecordInstance(long txId, int size) {
-        ByteBuffer buf = ByteBuffer.allocate(size);
-        return new LogRecord(txId, buf.array());
-    }
-
-    public static void verifyLogRecord(LogRecord record) {
-        assertEquals(generatePayload(record.getTransactionId()).length, record.getPayload().length);
-        assertArrayEquals(generatePayload(record.getTransactionId()), record.getPayload());
-        assertTrue(!record.isControl());
-        verifyPayload(record.getTransactionId(), record.getPayload());
-    }
-
-    static byte[] generatePayload(long txId) {
-        return String.format("%d;%d", txId, txId).getBytes();
-    }
-
-    static void verifyPayload(long txId, byte[] payload) {
-        String[] txIds = new String(payload).split(";");
-        assertEquals(Long.valueOf(txIds[0]), Long.valueOf(txIds[0]));
-    }
-
-    static LogRecord getLargeLogRecordInstance(long txId, boolean control) {
-        LogRecord record = new LogRecord(txId, payloadStatic);
-        if (control) {
-            record.setControl();
-        }
-        return record;
-    }
-
-    static LogRecord getLargeLogRecordInstance(long txId) {
-        return new LogRecord(txId, payloadStatic);
-    }
-
-    static List<LogRecord> getLargeLogRecordInstanceList(long firstTxId, int count) {
-        List<LogRecord> logrecs = new ArrayList<LogRecord>(count);
-        for (long i = 0; i < count; i++) {
-            logrecs.add(getLargeLogRecordInstance(firstTxId + i));
-        }
-        return logrecs;
-    }
-
-    static List<LogRecord> getLogRecordInstanceList(long firstTxId, int count, int size) {
-        List<LogRecord> logrecs = new ArrayList<LogRecord>(count);
-        for (long i = 0; i < count; i++) {
-            logrecs.add(getLogRecordInstance(firstTxId + i, size));
-        }
-        return logrecs;
-    }
-
-    static void verifyLargeLogRecord(LogRecord record) {
-        verifyLargeLogRecord(record.getPayload());
-    }
-
-    static void verifyLargeLogRecord(byte[] payload) {
-        assertArrayEquals(payloadStatic, payload);
-    }
-
-    static LogRecord getEmptyLogRecordInstance(long txId) {
-        return new LogRecord(txId, new byte[0]);
-    }
-
-    static void verifyEmptyLogRecord(LogRecord record) {
-        assertEquals(record.getPayload().length, 0);
-    }
-
-    public static LogRecordWithDLSN getLogRecordWithDLSNInstance(DLSN dlsn, long txId) {
-        return getLogRecordWithDLSNInstance(dlsn, txId, false);
-    }
-
-    public static LogRecordWithDLSN getLogRecordWithDLSNInstance(DLSN dlsn, long txId, boolean isControlRecord) {
-        LogRecordWithDLSN record = new LogRecordWithDLSN(dlsn, txId, generatePayload(txId), 1L);
-        record.setPositionWithinLogSegment((int)txId + 1);
-        if (isControlRecord) {
-            record.setControl();
-        }
-        return record;
-    }
-
-    public static String inprogressZNodeName(long logSegmentSeqNo) {
-        return String.format("%s_%018d", DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX, logSegmentSeqNo);
-    }
-
-    public static String completedLedgerZNodeNameWithVersion(long ledgerId, long firstTxId, long lastTxId, long logSegmentSeqNo) {
-        return String.format("%s_%018d_%018d_%018d_v%dl%d_%04d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX,
-                             firstTxId, lastTxId, logSegmentSeqNo, DistributedLogConstants.LOGSEGMENT_NAME_VERSION, ledgerId,
-                             DistributedLogConstants.LOCAL_REGION_ID);
-    }
-
-    public static String completedLedgerZNodeNameWithTxID(long firstTxId, long lastTxId) {
-        return String.format("%s_%018d_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX, firstTxId, lastTxId);
-    }
-
-    public static String completedLedgerZNodeNameWithLogSegmentSequenceNumber(long logSegmentSeqNo) {
-        return String.format("%s_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX, logSegmentSeqNo);
-    }
-
-    public static LogSegmentMetadata inprogressLogSegment(String ledgerPath,
-                                                          long ledgerId,
-                                                          long firstTxId,
-                                                          long logSegmentSeqNo) {
-        return inprogressLogSegment(ledgerPath, ledgerId, firstTxId, logSegmentSeqNo,
-                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
-    }
-
-    public static LogSegmentMetadata inprogressLogSegment(String ledgerPath,
-                                                          long ledgerId,
-                                                          long firstTxId,
-                                                          long logSegmentSeqNo,
-                                                          int version) {
-        return new LogSegmentMetadata.LogSegmentMetadataBuilder(
-                    ledgerPath + "/" + inprogressZNodeName(logSegmentSeqNo),
-                    version,
-                    ledgerId,
-                    firstTxId)
-                .setLogSegmentSequenceNo(logSegmentSeqNo)
-                .build();
-    }
-
-    public static LogSegmentMetadata completedLogSegment(String ledgerPath,
-                                                         long ledgerId,
-                                                         long firstTxId,
-                                                         long lastTxId,
-                                                         int recordCount,
-                                                         long logSegmentSeqNo,
-                                                         long lastEntryId,
-                                                         long lastSlotId) {
-        return completedLogSegment(ledgerPath, ledgerId, firstTxId, lastTxId,
-                recordCount, logSegmentSeqNo, lastEntryId, lastSlotId,
-                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
-    }
-
-    public static LogSegmentMetadata completedLogSegment(String ledgerPath,
-                                                         long ledgerId,
-                                                         long firstTxId,
-                                                         long lastTxId,
-                                                         int recordCount,
-                                                         long logSegmentSeqNo,
-                                                         long lastEntryId,
-                                                         long lastSlotId,
-                                                         int version) {
-        LogSegmentMetadata metadata =
-                new LogSegmentMetadata.LogSegmentMetadataBuilder(
-                        ledgerPath + "/" + inprogressZNodeName(logSegmentSeqNo),
-                        version,
-                        ledgerId,
-                        firstTxId)
-                    .setInprogress(false)
-                    .setLogSegmentSequenceNo(logSegmentSeqNo)
-                    .build();
-        return metadata.completeLogSegment(ledgerPath + "/" + completedLedgerZNodeNameWithLogSegmentSequenceNumber(logSegmentSeqNo),
-                lastTxId, recordCount, lastEntryId, lastSlotId, firstTxId);
-    }
-
-    public static void generateCompletedLogSegments(DistributedLogManager manager, DistributedLogConfiguration conf,
-                                                    long numCompletedSegments, long segmentSize) throws Exception {
-        BKDistributedLogManager dlm = (BKDistributedLogManager) manager;
-        long txid = 1L;
-        for (long i = 0; i < numCompletedSegments; i++) {
-            BKSyncLogWriter writer = dlm.startLogSegmentNonPartitioned();
-            for (long j = 1; j <= segmentSize; j++) {
-                writer.write(DLMTestUtil.getLogRecordInstance(txid++));
-            }
-            writer.closeAndComplete();
-        }
-    }
-
-    public static long generateLogSegmentNonPartitioned(DistributedLogManager dlm, int controlEntries, int userEntries, long startTxid)
-            throws Exception {
-        return generateLogSegmentNonPartitioned(dlm, controlEntries, userEntries, startTxid, 1L);
-    }
-
-    public static long generateLogSegmentNonPartitioned(DistributedLogManager dlm, int controlEntries, int userEntries, long startTxid, long txidStep) throws Exception {
-        AsyncLogWriter out = dlm.startAsyncLogSegmentNonPartitioned();
-        long txid = startTxid;
-        for (int i = 0; i < controlEntries; ++i) {
-            LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
-            record.setControl();
-            Await.result(out.write(record));
-            txid += txidStep;
-        }
-        for (int i = 0; i < userEntries; ++i) {
-            LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
-            Await.result(out.write(record));
-            txid += txidStep;
-        }
-        Utils.close(out);
-        return txid - startTxid;
-    }
-
-    public static ZooKeeperClient getZooKeeperClient(BKDistributedLogManager dlm) {
-        return ((BKNamespaceDriver) dlm.getNamespaceDriver()).getWriterZKC();
-    }
-
-    public static BookKeeperClient getBookKeeperClient(BKDistributedLogManager dlm) {
-        return ((BKNamespaceDriver) dlm.getNamespaceDriver()).getReaderBKC();
-    }
-
-    public static void injectLogSegmentWithGivenLogSegmentSeqNo(DistributedLogManager manager, DistributedLogConfiguration conf,
-                                                                long logSegmentSeqNo, long startTxID, boolean writeEntries, long segmentSize,
-                                                                boolean completeLogSegment)
-            throws Exception {
-        BKDistributedLogManager dlm = (BKDistributedLogManager) manager;
-        BKLogWriteHandler writeHandler = dlm.createWriteHandler(false);
-        FutureUtils.result(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(),
-                conf.getAckQuorumSize(), BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes());
-        String inprogressZnodeName = writeHandler.inprogressZNodeName(lh.getId(), startTxID, logSegmentSeqNo);
-        String znodePath = writeHandler.inprogressZNode(lh.getId(), startTxID, logSegmentSeqNo);
-        int logSegmentMetadataVersion = conf.getDLLedgerMetadataLayoutVersion();
-        LogSegmentMetadata l =
-            new LogSegmentMetadata.LogSegmentMetadataBuilder(znodePath,
-                    logSegmentMetadataVersion, lh.getId(), startTxID)
-                .setLogSegmentSequenceNo(logSegmentSeqNo)
-                .setEnvelopeEntries(LogSegmentMetadata.supportsEnvelopedEntries(logSegmentMetadataVersion))
-                .build();
-        l.write(getZooKeeperClient(dlm));
-        writeHandler.maxTxId.update(Version.ANY, startTxID);
-        writeHandler.addLogSegmentToCache(inprogressZnodeName, l);
-        BKLogSegmentWriter writer = new BKLogSegmentWriter(
-                writeHandler.getFullyQualifiedName(),
-                inprogressZnodeName,
-                conf,
-                conf.getDLLedgerMetadataLayoutVersion(),
-                new BKLogSegmentEntryWriter(lh),
-                writeHandler.lock,
-                startTxID,
-                logSegmentSeqNo,
-                writeHandler.scheduler,
-                writeHandler.statsLogger,
-                writeHandler.statsLogger,
-                writeHandler.alertStatsLogger,
-                PermitLimiter.NULL_PERMIT_LIMITER,
-                new SettableFeatureProvider("", 0),
-                ConfUtils.getConstDynConf(conf));
-        if (writeEntries) {
-            long txid = startTxID;
-            for (long j = 1; j <= segmentSize; j++) {
-                writer.write(DLMTestUtil.getLogRecordInstance(txid++));
-            }
-            FutureUtils.result(writer.flushAndCommit());
-        }
-        if (completeLogSegment) {
-            FutureUtils.result(writeHandler.completeAndCloseLogSegment(writer));
-        }
-        FutureUtils.result(writeHandler.unlockHandler());
-    }
-
-    public static void injectLogSegmentWithLastDLSN(DistributedLogManager manager, DistributedLogConfiguration conf,
-                                                    long logSegmentSeqNo, long startTxID, long segmentSize,
-                                                    boolean recordWrongLastDLSN) throws Exception {
-        BKDistributedLogManager dlm = (BKDistributedLogManager) manager;
-        BKLogWriteHandler writeHandler = dlm.createWriteHandler(false);
-        FutureUtils.result(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(),
-                conf.getAckQuorumSize(), BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes());
-        String inprogressZnodeName = writeHandler.inprogressZNodeName(lh.getId(), startTxID, logSegmentSeqNo);
-        String znodePath = writeHandler.inprogressZNode(lh.getId(), startTxID, logSegmentSeqNo);
-        LogSegmentMetadata l =
-            new LogSegmentMetadata.LogSegmentMetadataBuilder(znodePath,
-                conf.getDLLedgerMetadataLayoutVersion(), lh.getId(), startTxID)
-            .setLogSegmentSequenceNo(logSegmentSeqNo)
-            .setInprogress(false)
-            .build();
-        l.write(getZooKeeperClient(dlm));
-        writeHandler.maxTxId.update(Version.ANY, startTxID);
-        writeHandler.addLogSegmentToCache(inprogressZnodeName, l);
-        BKLogSegmentWriter writer = new BKLogSegmentWriter(
-                writeHandler.getFullyQualifiedName(),
-                inprogressZnodeName,
-                conf,
-                conf.getDLLedgerMetadataLayoutVersion(),
-                new BKLogSegmentEntryWriter(lh),
-                writeHandler.lock,
-                startTxID,
-                logSegmentSeqNo,
-                writeHandler.scheduler,
-                writeHandler.statsLogger,
-                writeHandler.statsLogger,
-                writeHandler.alertStatsLogger,
-                PermitLimiter.NULL_PERMIT_LIMITER,
-                new SettableFeatureProvider("", 0),
-                ConfUtils.getConstDynConf(conf));
-        long txid = startTxID;
-        DLSN wrongDLSN = null;
-        for (long j = 1; j <= segmentSize; j++) {
-            DLSN dlsn = Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(txid++)));
-            if (j == (segmentSize - 1)) {
-                wrongDLSN = dlsn;
-            }
-        }
-        assertNotNull(wrongDLSN);
-        if (recordWrongLastDLSN) {
-            FutureUtils.result(writer.asyncClose());
-            writeHandler.completeAndCloseLogSegment(
-                    writeHandler.inprogressZNodeName(writer.getLogSegmentId(), writer.getStartTxId(), writer.getLogSegmentSequenceNumber()),
-                    writer.getLogSegmentSequenceNumber(),
-                    writer.getLogSegmentId(),
-                    writer.getStartTxId(),
-                    startTxID + segmentSize - 2,
-                    writer.getPositionWithinLogSegment() - 1,
-                    wrongDLSN.getEntryId(),
-                    wrongDLSN.getSlotId());
-        } else {
-            FutureUtils.result(writeHandler.completeAndCloseLogSegment(writer));
-        }
-        FutureUtils.result(writeHandler.unlockHandler());
-    }
-
-    public static void updateSegmentMetadata(ZooKeeperClient zkc, LogSegmentMetadata segment) throws Exception {
-        byte[] finalisedData = segment.getFinalisedData().getBytes(UTF_8);
-        zkc.get().setData(segment.getZkPath(), finalisedData, -1);
-    }
-
-    public static ServerConfiguration loadTestBkConf() {
-        ServerConfiguration conf = new ServerConfiguration();
-        ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
-        URL confUrl = classLoader.getResource("bk_server.conf");
-        try {
-            if (null != confUrl) {
-                conf.loadConf(confUrl);
-                LOG.info("loaded bk_server.conf from resources");
-            }
-        } catch (org.apache.commons.configuration.ConfigurationException ex) {
-            LOG.warn("loading conf failed", ex);
-        }
-        conf.setAllowLoopback(true);
-        return conf;
-    }
-
-    public static <T> void validateFutureFailed(Future<T> future, Class exClass) {
-        try {
-            Await.result(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 {
-        try {
-            return Await.result(future, Duration.fromSeconds(10));
-        } catch (Exception ex) {
-            fail("unexpected exception " + ex.getClass().getName());
-            throw ex;
-        }
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/NonBlockingReadsTestUtil.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/NonBlockingReadsTestUtil.java
deleted file mode 100644
index 3a1ab88..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/NonBlockingReadsTestUtil.java
+++ /dev/null
@@ -1,145 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.exceptions.LogEmptyException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.LogReadException;
-import com.twitter.distributedlog.util.FutureUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.*;
-
-/**
- * Utils for non blocking reads tests
- */
-class NonBlockingReadsTestUtil {
-
-    static final Logger LOG = LoggerFactory.getLogger(NonBlockingReadsTestUtil.class);
-
-    static final long DEFAULT_SEGMENT_SIZE = 1000;
-
-    static void readNonBlocking(DistributedLogManager dlm, boolean forceStall) throws Exception {
-        readNonBlocking(dlm, forceStall, DEFAULT_SEGMENT_SIZE, false);
-    }
-
-    static void readNonBlocking(DistributedLogManager dlm,
-                                boolean forceStall,
-                                long segmentSize,
-                                boolean waitForIdle) throws Exception {
-        BKSyncLogReader reader = null;
-        try {
-            reader = (BKSyncLogReader) dlm.getInputStream(1);
-        } catch (LogNotFoundException lnfe) {
-        }
-        while (null == reader) {
-            TimeUnit.MILLISECONDS.sleep(20);
-            try {
-                reader = (BKSyncLogReader) dlm.getInputStream(1);
-            } catch (LogNotFoundException lnfe) {
-            } catch (LogEmptyException lee) {
-            } catch (IOException ioe) {
-                LOG.error("Failed to open reader reading from {}", dlm.getStreamName());
-                throw ioe;
-            }
-        }
-        try {
-            LOG.info("Created reader reading from {}", dlm.getStreamName());
-            if (forceStall) {
-                reader.getReadHandler().disableReadAheadLogSegmentsNotification();
-            }
-
-            long numTrans = 0;
-            long lastTxId = -1;
-
-            boolean exceptionEncountered = false;
-            try {
-                while (true) {
-                    LogRecordWithDLSN record = reader.readNext(true);
-                    if (null != record) {
-                        DLMTestUtil.verifyLogRecord(record);
-                        assertTrue(lastTxId < record.getTransactionId());
-                        assertEquals(record.getTransactionId() - 1, record.getSequenceId());
-                        lastTxId = record.getTransactionId();
-                        numTrans++;
-                        continue;
-                    }
-
-                    if (numTrans >= (3 * segmentSize)) {
-                        if (waitForIdle) {
-                            while (true) {
-                                reader.readNext(true);
-                                TimeUnit.MILLISECONDS.sleep(10);
-                            }
-                        }
-                        break;
-                    }
-
-                    TimeUnit.MILLISECONDS.sleep(2);
-                }
-            } catch (LogReadException readexc) {
-                exceptionEncountered = true;
-            } catch (LogNotFoundException exc) {
-                exceptionEncountered = true;
-            }
-            assertFalse(exceptionEncountered);
-        } finally {
-            reader.close();
-        }
-    }
-
-    static void writeRecordsForNonBlockingReads(DistributedLogConfiguration conf,
-                                         DistributedLogManager dlm,
-                                         boolean recover)
-            throws Exception {
-        writeRecordsForNonBlockingReads(conf, dlm, recover, DEFAULT_SEGMENT_SIZE);
-    }
-
-    static void writeRecordsForNonBlockingReads(DistributedLogConfiguration conf,
-                                         DistributedLogManager dlm,
-                                         boolean recover,
-                                         long segmentSize)
-            throws Exception {
-        long txId = 1;
-        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++)));
-            }
-            if (recover) {
-                FutureUtils.result(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());
-                LOG.debug("Recovered Segments");
-            } else {
-                FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
-                writer.closeAndComplete();
-            }
-            TimeUnit.MILLISECONDS.sleep(300);
-        }
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAppendOnlyStreamReader.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAppendOnlyStreamReader.java
deleted file mode 100644
index 6b723f8..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAppendOnlyStreamReader.java
+++ /dev/null
@@ -1,207 +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 com.twitter.distributedlog;
-
-import java.io.ByteArrayInputStream;
-import java.net.URI;
-import java.util.Arrays;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.junit.Assert.*;
-
-public class TestAppendOnlyStreamReader extends TestDistributedLogBase {
-    static final Logger LOG = LoggerFactory.getLogger(TestAppendOnlyStreamReader.class);
-
-    @Rule
-    public TestName testNames = new TestName();
-
-    // Simple test subroutine writes some records, reads some back, skips ahead, skips back.
-    public void skipForwardThenSkipBack(String name, DistributedLogConfiguration conf) throws Exception {
-        DistributedLogManager dlmwrite = createNewDLM(conf, name);
-        DistributedLogManager dlmreader = createNewDLM(conf, name);
-
-        long txid = 1;
-        AppendOnlyStreamWriter writer = dlmwrite.getAppendOnlyStreamWriter();
-        writer.write(DLMTestUtil.repeatString("abc", 5).getBytes());
-        writer.write(DLMTestUtil.repeatString("abc", 5).getBytes());
-        writer.write(DLMTestUtil.repeatString("def", 5).getBytes());
-        writer.write(DLMTestUtil.repeatString("def", 5).getBytes());
-        writer.write(DLMTestUtil.repeatString("ghi", 5).getBytes());
-        writer.write(DLMTestUtil.repeatString("ghi", 5).getBytes());
-        writer.force(false);
-        writer.close();
-
-        AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
-        byte[] bytesIn = new byte[30];
-
-        byte[] bytes1 = DLMTestUtil.repeatString("abc", 10).getBytes();
-        byte[] bytes2 = DLMTestUtil.repeatString("def", 10).getBytes();
-        byte[] bytes3 = DLMTestUtil.repeatString("ghi", 10).getBytes();
-
-        int read = reader.read(bytesIn, 0, 30);
-        assertEquals(30, read);
-        assertTrue(Arrays.equals(bytes1, bytesIn));
-
-        reader.skipTo(60);
-        read = reader.read(bytesIn, 0, 30);
-        assertEquals(30, read);
-        assertTrue(Arrays.equals(bytes3, bytesIn));
-
-        reader.skipTo(30);
-        read = reader.read(bytesIn, 0, 30);
-        assertEquals(30, read);
-        assertTrue(Arrays.equals(bytes2, bytesIn));
-    }
-
-    @Test(timeout = 60000)
-    public void testSkipToSkipsBytesWithImmediateFlush() throws Exception {
-        String name = testNames.getMethodName();
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setOutputBufferSize(0);
-
-        skipForwardThenSkipBack(name, confLocal);
-    }
-
-    @Test(timeout = 60000)
-    public void testSkipToSkipsBytesWithLargerLogRecords() throws Exception {
-        String name = testNames.getMethodName();
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(1024*100);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(1000*60);
-
-        skipForwardThenSkipBack(name, confLocal);
-    }
-
-    @Test(timeout = 60000)
-    public void testSkipToSkipsBytesUntilEndOfStream() throws Exception {
-        String name = testNames.getMethodName();
-
-        DistributedLogManager dlmwrite = createNewDLM(conf, name);
-        DistributedLogManager dlmreader = createNewDLM(conf, name);
-
-        long txid = 1;
-        AppendOnlyStreamWriter writer = dlmwrite.getAppendOnlyStreamWriter();
-        writer.write(DLMTestUtil.repeatString("abc", 5).getBytes());
-        writer.markEndOfStream();
-        writer.force(false);
-        writer.close();
-
-        AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
-        byte[] bytesIn = new byte[9];
-
-        int read = reader.read(bytesIn, 0, 9);
-        assertEquals(9, read);
-        assertTrue(Arrays.equals(DLMTestUtil.repeatString("abc", 3).getBytes(), bytesIn));
-
-        assertTrue(reader.skipTo(15));
-
-        try {
-            read = reader.read(bytesIn, 0, 1);
-            fail("Should have thrown");
-        } catch (EndOfStreamException ex) {
-        }
-
-        assertTrue(reader.skipTo(0));
-
-        try {
-            reader.skipTo(16);
-            fail("Should have thrown");
-        } catch (EndOfStreamException ex) {
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testSkipToreturnsFalseIfPositionDoesNotExistYetForUnSealedStream() throws Exception {
-        String name = testNames.getMethodName();
-
-        DistributedLogManager dlmwrite = createNewDLM(conf, name);
-        DistributedLogManager dlmreader = createNewDLM(conf, name);
-
-        long txid = 1;
-        AppendOnlyStreamWriter writer = dlmwrite.getAppendOnlyStreamWriter();
-        writer.write(DLMTestUtil.repeatString("abc", 5).getBytes());
-        writer.close();
-
-        final AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
-        byte[] bytesIn = new byte[9];
-
-        int read = reader.read(bytesIn, 0, 9);
-        assertEquals(9, read);
-        assertTrue(Arrays.equals(DLMTestUtil.repeatString("abc", 3).getBytes(), bytesIn));
-
-        assertFalse(reader.skipTo(16));
-        assertFalse(reader.skipTo(16));
-
-        AppendOnlyStreamWriter writer2 = dlmwrite.getAppendOnlyStreamWriter();
-        writer2.write(DLMTestUtil.repeatString("abc", 5).getBytes());
-        writer2.close();
-
-        assertTrue(reader.skipTo(16));
-
-        byte[] bytesIn2 = new byte[5];
-        read = reader.read(bytesIn2, 0, 5);
-        assertEquals(5, read);
-        assertTrue(Arrays.equals("bcabc".getBytes(), bytesIn2));
-    }
-
-    @Test(timeout = 60000)
-    public void testSkipToForNoPositionChange() throws Exception {
-        String name = testNames.getMethodName();
-
-        DistributedLogManager dlmwrite = createNewDLM(conf, name);
-        DistributedLogManager dlmreader = createNewDLM(conf, name);
-
-        long txid = 1;
-        AppendOnlyStreamWriter writer = dlmwrite.getAppendOnlyStreamWriter();
-        writer.write(DLMTestUtil.repeatString("abc", 5).getBytes());
-        writer.close();
-
-        final AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
-
-        assertTrue(reader.skipTo(0));
-
-        byte[] bytesIn = new byte[4];
-        int read = reader.read(bytesIn, 0, 4);
-        assertEquals(4, read);
-        assertEquals(new String("abca"), new String(bytesIn));
-
-        assertTrue(reader.skipTo(reader.position()));
-
-        assertTrue(reader.skipTo(1));
-
-        read = reader.read(bytesIn, 0, 4);
-        assertEquals(4, read);
-        assertEquals(new String("bcab"), new String(bytesIn));
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAppendOnlyStreamWriter.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAppendOnlyStreamWriter.java
deleted file mode 100644
index 6aa38c3..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAppendOnlyStreamWriter.java
+++ /dev/null
@@ -1,337 +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 com.twitter.distributedlog;
-
-import java.io.ByteArrayInputStream;
-import java.net.URI;
-
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.util.FutureUtils;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.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;
-
-import static org.junit.Assert.*;
-
-public class TestAppendOnlyStreamWriter extends TestDistributedLogBase {
-    static final Logger LOG = LoggerFactory.getLogger(TestAppendOnlyStreamWriter.class);
-
-    @Rule
-    public TestName testNames = new TestName();
-
-    @Test(timeout = 60000)
-    public void testBasicReadAndWriteBehavior() throws Exception {
-        String name = testNames.getMethodName();
-        DistributedLogManager dlmwrite = createNewDLM(conf, name);
-        DistributedLogManager dlmreader = createNewDLM(conf, name);
-        byte[] byteStream = DLMTestUtil.repeatString("abc", 51).getBytes();
-
-        long txid = 1;
-        AppendOnlyStreamWriter writer = dlmwrite.getAppendOnlyStreamWriter();
-        writer.write(DLMTestUtil.repeatString("abc", 11).getBytes());
-        writer.write(DLMTestUtil.repeatString("abc", 40).getBytes());
-        writer.force(false);
-        writer.close();
-        AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
-
-        byte[] bytesIn = new byte[byteStream.length];
-        int read = reader.read(bytesIn, 0, 23);
-        assertEquals(23, read);
-        read = reader.read(bytesIn, 23, 31);
-        assertEquals(read, 31);
-        byte[] bytesInTemp = new byte[byteStream.length];
-        read = reader.read(bytesInTemp, 0, byteStream.length);
-        assertEquals(read, byteStream.length - 23 - 31);
-        read = new ByteArrayInputStream(bytesInTemp).read(bytesIn, 23 + 31, byteStream.length - 23 - 31);
-        assertEquals(read, byteStream.length - 23 - 31);
-        assertArrayEquals(bytesIn, byteStream);
-        reader.close();
-        dlmreader.close();
-        dlmwrite.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteFutureDoesNotCompleteUntilWritePersisted() throws Exception {
-        String name = testNames.getMethodName();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.setPeriodicFlushFrequencyMilliSeconds(Integer.MAX_VALUE);
-        conf.setImmediateFlushEnabled(false);
-
-        DistributedLogManager dlmwriter = createNewDLM(conf, name);
-        DistributedLogManager dlmreader = createNewDLM(conf, name);
-        byte[] byteStream = DLMTestUtil.repeatString("abc", 51).getBytes();
-
-        // Can't reliably test the future is not completed until fsync is called, since writer.force may just
-        // 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());
-
-        // The real problem is the fsync completes before writes are submitted, so it never takes effect.
-        Thread.sleep(1000);
-        assertFalse(dlsnFuture.isDefined());
-        writer.force(false);
-        // Must not throw.
-        Await.result(dlsnFuture, Duration.fromSeconds(5));
-        writer.close();
-        dlmwriter.close();
-
-        AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
-        byte[] bytesIn = new byte[byteStream.length];
-        int read = reader.read(bytesIn, 0, 31);
-        assertEquals(31, read);
-        reader.close();
-        dlmreader.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testPositionUpdatesOnlyAfterWriteCompletion() throws Exception {
-        String name = testNames.getMethodName();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.setPeriodicFlushFrequencyMilliSeconds(10*1000);
-        conf.setImmediateFlushEnabled(false);
-
-        DistributedLogManager dlmwriter = createNewDLM(conf, name);
-        DistributedLogManager dlmreader = createNewDLM(conf, name);
-        byte[] byteStream = DLMTestUtil.repeatString("abc", 11).getBytes();
-
-        // Can't reliably test the future is not completed until fsync is called, since writer.force may just
-        // 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);
-        Thread.sleep(100);
-
-        // Write hasn't been persisted, position better not be updated.
-        assertFalse(dlsnFuture.isDefined());
-        assertEquals(0, writer.position());
-        writer.force(false);
-        // Position guaranteed to be accurate after writer.force().
-        assertEquals(byteStream.length, writer.position());
-
-        // Close writer.
-        writer.close();
-        dlmwriter.close();
-
-        // Make sure we can read it.
-        AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
-        byte[] bytesIn = new byte[byteStream.length];
-        int read = reader.read(bytesIn, 0, byteStream.length);
-        assertEquals(byteStream.length, read);
-        assertEquals(byteStream.length, reader.position());
-        reader.close();
-        dlmreader.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testPositionDoesntUpdateBeforeWriteCompletion() throws Exception {
-        String name = testNames.getMethodName();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-
-        // Long flush time, but we don't wait for it.
-        conf.setPeriodicFlushFrequencyMilliSeconds(100*1000);
-        conf.setImmediateFlushEnabled(false);
-        conf.setOutputBufferSize(1024*1024);
-
-        DistributedLogManager dlmwriter = createNewDLM(conf, name);
-        byte[] byteStream = DLMTestUtil.repeatString("abc", 11).getBytes();
-
-        AppendOnlyStreamWriter writer = dlmwriter.getAppendOnlyStreamWriter();
-        assertEquals(0, writer.position());
-
-        // 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);
-        assertEquals(0, writer.position());
-
-        writer.close();
-        dlmwriter.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testPositionUpdatesOnlyAfterWriteCompletionWithoutFsync() throws Exception {
-        String name = testNames.getMethodName();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.setPeriodicFlushFrequencyMilliSeconds(1*1000);
-        conf.setImmediateFlushEnabled(false);
-        conf.setOutputBufferSize(1024*1024);
-
-        DistributedLogManager dlmwriter = createNewDLM(conf, name);
-        byte[] byteStream = DLMTestUtil.repeatString("abc", 11).getBytes();
-
-        AppendOnlyStreamWriter writer = dlmwriter.getAppendOnlyStreamWriter();
-        assertEquals(0, writer.position());
-
-        Await.result(writer.write(byteStream));
-        assertEquals(33, writer.position());
-
-        writer.close();
-        dlmwriter.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testWriterStartsAtTxidZeroForEmptyStream() throws Exception {
-        String name = testNames.getMethodName();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.setImmediateFlushEnabled(true);
-        conf.setOutputBufferSize(1024);
-        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
-
-        URI uri = createDLMURI("/" + name);
-        FutureUtils.result(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));
-
-        writer.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testOffsetGapAfterSegmentWriterFailure() throws Exception {
-        String name = testNames.getMethodName();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.setImmediateFlushEnabled(false);
-        conf.setPeriodicFlushFrequencyMilliSeconds(60*1000);
-        conf.setOutputBufferSize(1024*1024);
-        conf.setLogSegmentSequenceNumberValidationEnabled(false);
-
-        final int WRITE_LEN = 5;
-        final int SECTION_WRITES = 10;
-        long read = writeRecordsAndReadThemBackAfterInjectingAFailedTransmit(conf, name, WRITE_LEN, SECTION_WRITES);
-        assertEquals((2*SECTION_WRITES + 1)*WRITE_LEN, read);
-    }
-
-    @Test(timeout = 60000)
-    public void testNoOffsetGapAfterSegmentWriterFailure() throws Exception {
-        String name = testNames.getMethodName();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.setImmediateFlushEnabled(false);
-        conf.setPeriodicFlushFrequencyMilliSeconds(60*1000);
-        conf.setOutputBufferSize(1024*1024);
-        conf.setDisableRollingOnLogSegmentError(true);
-
-        final int WRITE_LEN = 5;
-        final int SECTION_WRITES = 10;
-
-        try {
-            writeRecordsAndReadThemBackAfterInjectingAFailedTransmit(conf, name, WRITE_LEN, SECTION_WRITES);
-            fail("should have thrown");
-        } catch (BKTransmitException ex) {
-            ;
-        }
-
-        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
-        long length = dlm.getLastTxId();
-        long read = read(dlm, length);
-        assertEquals(length, read);
-    }
-
-    long writeRecordsAndReadThemBackAfterInjectingAFailedTransmit(
-            DistributedLogConfiguration conf,
-            String name,
-            int writeLen,
-            int sectionWrites)
-            throws Exception {
-
-        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
-
-        URI uri = createDLMURI("/" + name);
-        FutureUtils.result(dlm.getWriterMetadataStore().getLog(uri, name, true, true));
-
-        // Log exists but is empty, better not throw.
-        AppendOnlyStreamWriter writer = dlm.getAppendOnlyStreamWriter();
-        byte[] byteStream = DLMTestUtil.repeatString("A", writeLen).getBytes();
-
-        // Log a hundred entries. Offset is advanced accordingly.
-        for (int i = 0; i < sectionWrites; i++) {
-            writer.write(byteStream);
-        }
-        writer.force(false);
-
-        long read = read(dlm, 1*sectionWrites*writeLen);
-        assertEquals(1*sectionWrites*writeLen, read);
-
-        // Now write another 100, but trigger failure during transmit.
-        for (int i = 0; i < sectionWrites; i++) {
-            writer.write(byteStream);
-        }
-
-        try {
-            FailpointUtils.setFailpoint(
-                FailpointUtils.FailPointName.FP_TransmitFailGetBuffer,
-                FailpointUtils.FailPointActions.FailPointAction_Throw);
-
-            writer.force(false);
-            fail("should have thown ⊙﹏⊙");
-        } catch (WriteException we) {
-            ;
-        } finally {
-            FailpointUtils.removeFailpoint(
-                FailpointUtils.FailPointName.FP_TransmitFailGetBuffer);
-        }
-
-        // This actually fails because we try to close an errored out stream.
-        writer.write(byteStream);
-
-        // Writing another 100 triggers offset gap.
-        for (int i = 0; i < sectionWrites; i++) {
-            writer.write(byteStream);
-        }
-
-        writer.force(false);
-        writer.markEndOfStream();
-        writer.close();
-
-        long length = dlm.getLastTxId();
-        assertEquals(3*sectionWrites*writeLen+5, length);
-        read = read(dlm, length);
-        dlm.close();
-        return read;
-    }
-
-    long read(DistributedLogManager dlm, long n) throws Exception {
-        AppendOnlyStreamReader reader = dlm.getAppendOnlyStreamReader();
-        byte[] bytesIn = new byte[1];
-        long offset = 0;
-        try {
-            while (offset < n) {
-                int read = reader.read(bytesIn, 0, 1);
-                offset += read;
-            }
-        } catch (EndOfStreamException ex) {
-            LOG.info("Caught ex", ex);
-        } finally {
-            reader.close();
-        }
-        return offset;
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncBulkWrite.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncBulkWrite.java
deleted file mode 100644
index e4fc5e3..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncBulkWrite.java
+++ /dev/null
@@ -1,351 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.WriteCancelledException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.List;
-
-import static com.twitter.distributedlog.DLMTestUtil.validateFutureFailed;
-import static com.twitter.distributedlog.DLMTestUtil.validateFutureSucceededAndGetResult;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
-import static org.junit.Assert.*;
-
-/**
- * Test cases for bulk writes.
- */
-public class TestAsyncBulkWrite extends TestDistributedLogBase {
-
-    static final Logger LOG = LoggerFactory.getLogger(TestAsyncBulkWrite.class);
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    protected final DistributedLogConfiguration testConf;
-
-    public TestAsyncBulkWrite() {
-        this.testConf = new DistributedLogConfiguration();
-        this.testConf.addConfiguration(conf);
-        this.testConf.setReaderIdleErrorThresholdMillis(1200000);
-    }
-
-    /**
-     * Test Case for partial failure in a bulk write.
-     *
-     * Write a batch: 10 good records + 1 too large record + 10 good records.
-     *
-     * Expected: first 10 good records succeed, the too-large-record will be rejected, while
-     *           the last 10 good records will be cancelled because their previous write is rejected.
-     */
-    @Test(timeout = 60000)
-    public void testAsyncBulkWritePartialFailureBufferFailure() throws Exception {
-        String name = "distrlog-testAsyncBulkWritePartialFailure";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(1024);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-
-        final int goodRecs = 10;
-
-        // Generate records: 10 good records, 1 too large record, 10 good records
-        final List<LogRecord> records = DLMTestUtil.getLargeLogRecordInstanceList(1, goodRecs);
-        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);
-
-        // One future returned for each write.
-        assertEquals(2*goodRecs + 1, results.size());
-
-        // First goodRecs are good.
-        for (int i = 0; i < goodRecs; i++) {
-            DLSN dlsn = validateFutureSucceededAndGetResult(results.get(i));
-        }
-
-        // First failure is log rec too big.
-        validateFutureFailed(results.get(goodRecs), LogRecordTooLongException.class);
-
-        // Rest are WriteCancelledException.
-        for (int i = goodRecs+1; i < 2*goodRecs+1; i++) {
-            validateFutureFailed(results.get(i), WriteCancelledException.class);
-        }
-
-        writer.closeAndComplete();
-        dlm.close();
-    }
-
-    /**
-     * Test Case for a total failure in a bulk write.
-     *
-     * Write 100 records as a batch. Inject failure on transmit and all records should be failed.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testAsyncBulkWriteTotalFailureTransmitFailure() throws Exception {
-        String name = "distrlog-testAsyncBulkWriteTotalFailureDueToTransmitFailure";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(1024);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-
-        final int batchSize = 100;
-        FailpointUtils.setFailpoint(
-                FailpointUtils.FailPointName.FP_TransmitComplete,
-                FailpointUtils.FailPointActions.FailPointAction_Default
-        );
-        try {
-            // Since we don't hit MAX_TRANSMISSION_SIZE, the failure is triggered on final flush, which
-            // will enqueue cancel promises task to the ordered future pool.
-            checkAllSubmittedButFailed(writer, batchSize, 1024, 1);
-        } finally {
-            FailpointUtils.removeFailpoint(
-                FailpointUtils.FailPointName.FP_TransmitComplete
-            );
-        }
-
-        writer.abort();
-        dlm.close();
-    }
-
-    /**
-     * Test Case: There is no log segment rolling when there is partial failure in async bulk write.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testAsyncBulkWriteNoLedgerRollWithPartialFailures() throws Exception {
-        String name = "distrlog-testAsyncBulkWriteNoLedgerRollWithPartialFailures";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(1024);
-        confLocal.setMaxLogSegmentBytes(1024);
-        confLocal.setLogSegmentRollingIntervalMinutes(0);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-
-        // 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 = 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;
-        records = new ArrayList<LogRecord>(2);
-        records.add(DLMTestUtil.getLogRecordInstance(txid++, 2048));
-        records.add(DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1));
-        futureResults = writer.writeBulk(records);
-        results = validateFutureSucceededAndGetResult(futureResults);
-        result = results.get(0);
-        dlsn = validateFutureSucceededAndGetResult(result);
-        assertEquals(1, dlsn.getLogSegmentSequenceNo());
-
-        // Now writer is in a bad state.
-        records = new ArrayList<LogRecord>(1);
-        records.add(DLMTestUtil.getLogRecordInstance(txid++, 2048));
-        futureResults = writer.writeBulk(records);
-        validateFutureFailed(futureResults, WriteException.class);
-
-        writer.closeAndComplete();
-        dlm.close();
-    }
-
-    /**
-     * Test Case: A large write batch will span records into multiple entries and ledgers.
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testSimpleAsyncBulkWriteSpanningEntryAndLedger() throws Exception {
-        String name = "distrlog-testSimpleAsyncBulkWriteSpanningEntryAndLedger";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(1024);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-
-        int batchSize = 100;
-        int recSize = 1024;
-
-        // First entry.
-        long ledgerIndex = 1;
-        long entryIndex = 0;
-        long slotIndex = 0;
-        long txIndex = 1;
-        checkAllSucceeded(writer, batchSize, recSize, ledgerIndex, entryIndex, slotIndex, txIndex);
-
-        // New entry.
-        entryIndex++;
-        slotIndex = 0;
-        txIndex += batchSize;
-        checkAllSucceeded(writer, batchSize, recSize, ledgerIndex, entryIndex, slotIndex, txIndex);
-
-        // Roll ledger.
-        ledgerIndex++;
-        entryIndex = 0;
-        slotIndex = 0;
-        txIndex += batchSize;
-        writer.closeAndComplete();
-        writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-        checkAllSucceeded(writer, batchSize, recSize, ledgerIndex, entryIndex, slotIndex, txIndex);
-
-        writer.closeAndComplete();
-        dlm.close();
-    }
-
-    /**
-     * Test Case: A large write batch will span multiple packets.
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testAsyncBulkWriteSpanningPackets() throws Exception {
-        String name = "distrlog-testAsyncBulkWriteSpanningPackets";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(1024);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-
-        // First entry.
-        int numTransmissions = 4;
-        int recSize = 10*1024;
-        int batchSize = (numTransmissions*MAX_LOGRECORDSET_SIZE+1)/recSize;
-        long ledgerIndex = 1;
-        long entryIndex = 0;
-        long slotIndex = 0;
-        long txIndex = 1;
-        DLSN dlsn = checkAllSucceeded(writer, batchSize, recSize, ledgerIndex, entryIndex, slotIndex, txIndex);
-        assertEquals(4, dlsn.getEntryId());
-        assertEquals(1, dlsn.getLogSegmentSequenceNo());
-
-        writer.closeAndComplete();
-        dlm.close();
-    }
-
-    /**
-     * Test Case: Test Transmit Failures when a large write batch spans multiple packets.
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testAsyncBulkWriteSpanningPacketsWithTransmitFailure() throws Exception {
-        String name = "distrlog-testAsyncBulkWriteSpanningPacketsWithTransmitFailure";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(1024);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-
-        // First entry.
-        int numTransmissions = 4;
-        int recSize = 10*1024;
-        int batchSize = (numTransmissions*MAX_LOGRECORDSET_SIZE+1)/recSize;
-        long ledgerIndex = 1;
-        long entryIndex = 0;
-        long slotIndex = 0;
-        long txIndex = 1;
-
-        DLSN dlsn = checkAllSucceeded(writer, batchSize, recSize, ledgerIndex, entryIndex, slotIndex, txIndex);
-        assertEquals(4, dlsn.getEntryId());
-        assertEquals(1, dlsn.getLogSegmentSequenceNo());
-
-        FailpointUtils.setFailpoint(
-            FailpointUtils.FailPointName.FP_TransmitComplete,
-            FailpointUtils.FailPointActions.FailPointAction_Default
-        );
-
-        try {
-            checkAllSubmittedButFailed(writer, batchSize, recSize, 1);
-        } finally {
-            FailpointUtils.removeFailpoint(
-                FailpointUtils.FailPointName.FP_TransmitComplete
-            );
-        }
-        writer.abort();
-        dlm.close();
-    }
-
-    private DLSN checkAllSucceeded(BKAsyncLogWriter writer,
-                                   int batchSize,
-                                   int recSize,
-                                   long ledgerIndex,
-                                   long entryIndex,
-                                   long slotIndex,
-                                   long txIndex) throws Exception {
-
-        List<LogRecord> records = DLMTestUtil.getLogRecordInstanceList(txIndex, batchSize, recSize);
-        Future<List<Future<DLSN>>> futureResults = writer.writeBulk(records);
-        assertNotNull(futureResults);
-        List<Future<DLSN>> results = Await.result(futureResults, Duration.fromSeconds(10));
-        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));
-            lastDlsn = dlsn;
-
-            // If we cross a transmission boundary, slot id gets reset.
-            if (dlsn.getEntryId() > prevEntryId) {
-                slotIndex = 0;
-            }
-            assertEquals(ledgerIndex, dlsn.getLogSegmentSequenceNo());
-            assertEquals(slotIndex, dlsn.getSlotId());
-            slotIndex++;
-            prevEntryId = dlsn.getEntryId();
-        }
-        return lastDlsn;
-    }
-
-    private void checkAllSubmittedButFailed(BKAsyncLogWriter writer,
-                                            int batchSize,
-                                            int recSize,
-                                            long txIndex) throws Exception {
-
-        List<LogRecord> records = DLMTestUtil.getLogRecordInstanceList(txIndex, batchSize, recSize);
-        Future<List<Future<DLSN>>> futureResults = writer.writeBulk(records);
-        assertNotNull(futureResults);
-        List<Future<DLSN>> results = Await.result(futureResults, Duration.fromSeconds(10));
-        assertNotNull(results);
-        assertEquals(results.size(), records.size());
-        for (Future<DLSN> result : results) {
-            validateFutureFailed(result, IOException.class);
-        }
-    }
-}
-
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderLock.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderLock.java
deleted file mode 100644
index 124ea77..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderLock.java
+++ /dev/null
@@ -1,607 +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 com.twitter.distributedlog;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-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 com.twitter.distributedlog.exceptions.LockCancelledException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.lock.LockClosedException;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.namespace.NamespaceDriver;
-import com.twitter.distributedlog.subscription.SubscriptionsStore;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.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;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.fail;
-
-public class TestAsyncReaderLock extends TestDistributedLogBase {
-    static final Logger LOG = LoggerFactory.getLogger(TestAsyncReaderLock.class);
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    void assertAcquiredFlagsSet(boolean[] acquiredFlags, int endIndex) {
-        for (int i = 0; i < endIndex; i++) {
-            assertTrue("reader " + i + " should have acquired lock", acquiredFlags[i]);
-        }
-        for (int i = endIndex; i < acquiredFlags.length; i++) {
-            assertFalse("reader " + i + " should not have acquired lock", acquiredFlags[i]);
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testReaderLockIfLockPathDoesntExist() throws Exception {
-        final String name = runtime.getMethodName();
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-        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());
-        assertEquals(1L, record.getTransactionId());
-        assertEquals(0L, record.getSequenceId());
-        DLMTestUtil.verifyLogRecord(record);
-
-        String readLockPath = reader1.readHandler.getReadLockPath();
-        Utils.close(reader1);
-
-        // 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());
-        assertEquals(1L, record.getTransactionId());
-        assertEquals(0L, record.getSequenceId());
-        DLMTestUtil.verifyLogRecord(record);
-    }
-
-    @Test(timeout = 60000)
-    public void testReaderLockCloseInAcquireCallback() throws Exception {
-        final String name = runtime.getMethodName();
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-        writer.write(DLMTestUtil.getLogRecordInstance(1L));
-        writer.closeAndComplete();
-
-        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) {
-                        latch.countDown();
-                        return null;
-                    }
-                });
-            }
-        });
-
-        latch.await();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReaderLockBackgroundReaderLockAcquire() throws Exception {
-        final String name = runtime.getMethodName();
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-        writer.write(DLMTestUtil.getLogRecordInstance(1L));
-        writer.closeAndComplete();
-
-        Future<AsyncLogReader> futureReader1 = dlm.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        AsyncLogReader reader1 = Await.result(futureReader1);
-        reader1.readNext();
-
-        final CountDownLatch acquiredLatch = new CountDownLatch(1);
-        final AtomicBoolean acquired = new AtomicBoolean(false);
-        Thread acquireThread = new Thread(new Runnable() {
-            @Override
-            public void run() {
-                Future<AsyncLogReader> futureReader2 = null;
-                DistributedLogManager dlm2 = null;
-                try {
-                    dlm2 = createNewDLM(conf, name);
-                    futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-                    AsyncLogReader reader2 = Await.result(futureReader2);
-                    acquired.set(true);
-                    acquiredLatch.countDown();
-                } catch (Exception ex) {
-                    fail("shouldn't reach here");
-                } finally {
-                    try {
-                        dlm2.close();
-                    } catch (Exception ex) {
-                        fail("shouldn't reach here");
-                    }
-                }
-            }
-        }, "acquire-thread");
-        acquireThread.start();
-
-        Thread.sleep(1000);
-        assertEquals(false, acquired.get());
-        Utils.close(reader1);
-
-        acquiredLatch.await();
-        assertEquals(true, acquired.get());
-        dlm.close();
-    }
-
-    int countDefined(ArrayList<Future<AsyncLogReader>> readers) {
-        int done = 0;
-        for (Future<AsyncLogReader> futureReader : readers) {
-            if (futureReader.isDefined()) {
-                done++;
-            }
-        }
-        return done;
-    }
-
-    @Test(timeout = 60000)
-    public void testReaderLockManyLocks() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-        writer.write(DLMTestUtil.getLogRecordInstance(1L));
-        writer.write(DLMTestUtil.getLogRecordInstance(2L));
-        writer.closeAndComplete();
-
-        int count = 5;
-        final CountDownLatch acquiredLatch = new CountDownLatch(count);
-        final ArrayList<Future<AsyncLogReader>> readers = new ArrayList<Future<AsyncLogReader>>(count);
-        for (int i = 0; i < count; i++) {
-            readers.add(null);
-        }
-        final DistributedLogManager[] dlms = new DistributedLogManager[count];
-        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>() {
-                @Override
-                public void onSuccess(AsyncLogReader reader) {
-                    acquiredLatch.countDown();
-                    reader.asyncClose();
-                }
-                @Override
-                public void onFailure(Throwable cause) {
-                    fail("acquire shouldnt have failed");
-                }
-            });
-        }
-
-        acquiredLatch.await();
-        for (int i = 0; i < count; i++) {
-            dlms[i].close();
-        }
-
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReaderLockDlmClosed() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogManager dlm0 = createNewDLM(conf, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
-        writer.write(DLMTestUtil.getLogRecordInstance(1L));
-        writer.write(DLMTestUtil.getLogRecordInstance(2L));
-        writer.closeAndComplete();
-
-        DistributedLogManager dlm1 = createNewDLM(conf, name);
-        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        AsyncLogReader reader1 = Await.result(futureReader1);
-
-        BKDistributedLogManager dlm2 = (BKDistributedLogManager) createNewDLM(conf, name);
-        Future<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-
-        dlm2.close();
-        try {
-            Await.result(futureReader2);
-            fail("should have thrown exception!");
-        } catch (LockClosedException ex) {
-        } catch (LockCancelledException ex) {
-        }
-
-        Utils.close(reader1);
-        dlm0.close();
-        dlm1.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReaderLockSessionExpires() throws Exception {
-        String name = runtime.getMethodName();
-        URI uri = createDLMURI("/" + name);
-        ensureURICreated(uri);
-        DistributedLogNamespace ns0 = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(uri)
-                .build();
-        DistributedLogManager dlm0 = ns0.openLog(name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
-        writer.write(DLMTestUtil.getLogRecordInstance(1L));
-        writer.write(DLMTestUtil.getLogRecordInstance(2L));
-        writer.closeAndComplete();
-
-        DistributedLogNamespace ns1 = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(uri)
-                .build();
-        DistributedLogManager dlm1 = ns1.openLog(name);
-        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        AsyncLogReader reader1 = Await.result(futureReader1);
-        ZooKeeperClientUtils.expireSession(((BKNamespaceDriver) ns1.getNamespaceDriver()).getWriterZKC(), zkServers, 1000);
-
-        // The result of expireSession is somewhat non-deterministic with this lock.
-        // It may fail with LockingException or it may succesfully reacquire, so for
-        // the moment rather than make it deterministic we accept either result.
-        boolean success = false;
-        try {
-            Await.result(reader1.readNext());
-            success = true;
-        } catch (LockingException ex) {
-        }
-        if (success) {
-            Await.result(reader1.readNext());
-        }
-
-        Utils.close(reader1);
-        dlm0.close();
-        ns0.close();
-        dlm1.close();
-        ns1.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReaderLockFutureCancelledWhileWaiting() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogManager dlm0 = createNewDLM(conf, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
-        writer.write(DLMTestUtil.getLogRecordInstance(1L));
-        writer.write(DLMTestUtil.getLogRecordInstance(2L));
-        writer.closeAndComplete();
-
-        DistributedLogManager dlm1 = createNewDLM(conf, name);
-        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        AsyncLogReader reader1 = Await.result(futureReader1);
-
-        DistributedLogManager dlm2 = createNewDLM(conf, name);
-        Future<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        try {
-            FutureUtils.cancel(futureReader2);
-            Await.result(futureReader2);
-            fail("Should fail getting log reader as it is cancelled");
-        } catch (LockClosedException ex) {
-        } catch (LockCancelledException ex) {
-        } catch (OwnershipAcquireFailedException oafe) {
-        }
-
-        futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        Utils.close(reader1);
-
-        Await.result(futureReader2);
-
-        dlm0.close();
-        dlm1.close();
-        dlm2.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReaderLockFutureCancelledWhileLocked() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogManager dlm0 = createNewDLM(conf, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
-        writer.write(DLMTestUtil.getLogRecordInstance(1L));
-        writer.write(DLMTestUtil.getLogRecordInstance(2L));
-        writer.closeAndComplete();
-
-        DistributedLogManager dlm1 = createNewDLM(conf, name);
-        Future<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());
-
-        dlm0.close();
-        dlm1.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReaderLockSharedDlmDoesNotConflict() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogManager dlm0 = createNewDLM(conf, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
-        writer.write(DLMTestUtil.getLogRecordInstance(1L));
-        writer.write(DLMTestUtil.getLogRecordInstance(2L));
-        writer.closeAndComplete();
-
-        DistributedLogManager dlm1 = createNewDLM(conf, name);
-        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        Future<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);
-
-        dlm0.close();
-        dlm1.close();
-    }
-
-    static class ReadRecordsListener implements FutureEventListener<AsyncLogReader> {
-
-        final AtomicReference<DLSN> currentDLSN;
-        final String name;
-        final ExecutorService executorService;
-
-        final CountDownLatch latch = new CountDownLatch(1);
-        boolean failed = false;
-
-        public ReadRecordsListener(AtomicReference<DLSN> currentDLSN,
-                                   String name,
-                                   ExecutorService executorService) {
-            this.currentDLSN = currentDLSN;
-            this.name = name;
-            this.executorService = executorService;
-        }
-        public CountDownLatch getLatch() {
-            return latch;
-        }
-        public boolean failed() {
-            return failed;
-        }
-        public boolean done() {
-            return latch.getCount() == 0;
-        }
-
-        @Override
-        public void onSuccess(final AsyncLogReader reader) {
-            LOG.info("Reader {} is ready to read entries", name);
-            executorService.submit(new Runnable() {
-                @Override
-                public void run() {
-                    readEntries(reader);
-                }
-            });
-        }
-
-        private void readEntries(AsyncLogReader reader) {
-            try {
-                for (int i = 0; i < 300; i++) {
-                    LogRecordWithDLSN record = Await.result(reader.readNext());
-                    currentDLSN.set(record.getDlsn());
-                }
-            } catch (Exception ex) {
-                failed = true;
-            } finally {
-                latch.countDown();
-            }
-        }
-
-        @Override
-        public void onFailure(Throwable cause) {
-            LOG.error("{} failed to open reader", name, cause);
-            failed = true;
-            latch.countDown();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testReaderLockMultiReadersScenario() throws Exception {
-        final String name = runtime.getMethodName();
-        URI uri = createDLMURI("/" + name);
-        ensureURICreated(uri);
-
-        DistributedLogConfiguration localConf = new DistributedLogConfiguration();
-        localConf.addConfiguration(conf);
-        localConf.setImmediateFlushEnabled(false);
-        localConf.setPeriodicFlushFrequencyMilliSeconds(60 * 1000);
-        localConf.setOutputBufferSize(0);
-        // Otherwise, we won't be able to run scheduled threads for readahead when we're in a callback.
-        localConf.setNumWorkerThreads(2);
-        localConf.setLockTimeout(Long.MAX_VALUE);
-
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(localConf).uri(uri).clientId("main").build();
-
-        DistributedLogManager dlm0 = namespace.openLog(name);
-        DLMTestUtil.generateCompletedLogSegments(dlm0, localConf, 9, 100);
-        dlm0.close();
-
-        int recordCount = 0;
-        AtomicReference<DLSN> currentDLSN = new AtomicReference<DLSN>(DLSN.InitialDLSN);
-
-        String clientId1 = "reader1";
-        DistributedLogNamespace namespace1 = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(localConf).uri(uri).clientId(clientId1).build();
-        DistributedLogManager dlm1 = namespace1.openLog(name);
-        String clientId2 = "reader2";
-        DistributedLogNamespace namespace2 = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(localConf).uri(uri).clientId(clientId2).build();
-        DistributedLogManager dlm2 = namespace2.openLog(name);
-        String clientId3 = "reader3";
-        DistributedLogNamespace namespace3 = DistributedLogNamespaceBuilder.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);
-        LOG.info("{} opened reader on stream {}", clientId1, name);
-
-        LOG.info("{} is opening reader on stream {}", clientId2, name);
-        Future<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-        LOG.info("{} is opening reader on stream {}", clientId3, name);
-        Future<AsyncLogReader> futureReader3 = dlm3.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
-
-        ExecutorService executorService = Executors.newCachedThreadPool();
-
-        ReadRecordsListener listener2 =
-                new ReadRecordsListener(currentDLSN, clientId2, executorService);
-        ReadRecordsListener listener3 =
-                new ReadRecordsListener(currentDLSN, clientId3, executorService);
-        futureReader2.addEventListener(listener2);
-        futureReader3.addEventListener(listener3);
-
-        // Get reader1 and start reading.
-        for ( ; recordCount < 200; recordCount++) {
-            LogRecordWithDLSN record = Await.result(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);
-        listener2.getLatch().await();
-        assertTrue(listener2.done());
-        assertTrue(listener2.failed());
-
-        // Reader1 starts reading again.
-        for (; recordCount < 300; recordCount++) {
-            LogRecordWithDLSN record = Await.result(reader1.readNext());
-            currentDLSN.set(record.getDlsn());
-        }
-
-        // Reader1 is done, someone else can take over. Since reader2 was
-        // aborted, reader3 should take its place.
-        assertFalse(listener3.done());
-        Utils.close(reader1);
-        listener3.getLatch().await();
-        assertTrue(listener3.done());
-        assertFalse(listener3.failed());
-
-        assertEquals(new DLSN(3, 99, 0), currentDLSN.get());
-
-        try {
-            Await.result(futureReader2);
-        } catch (Exception ex) {
-            // Can't get this one to close it--the dlm will take care of it.
-        }
-
-        Utils.close(Await.result(futureReader3));
-
-        dlm1.close();
-        namespace1.close();
-        dlm2.close();
-        namespace2.close();
-        dlm3.close();
-        namespace3.close();
-
-        executorService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testAsyncReadWithSubscriberId() throws Exception {
-        String name = "distrlog-asyncread-with-sbuscriber-id";
-        String subscriberId = "asyncreader";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        DLSN readDLSN = DLSN.InitialDLSN;
-
-        int txid = 1;
-        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++)));
-                if (i == 1 && j == 1L) {
-                    readDLSN = dlsn;
-                }
-            }
-            writer.closeAndComplete();
-        }
-
-        BKAsyncLogReader reader0 = (BKAsyncLogReader) Await.result(dlm.getAsyncLogReaderWithLock(subscriberId));
-        assertEquals(DLSN.NonInclusiveLowerBound, reader0.getStartDLSN());
-        long numTxns = 0;
-        LogRecordWithDLSN record = Await.result(reader0.readNext());
-        while (null != record) {
-            DLMTestUtil.verifyEmptyLogRecord(record);
-            ++numTxns;
-            assertEquals(numTxns, record.getTransactionId());
-            assertEquals(record.getTransactionId() - 1, record.getSequenceId());
-
-            if (txid - 1 == numTxns) {
-                break;
-            }
-            record = Await.result(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));
-        assertEquals(readDLSN, reader1.getStartDLSN());
-        numTxns = 0;
-        long startTxID =  10L;
-        record = Await.result(reader1.readNext());
-        while (null != record) {
-            DLMTestUtil.verifyEmptyLogRecord(record);
-            ++numTxns;
-            ++startTxID;
-            assertEquals(startTxID, record.getTransactionId());
-            assertEquals(record.getTransactionId() - 1L, record.getSequenceId());
-
-            if (startTxID == txid - 1) {
-                break;
-            }
-            record = Await.result(reader1.readNext());
-        }
-        assertEquals(txid - 1, startTxID);
-        assertEquals(20, numTxns);
-        Utils.close(reader1);
-
-        dlm.close();
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderWriter.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderWriter.java
deleted file mode 100644
index 46c8523..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestAsyncReaderWriter.java
+++ /dev/null
@@ -1,2206 +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 com.twitter.distributedlog;
-
-import java.io.IOException;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicReference;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.annotations.DistributedLogAnnotations;
-import com.twitter.distributedlog.config.ConcurrentBaseConfiguration;
-import com.twitter.distributedlog.config.ConcurrentConstConfiguration;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.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;
-import org.apache.bookkeeper.client.LedgerMetadata;
-import org.apache.bookkeeper.feature.FixedValueFeature;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.distributedlog.exceptions.IdleReaderException;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-import com.twitter.distributedlog.exceptions.ReadCancelledException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.lock.DistributedLock;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.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;
-import static com.twitter.distributedlog.DLMTestUtil.validateFutureFailed;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-import static org.junit.Assert.*;
-
-public class TestAsyncReaderWriter extends TestDistributedLogBase {
-    static final Logger LOG = LoggerFactory.getLogger(TestAsyncReaderWriter.class);
-
-    protected DistributedLogConfiguration testConf;
-
-    public TestAsyncReaderWriter() {
-        this.testConf = new DistributedLogConfiguration();
-        this.testConf.loadConf(conf);
-        this.testConf.setReaderIdleErrorThresholdMillis(1200000);
-        this.testConf.setReadAheadWaitTimeOnEndOfStream(20);
-    }
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    /**
-     * Test writing control records to writers: writers should be able to write control records, and
-     * the readers should skip control records while reading.
-     */
-    @Test(timeout = 60000)
-    public void testWriteControlRecord() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(1024);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        // Write 3 log segments. For each log segments, write one control record and nine user records.
-        int txid = 1;
-        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))));
-            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));
-            }
-            writer.closeAndComplete();
-        }
-        dlm.close();
-
-        // Read all the written data: It should skip control records and only return user records.
-        DistributedLogManager readDlm = createNewDLM(confLocal, name);
-        LogReader reader = readDlm.getInputStream(1);
-
-        long numTrans = 0;
-        long expectedTxId = 2;
-        LogRecord record = reader.readNext(false);
-        while (null != record) {
-            DLMTestUtil.verifyLargeLogRecord(record);
-            numTrans++;
-            assertEquals(expectedTxId, record.getTransactionId());
-            if (expectedTxId % 10 == 0) {
-                expectedTxId += 2;
-            } else {
-                ++expectedTxId;
-            }
-            record = reader.readNext(false);
-        }
-        reader.close();
-        assertEquals(3 * 9, numTrans);
-        assertEquals(3 * 9, readDlm.getLogRecordCount());
-        readDlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testAsyncWritePendingWritesAbortedWhenLedgerRollTriggerFails() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(1024);
-        confLocal.setMaxLogSegmentBytes(1024);
-        confLocal.setLogSegmentRollingIntervalMinutes(0);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-
-        // 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));
-        assertEquals(1, dlsn.getLogSegmentSequenceNo());
-
-        record = DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1);
-        result = writer.write(record);
-        validateFutureFailed(result, LogRecordTooLongException.class);
-
-        record = DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1);
-        result = writer.write(record);
-        validateFutureFailed(result, WriteException.class);
-
-        record = DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1);
-        result = writer.write(record);
-        validateFutureFailed(result, WriteException.class);
-
-        writer.closeAndComplete();
-        dlm.close();
-    }
-
-    /**
-     * Test Case: Simple Async Writes. Writes 30 records. They should be written correctly.
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testSimpleAsyncWrite() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(1024);
-
-        int numLogSegments = 3;
-        int numRecordsPerLogSegment = 10;
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        final CountDownLatch syncLatch = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
-        final AtomicBoolean errorsFound = new AtomicBoolean(false);
-        final AtomicReference<DLSN> maxDLSN = new AtomicReference<DLSN>(DLSN.InvalidDLSN);
-        int txid = 1;
-        for (long i = 0; i < numLogSegments; i++) {
-            final long currentLogSegmentSeqNo = i + 1;
-            BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-            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>() {
-                    @Override
-                    public void onSuccess(DLSN value) {
-                        if(value.getLogSegmentSequenceNo() != currentLogSegmentSeqNo) {
-                            LOG.debug("LogSegmentSequenceNumber: {}, Expected {}", value.getLogSegmentSequenceNo(), currentLogSegmentSeqNo);
-                            errorsFound.set(true);
-                        }
-
-                        if(value.getEntryId() != currentEntryId) {
-                            LOG.debug("EntryId: {}, Expected {}", value.getEntryId(), currentEntryId);
-                            errorsFound.set(true);
-                        }
-
-                        if (value.compareTo(maxDLSN.get()) > 0) {
-                            maxDLSN.set(value);
-                        }
-
-                        syncLatch.countDown();
-                        LOG.debug("SyncLatch: {}", syncLatch.getCount());
-                    }
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        LOG.error("Encountered exception on writing record {} in log segment {}", currentEntryId, currentLogSegmentSeqNo);
-                        errorsFound.set(true);
-                    }
-                });
-            }
-            writer.closeAndComplete();
-        }
-
-        syncLatch.await();
-        assertFalse("Should not encounter any errors for async writes", errorsFound.get());
-
-        LogRecordWithDLSN last = dlm.getLastLogRecord();
-        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()));
-        DLMTestUtil.verifyLargeLogRecord(last);
-
-        dlm.close();
-    }
-
-    /**
-     * Write records into <i>numLogSegments</i> log segments. Each log segment has <i>numRecordsPerLogSegment</i> records.
-     *
-     * @param dlm
-     *          distributedlog manager
-     * @param numLogSegments
-     *          number of log segments
-     * @param numRecordsPerLogSegment
-     *          number records per log segment
-     * @param startTxId
-     *          start tx id
-     * @return next tx id
-     */
-    private static long writeRecords(DistributedLogManager dlm,
-                                     int numLogSegments,
-                                     int numRecordsPerLogSegment,
-                                     long startTxId,
-                                     boolean emptyRecord) throws IOException {
-        long txid = startTxId;
-        for (long i = 0; i < numLogSegments; i++) {
-            BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-            for (long j = 1; j <= numRecordsPerLogSegment; j++) {
-                if (emptyRecord) {
-                    writer.write(DLMTestUtil.getEmptyLogRecordInstance(txid++));
-                } else {
-                    writer.write(DLMTestUtil.getLargeLogRecordInstance(txid++));
-                }
-            }
-            writer.closeAndComplete();
-        }
-        return txid;
-    }
-
-    /**
-     * Write <code>numRecords</code> records to the log, starting with <code>startTxId</code>.
-     * It flushes every <code>flushPerNumRecords</code> records.
-     *
-     * @param dlm
-     *          distributedlog manager
-     * @param numRecords
-     *          num records to write
-     * @param startTxId
-     *          start tx id
-     * @param flushPerNumRecords
-     *          number records to flush
-     * @return next tx id
-     * @throws IOException
-     */
-    private static long writeLogSegment(DistributedLogManager dlm,
-                                        int numRecords,
-                                        long startTxId,
-                                        int flushPerNumRecords,
-                                        boolean emptyRecord) throws IOException {
-        long txid = startTxId;
-        LogWriter writer = dlm.startLogSegmentNonPartitioned();
-        for (long j = 1; j <= numRecords; j++) {
-            if (emptyRecord) {
-                writer.write(DLMTestUtil.getEmptyLogRecordInstance(txid++));
-            } else {
-                writer.write(DLMTestUtil.getLargeLogRecordInstance(txid++));
-            }
-            if (j % flushPerNumRecords == 0 ) {
-                writer.setReadyToFlush();
-                writer.flushAndSync();
-            }
-        }
-        writer.setReadyToFlush();
-        writer.flushAndSync();
-        writer.close();
-        return txid;
-    }
-
-    private static void readNext(final AsyncLogReader reader,
-                                 final DLSN startPosition,
-                                 final long startSequenceId,
-                                 final boolean monotonic,
-                                 final CountDownLatch syncLatch,
-                                 final CountDownLatch completionLatch,
-                                 final AtomicBoolean errorsFound) {
-        Future<LogRecordWithDLSN> record = reader.readNext();
-        record.addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
-            @Override
-            public void onSuccess(LogRecordWithDLSN value) {
-                try {
-                    if (monotonic) {
-                        assertEquals(startSequenceId, value.getSequenceId());
-                    } else {
-                        assertTrue(value.getSequenceId() < 0);
-                        assertTrue(value.getSequenceId() > startSequenceId);
-                    }
-                    LOG.info("Received record {} from {}", value, reader.getStreamName());
-                    assertTrue(!value.isControl());
-                    assertTrue(value.getDlsn().getSlotId() == 0);
-                    assertTrue(value.getDlsn().compareTo(startPosition) >= 0);
-                    DLMTestUtil.verifyLargeLogRecord(value);
-                } catch (Exception exc) {
-                    LOG.debug("Exception Encountered when verifying log record {} : ", value.getDlsn(), exc);
-                    errorsFound.set(true);
-                    completionLatch.countDown();
-                    return;
-                }
-                syncLatch.countDown();
-                if (syncLatch.getCount() <= 0) {
-                    completionLatch.countDown();
-                } else {
-                    TestAsyncReaderWriter.readNext(
-                            reader,
-                            value.getDlsn().getNextDLSN(),
-                            monotonic ? value.getSequenceId() + 1 : value.getSequenceId(),
-                            monotonic,
-                            syncLatch,
-                            completionLatch,
-                            errorsFound);
-                }
-            }
-            @Override
-            public void onFailure(Throwable cause) {
-                LOG.error("Encountered Exception on reading {}", reader.getStreamName(), cause);
-                errorsFound.set(true);
-                completionLatch.countDown();
-            }
-        });
-    }
-
-    void simpleAsyncReadTest(String name, DistributedLogConfiguration confLocal) throws Exception {
-        confLocal.setOutputBufferSize(1024);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadBatchSize(10);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        int numLogSegments = 3;
-        int numRecordsPerLogSegment = 10;
-
-        // Write 30 records: 3 log segments, 10 records per log segment
-        long txid = 1L;
-        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false);
-        // Write another log segment with 5 records and flush every 2 records
-        txid = writeLogSegment(dlm, 5, txid, 2, false);
-
-        final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
-        final CountDownLatch syncLatch = new CountDownLatch((int) (txid - 1));
-        final CountDownLatch completionLatch = new CountDownLatch(1);
-        final AtomicBoolean errorsFound = new AtomicBoolean(false);
-
-        boolean monotonic = LogSegmentMetadata.supportsSequenceId(confLocal.getDLLedgerMetadataLayoutVersion());
-        TestAsyncReaderWriter.readNext(
-                reader,
-                DLSN.InvalidDLSN,
-                monotonic ? 0L : Long.MIN_VALUE,
-                monotonic,
-                syncLatch,
-                completionLatch,
-                errorsFound);
-
-        completionLatch.await();
-        assertFalse("Errors encountered on reading records", errorsFound.get());
-        syncLatch.await();
-
-        Utils.close(reader);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testSimpleAsyncRead() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        simpleAsyncReadTest(name, confLocal);
-    }
-
-    @Test(timeout = 60000)
-    public void testSimpleAsyncReadWriteWithMonitoredFuturePool() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setTaskExecutionWarnTimeMicros(1000);
-        confLocal.setEnableTaskExecutionStats(true);
-        simpleAsyncReadTest(name, confLocal);
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkAsyncRead() throws Exception {
-        String name = "distrlog-bulkasyncread";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadMaxRecords(10000);
-        confLocal.setReadAheadBatchSize(10);
-
-        int numLogSegments = 3;
-        int numRecordsPerLogSegment = 20;
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, 1L, false);
-
-        final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN);
-        int expectedTxID = 1;
-        int numReads = 0;
-        while (expectedTxID <= numLogSegments * numRecordsPerLogSegment) {
-            if (expectedTxID == numLogSegments * numRecordsPerLogSegment) {
-                break;
-            }
-            List<LogRecordWithDLSN> records = Await.result(reader.readBulk(20));
-            LOG.info("Bulk read {} entries.", records.size());
-
-            assertTrue(records.size() >= 1);
-            for (LogRecordWithDLSN record : records) {
-                assertEquals(expectedTxID, record.getTransactionId());
-                ++expectedTxID;
-            }
-            ++numReads;
-        }
-
-        // we expect bulk read works
-        assertTrue(numReads < 60);
-
-        Utils.close(reader);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkAsyncReadWithWriteBatch() throws Exception {
-        String name = "distrlog-bulkasyncread-with-writebatch";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setOutputBufferSize(1024000);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadMaxRecords(10000);
-        confLocal.setReadAheadBatchSize(10);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        int numLogSegments = 3;
-        int numRecordsPerLogSegment = 20;
-
-        writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, 1L, false);
-
-        final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN);
-        int expectedTxID = 1;
-        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));
-            assertEquals(20, records.size());
-            for (LogRecordWithDLSN record : records) {
-                assertEquals(expectedTxID, record.getTransactionId());
-                ++expectedTxID;
-            }
-        }
-
-        Utils.close(reader);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testAsyncReadEmptyRecords() throws Exception {
-        String name = "distrlog-simpleasyncreadempty";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadBatchSize(10);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        int numLogSegments = 3;
-        int numRecordsPerLogSegment = 10;
-
-        long txid = 1L;
-        // write 3 log segments, 10 records per log segment
-        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, true);
-        // write another log segment with 5 records and flush every 2 records
-        txid = writeLogSegment(dlm, 5, txid, 2, true);
-
-        AsyncLogReader asyncReader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
-        assertEquals("Expected stream name = " + name + " but " + asyncReader.getStreamName() + " found",
-                name, asyncReader.getStreamName());
-        long numTrans = 0;
-        DLSN lastDLSN = DLSN.InvalidDLSN;
-        LogRecordWithDLSN record = Await.result(asyncReader.readNext());
-        while (null != record) {
-            DLMTestUtil.verifyEmptyLogRecord(record);
-            assertEquals(0, record.getDlsn().getSlotId());
-            assertTrue(record.getDlsn().compareTo(lastDLSN) > 0);
-            lastDLSN = record.getDlsn();
-            numTrans++;
-            if (numTrans >= (txid - 1)) {
-                break;
-            }
-            record = Await.result(asyncReader.readNext());
-        }
-        assertEquals((txid - 1), numTrans);
-        Utils.close(asyncReader);
-        dlm.close();
-    }
-
-    /**
-     * Test Async Read by positioning to a given position in the log
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testSimpleAsyncReadPosition() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(1024);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadBatchSize(10);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        int numLogSegments = 3;
-        int numRecordsPerLogSegment = 10;
-
-        long txid = 1L;
-        // write 3 log segments, 10 records per log segment
-        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false);
-        // write another log segment with 5 records
-        txid = writeLogSegment(dlm, 5, txid, Integer.MAX_VALUE, false);
-
-        final CountDownLatch syncLatch = new CountDownLatch((int)(txid - 14));
-        final CountDownLatch doneLatch = new CountDownLatch(1);
-        final AtomicBoolean errorsFound = new AtomicBoolean(false);
-        final AsyncLogReader reader = dlm.getAsyncLogReader(new DLSN(2, 2, 4));
-        assertEquals(name, reader.getStreamName());
-
-        boolean monotonic = LogSegmentMetadata.supportsSequenceId(confLocal.getDLLedgerMetadataLayoutVersion());
-        TestAsyncReaderWriter.readNext(
-                reader,
-                new DLSN(2, 3, 0),
-                monotonic ? 13L : Long.MIN_VALUE,
-                monotonic,
-                syncLatch,
-                doneLatch,
-                errorsFound);
-
-        doneLatch.await();
-        assertFalse("Errors found on reading records", errorsFound.get());
-        syncLatch.await();
-
-        Utils.close(reader);
-        dlm.close();
-    }
-
-    /**
-     * Test write/read entries when immediate flush is disabled.
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testSimpleAsyncReadWrite() throws Exception {
-        testSimpleAsyncReadWriteInternal(runtime.getMethodName(), false);
-    }
-
-    /**
-     * Test write/read entries when immediate flush is enabled.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testSimpleAsyncReadWriteImmediateFlush() throws Exception {
-        testSimpleAsyncReadWriteInternal(runtime.getMethodName(), true);
-    }
-
-    /**
-     * Test if entries written using log segment metadata that doesn't support enveloping
-     * can be read correctly by a reader supporting both.
-     *
-     * NOTE: An older reader cannot read enveloped entry, so we don't have a test case covering
-     *       the other scenario.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testNoEnvelopeWriterEnvelopeReader() throws Exception {
-        testSimpleAsyncReadWriteInternal(runtime.getMethodName(), true,
-                LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value - 1);
-    }
-
-    static class WriteFutureEventListener implements FutureEventListener<DLSN> {
-        private final LogRecord record;
-        private final long currentLogSegmentSeqNo;
-        private final long currentEntryId;
-        private final CountDownLatch syncLatch;
-        private final AtomicBoolean errorsFound;
-        private final boolean verifyEntryId;
-
-        WriteFutureEventListener(LogRecord record,
-                                 long currentLogSegmentSeqNo,
-                                 long currentEntryId,
-                                 CountDownLatch syncLatch,
-                                 AtomicBoolean errorsFound,
-                                 boolean verifyEntryId) {
-            this.record = record;
-            this.currentLogSegmentSeqNo = currentLogSegmentSeqNo;
-            this.currentEntryId = currentEntryId;
-            this.syncLatch = syncLatch;
-            this.errorsFound = errorsFound;
-            this.verifyEntryId = verifyEntryId;
-        }
-
-        /**
-         * Invoked if the computation completes successfully
-         */
-        @Override
-        public void onSuccess(DLSN value) {
-            if(value.getLogSegmentSequenceNo() != currentLogSegmentSeqNo) {
-                LOG.error("Ledger Seq No: {}, Expected: {}", value.getLogSegmentSequenceNo(), currentLogSegmentSeqNo);
-                errorsFound.set(true);
-            }
-
-            if(verifyEntryId && value.getEntryId() != currentEntryId) {
-                LOG.error("EntryId: {}, Expected: {}", value.getEntryId(), currentEntryId);
-                errorsFound.set(true);
-            }
-            syncLatch.countDown();
-        }
-
-        /**
-         * Invoked if the computation completes unsuccessfully
-         */
-        @Override
-        public void onFailure(Throwable cause) {
-            LOG.error("Encountered failures on writing record as (lid = {}, eid = {}) :",
-                    new Object[]{currentLogSegmentSeqNo, currentEntryId, cause});
-            errorsFound.set(true);
-            syncLatch.countDown();
-        }
-    }
-
-    void testSimpleAsyncReadWriteInternal(String name, boolean immediateFlush)
-            throws Exception {
-        testSimpleAsyncReadWriteInternal(name, immediateFlush,
-                                         LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
-    }
-
-    void testSimpleAsyncReadWriteInternal(String name, boolean immediateFlush,
-                                          int logSegmentVersion) throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadBatchSize(10);
-        confLocal.setOutputBufferSize(1024);
-        confLocal.setDLLedgerMetadataLayoutVersion(logSegmentVersion);
-        confLocal.setImmediateFlushEnabled(immediateFlush);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        int numLogSegments = 3;
-        int numRecordsPerLogSegment = 10;
-
-        final CountDownLatch readLatch = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
-        final CountDownLatch readDoneLatch = new CountDownLatch(1);
-        final AtomicBoolean readErrors = new AtomicBoolean(false);
-        final CountDownLatch writeLatch = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
-        final AtomicBoolean writeErrors = new AtomicBoolean(false);
-        final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
-        assertEquals(name, reader.getStreamName());
-
-        int txid = 1;
-        for (long i = 0; i < 3; i++) {
-            final long currentLogSegmentSeqNo = i + 1;
-            BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-            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(
-                        record, currentLogSegmentSeqNo, currentEntryId, writeLatch, writeErrors, true));
-                if (i == 0 && j == 0) {
-                    boolean monotonic = LogSegmentMetadata.supportsSequenceId(logSegmentVersion);
-                    TestAsyncReaderWriter.readNext(
-                            reader,
-                            DLSN.InvalidDLSN,
-                            monotonic ? 0L : Long.MIN_VALUE,
-                            monotonic,
-                            readLatch,
-                            readDoneLatch,
-                            readErrors);
-                }
-            }
-            writer.closeAndComplete();
-        }
-
-        writeLatch.await();
-        assertFalse("All writes should succeed", writeErrors.get());
-
-        readDoneLatch.await();
-        assertFalse("All reads should succeed", readErrors.get());
-        readLatch.await();
-
-        Utils.close(reader);
-        dlm.close();
-    }
-
-
-    /**
-     * Test Case: starting reading when the streams don't exist.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testSimpleAsyncReadWriteStartEmpty() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadBatchSize(10);
-        confLocal.setOutputBufferSize(1024);
-
-        int numLogSegments = 3;
-        int numRecordsPerLogSegment = 10;
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        final CountDownLatch readerReadyLatch = new CountDownLatch(1);
-        final CountDownLatch readerDoneLatch = new CountDownLatch(1);
-        final CountDownLatch readerSyncLatch = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
-
-        final TestReader reader = new TestReader(
-                "test-reader",
-                dlm,
-                DLSN.InitialDLSN,
-                false,
-                0,
-                readerReadyLatch,
-                readerSyncLatch,
-                readerDoneLatch);
-
-        reader.start();
-
-        // Increase the probability of reader failure and retry
-        Thread.sleep(500);
-
-        final AtomicBoolean writeErrors = new AtomicBoolean(false);
-        final CountDownLatch writeLatch = new CountDownLatch(30);
-
-        int txid = 1;
-        for (long i = 0; i < 3; i++) {
-            final long currentLogSegmentSeqNo = i + 1;
-            BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-            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(
-                        record, currentLogSegmentSeqNo, currentEntryId, writeLatch, writeErrors, true));
-            }
-            writer.closeAndComplete();
-        }
-
-        writeLatch.await();
-        assertFalse("All writes should succeed", writeErrors.get());
-
-        readerDoneLatch.await();
-        assertFalse("Should not encounter errors during reading", reader.areErrorsFound());
-        readerSyncLatch.await();
-
-        assertTrue("Should position reader at least once", reader.getNumReaderPositions().get() > 1);
-        reader.stop();
-        dlm.close();
-    }
-
-
-    /**
-     * Test Case: starting reading when the streams don't exist.
-     * {@link https://issues.apache.org/jira/browse/DL-42}
-     */
-    @DistributedLogAnnotations.FlakyTest
-    @Ignore
-    @Test(timeout = 120000)
-    public void testSimpleAsyncReadWriteStartEmptyFactory() throws Exception {
-        // int count = 50;
-        int count = 1;
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadBatchSize(10);
-        confLocal.setOutputBufferSize(1024);
-
-        int numLogSegments = 3;
-        int numRecordsPerLogSegment = 1;
-
-        URI uri = createDLMURI("/" + name);
-        ensureURICreated(uri);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(confLocal).uri(uri).build();
-        final DistributedLogManager[] dlms = new DistributedLogManager[count];
-        final TestReader[] readers = new TestReader[count];
-        final CountDownLatch readyLatch = new CountDownLatch(count);
-        final CountDownLatch[] syncLatches = new CountDownLatch[count];
-        final CountDownLatch[] readerDoneLatches = new CountDownLatch[count];
-        for (int s = 0; s < count; s++) {
-            dlms[s] = namespace.openLog(name + String.format("%d", s));
-            readerDoneLatches[s] = new CountDownLatch(1);
-            syncLatches[s] = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
-            readers[s] = new TestReader("reader-" + s,
-                    dlms[s], DLSN.InitialDLSN, false, 0, readyLatch, syncLatches[s], readerDoneLatches[s]);
-            readers[s].start();
-        }
-
-        // wait all readers were positioned at least once
-        readyLatch.await();
-
-        final CountDownLatch writeLatch = new CountDownLatch(3 * count);
-        final AtomicBoolean writeErrors = new AtomicBoolean(false);
-
-        int txid = 1;
-        for (long i = 0; i < 3; i++) {
-            final long currentLogSegmentSeqNo = i + 1;
-            BKAsyncLogWriter[] writers = new BKAsyncLogWriter[count];
-            for (int s = 0; s < count; s++) {
-                writers[s] = (BKAsyncLogWriter)(dlms[s].startAsyncLogSegmentNonPartitioned());
-            }
-            for (long j = 0; j < 1; j++) {
-                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(
-                            record, currentLogSegmentSeqNo, currentEntryId, writeLatch, writeErrors, true));
-                }
-            }
-            for (int s = 0; s < count; s++) {
-                writers[s].closeAndComplete();
-            }
-        }
-
-        writeLatch.await();
-        assertFalse("All writes should succeed", writeErrors.get());
-
-        for (int s = 0; s < count; s++) {
-            readerDoneLatches[s].await();
-            assertFalse("Reader " + s + " should not encounter errors", readers[s].areErrorsFound());
-            syncLatches[s].await();
-            assertEquals(numLogSegments * numRecordsPerLogSegment, readers[s].getNumReads().get());
-            assertTrue("Reader " + s + " should position at least once", readers[s].getNumReaderPositions().get() > 0);
-        }
-
-        for (int s = 0; s < count; s++) {
-            readers[s].stop();
-            dlms[s].close();
-        }
-    }
-
-    /**
-     * Flaky test fixed: readers need to be added to the pendingReaders
-     * @throws Exception
-     */
-    @Test(timeout = 300000)
-    public void testSimpleAsyncReadWriteSimulateErrors() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadBatchSize(10);
-        confLocal.setOutputBufferSize(1024);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        int numLogSegments = 5;
-        int numRecordsPerLogSegment = 10;
-
-        final CountDownLatch doneLatch = new CountDownLatch(1);
-        final CountDownLatch syncLatch = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
-
-        TestReader reader = new TestReader(
-                "test-reader",
-                dlm,
-                DLSN.InitialDLSN,
-                true,
-                0,
-                new CountDownLatch(1),
-                syncLatch,
-                doneLatch);
-
-        reader.start();
-
-        final CountDownLatch writeLatch = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
-        final AtomicBoolean writeErrors = new AtomicBoolean(false);
-
-        int txid = 1;
-        for (long i = 0; i < numLogSegments; i++) {
-            final long currentLogSegmentSeqNo = i + 1;
-            BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-            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(
-                        record, currentLogSegmentSeqNo, currentEntryId, writeLatch, writeErrors, true));
-            }
-            writer.closeAndComplete();
-        }
-
-        writeLatch.await();
-        assertFalse("All writes should succeed", writeErrors.get());
-
-        doneLatch.await();
-        assertFalse("Should not encounter errors during reading", reader.areErrorsFound());
-        syncLatch.await();
-
-        assertTrue("Should position reader at least once", reader.getNumReaderPositions().get() > 1);
-        reader.stop();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testSimpleAsyncReadWritePiggyBack() throws Exception {
-        String name = runtime.getMethodName();
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setEnableReadAhead(true);
-        confLocal.setReadAheadWaitTime(500);
-        confLocal.setReadAheadBatchSize(10);
-        confLocal.setReadAheadMaxRecords(100);
-        confLocal.setOutputBufferSize(1024);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(100);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
-
-        int numLogSegments = 3;
-        int numRecordsPerLogSegment = 10;
-
-        final CountDownLatch readLatch = new CountDownLatch(30);
-        final CountDownLatch readDoneLatch = new CountDownLatch(1);
-        final AtomicBoolean readErrors = new AtomicBoolean(false);
-        final CountDownLatch writeLatch = new CountDownLatch(30);
-        final AtomicBoolean writeErrors = new AtomicBoolean(false);
-
-        int txid = 1;
-        for (long i = 0; i < numLogSegments; i++) {
-            final long currentLogSegmentSeqNo = i + 1;
-            BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-            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(
-                        record, currentLogSegmentSeqNo, j, writeLatch, writeErrors, false));
-                if (i == 0 && j == 0) {
-                    boolean monotonic = LogSegmentMetadata.supportsSequenceId(confLocal.getDLLedgerMetadataLayoutVersion());
-                    TestAsyncReaderWriter.readNext(
-                            reader,
-                            DLSN.InvalidDLSN,
-                            monotonic ? 0L : Long.MIN_VALUE,
-                            monotonic,
-                            readLatch,
-                            readDoneLatch,
-                            readErrors);
-                }
-            }
-            writer.closeAndComplete();
-        }
-
-        writeLatch.await();
-        assertFalse("All writes should succeed", writeErrors.get());
-
-        readDoneLatch.await();
-        assertFalse("All reads should succeed", readErrors.get());
-        readLatch.await();
-
-        Utils.close(reader);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testCancelReadRequestOnReaderClosed() throws Exception {
-        final String name = "distrlog-cancel-read-requests-on-reader-closed";
-
-        DistributedLogManager dlm = createNewDLM(testConf, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-        writer.write(DLMTestUtil.getLogRecordInstance(1L));
-        writer.closeAndComplete();
-
-        final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN);
-        LogRecordWithDLSN record = Await.result(reader.readNext());
-        assertEquals(1L, record.getTransactionId());
-        DLMTestUtil.verifyLogRecord(record);
-
-        final CountDownLatch readLatch = new CountDownLatch(1);
-        final AtomicBoolean receiveExpectedException = new AtomicBoolean(false);
-        Thread readThread = new Thread(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    Await.result(reader.readNext());
-                } catch (ReadCancelledException rce) {
-                    receiveExpectedException.set(true);
-                } catch (Throwable t) {
-                    LOG.error("Receive unexpected exception on reading stream {} : ", name, t);
-                }
-                readLatch.countDown();
-            }
-        }, "read-thread");
-        readThread.start();
-
-        Thread.sleep(1000);
-
-        // close reader should cancel the pending read next
-        Utils.close(reader);
-
-        readLatch.await();
-        readThread.join();
-
-        assertTrue("Read request should be cancelled.", receiveExpectedException.get());
-
-        // closed reader should reject any readNext
-        try {
-            Await.result(reader.readNext());
-            fail("Reader should reject readNext if it is closed.");
-        } catch (ReadCancelledException rce) {
-            // expected
-        }
-
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testAsyncWriteWithMinDelayBetweenFlushes() throws Exception {
-        String name = "distrlog-asyncwrite-mindelay";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setMinDelayBetweenImmediateFlushMs(100);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        final Thread currentThread = Thread.currentThread();
-        final int COUNT = 5000;
-        final CountDownLatch syncLatch = new CountDownLatch(COUNT);
-        int txid = 1;
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-        Stopwatch executionTime = Stopwatch.createStarted();
-        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>() {
-                @Override
-                public void onSuccess(DLSN value) {
-                    syncLatch.countDown();
-                    LOG.debug("SyncLatch: {} ; DLSN: {} ", syncLatch.getCount(), value);
-                }
-                @Override
-                public void onFailure(Throwable cause) {
-                    currentThread.interrupt();
-                }
-            });
-        }
-
-        boolean success = false;
-        if (!(Thread.interrupted())) {
-            try {
-                success = syncLatch.await(10, TimeUnit.SECONDS);
-            } catch (InterruptedException exc) {
-                Thread.currentThread().interrupt();
-            }
-        }
-
-        // Abort, not graceful close, since the latter will
-        // flush as well, and may add an entry.
-        writer.abort();
-
-        executionTime.stop();
-        assertTrue(!(Thread.interrupted()));
-        assertTrue(success);
-
-        LogRecordWithDLSN last = dlm.getLastLogRecord();
-        LOG.info("Last Entry {}; elapsed time {}", last.getDlsn().getEntryId(), executionTime.elapsed(TimeUnit.MILLISECONDS));
-
-        // Regardless of how many records we wrote; the number of BK entries should always be bounded by the min delay.
-        // Since there are two flush processes--data flush and control flush, and since control flush may also end up flushing
-        // data if data is available, the upper bound is 2*(time/min_delay + 1)
-        assertTrue(last.getDlsn().getEntryId() <= ((executionTime.elapsed(TimeUnit.MILLISECONDS) / confLocal.getMinDelayBetweenImmediateFlushMs() + 1))*2);
-        DLMTestUtil.verifyLogRecord(last);
-
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testAsyncWriteWithMinDelayBetweenFlushesFlushFailure() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setMinDelayBetweenImmediateFlushMs(1);
-
-        URI uri = createDLMURI("/" + name);
-        ensureURICreated(uri);
-
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(confLocal).uri(uri).clientId("gabbagoo").build();
-        DistributedLogManager dlm = namespace.openLog(name);
-        DistributedLogNamespace namespace1 = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(confLocal).uri(uri).clientId("tortellini").build();
-        DistributedLogManager dlm1 = namespace1.openLog(name);
-
-        int txid = 1;
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-
-        // First write succeeds since lock isnt checked until transmit, which is scheduled
-        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
-        writer.flushAndCommit();
-
-        BKLogSegmentWriter perStreamWriter = writer.getCachedLogWriter();
-        DistributedLock lock = perStreamWriter.getLock();
-        FutureUtils.result(lock.asyncClose());
-
-        // Get second writer, steal lock
-        BKAsyncLogWriter writer2 = (BKAsyncLogWriter)(dlm1.startAsyncLogSegmentNonPartitioned());
-
-        try {
-            // Succeeds, kicks off scheduked flush
-            writer.write(DLMTestUtil.getLogRecordInstance(txid++));
-
-            // Succeeds, kicks off scheduled flush
-            Thread.sleep(100);
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
-            fail("should have thrown");
-        } catch (LockingException ex) {
-            LOG.debug("caught exception ", ex);
-        }
-
-        writer.close();
-        dlm.close();
-    }
-
-    public void writeRecordsWithOutstandingWriteLimit(int stream, int global, boolean shouldFail) throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setPerWriterOutstandingWriteLimit(stream);
-        confLocal.setOutstandingWriteLimitDarkmode(false);
-        DistributedLogManager dlm;
-        if (global > -1) {
-            dlm = createNewDLM(confLocal, runtime.getMethodName(),
-                    new SimplePermitLimiter(false, global, new NullStatsLogger(), true, new FixedValueFeature("", 0)));
-        } else {
-            dlm = createNewDLM(confLocal, runtime.getMethodName());
-        }
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-        ArrayList<Future<DLSN>> results = new ArrayList<Future<DLSN>>(1000);
-        for (int i = 0; i < 1000; i++) {
-            results.add(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
-        }
-        for (Future<DLSN> result : results) {
-            try {
-                Await.result(result);
-                if (shouldFail) {
-                    fail("should fail due to no outstanding writes permitted");
-                }
-            } catch (OverCapacityException ex) {
-                assertTrue(shouldFail);
-            }
-        }
-        writer.closeAndComplete();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testOutstandingWriteLimitNoLimit() throws Exception {
-        writeRecordsWithOutstandingWriteLimit(-1, -1, false);
-    }
-
-    @Test(timeout = 60000)
-    public void testOutstandingWriteLimitVeryHighLimit() throws Exception {
-        writeRecordsWithOutstandingWriteLimit(Integer.MAX_VALUE, Integer.MAX_VALUE, false);
-    }
-
-    @Test(timeout = 60000)
-    public void testOutstandingWriteLimitBlockAllStreamLimit() throws Exception {
-        writeRecordsWithOutstandingWriteLimit(0, Integer.MAX_VALUE, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testOutstandingWriteLimitBlockAllGlobalLimit() throws Exception {
-        writeRecordsWithOutstandingWriteLimit(Integer.MAX_VALUE, 0, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testOutstandingWriteLimitBlockAllLimitWithDarkmode() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setPerWriterOutstandingWriteLimit(0);
-        confLocal.setOutstandingWriteLimitDarkmode(true);
-        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-        ArrayList<Future<DLSN>> results = new ArrayList<Future<DLSN>>(1000);
-        for (int i = 0; i < 1000; i++) {
-            results.add(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
-        }
-        for (Future<DLSN> result : results) {
-            Await.result(result);
-        }
-        writer.closeAndComplete();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseAndCompleteLogSegmentWhenStreamIsInError() throws Exception {
-        String name = "distrlog-close-and-complete-logsegment-when-stream-is-in-error";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-
-        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-
-        long txId = 1L;
-        for (int i = 0; i < 5; i++) {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
-        }
-
-        BKLogSegmentWriter logWriter = writer.getCachedLogWriter();
-
-        BKNamespaceDriver driver = (BKNamespaceDriver) dlm.getNamespaceDriver();
-        // fence the ledger
-        driver.getReaderBKC().get().openLedger(logWriter.getLogSegmentId(),
-                BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
-
-        try {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
-            fail("Should fail write to a fenced ledger with BKTransmitException");
-        } catch (BKTransmitException bkte) {
-            // expected
-        }
-
-        try {
-            writer.closeAndComplete();
-            fail("Should fail to complete a log segment when its ledger is fenced");
-        } catch (BKTransmitException bkte) {
-            // expected
-        }
-
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(1, segments.size());
-        assertTrue(segments.get(0).isInProgress());
-
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseAndCompleteLogSegmentWhenCloseFailed() throws Exception {
-        String name = "distrlog-close-and-complete-logsegment-when-close-failed";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-
-        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-
-        long txId = 1L;
-        for (int i = 0; i < 5; i++) {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
-        }
-
-        BKLogSegmentWriter logWriter = writer.getCachedLogWriter();
-
-        BKNamespaceDriver driver = (BKNamespaceDriver) dlm.getNamespaceDriver();
-        // fence the ledger
-        driver.getReaderBKC().get().openLedger(logWriter.getLogSegmentId(),
-                BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
-
-        try {
-            // insert a write to detect the fencing state, to make test more robust.
-            writer.write(DLMTestUtil.getLogRecordInstance(txId++));
-            writer.closeAndComplete();
-            fail("Should fail to complete a log segment when its ledger is fenced");
-        } catch (IOException ioe) {
-            // expected
-            LOG.error("Failed to close and complete log segment {} : ", logWriter.getFullyQualifiedLogSegment(), ioe);
-        }
-
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(1, segments.size());
-        assertTrue(segments.get(0).isInProgress());
-
-        dlm.close();
-    }
-
-    private void testAsyncReadIdleErrorInternal(String name,
-                                                final int idleReaderErrorThreshold,
-                                                final boolean heartBeatUsingControlRecs,
-                                                final boolean simulateReaderStall) throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setReadAheadBatchSize(1);
-        confLocal.setReadAheadMaxRecords(1);
-        confLocal.setReaderIdleWarnThresholdMillis(0);
-        confLocal.setReaderIdleErrorThresholdMillis(idleReaderErrorThreshold);
-        final DistributedLogManager dlm = createNewDLM(confLocal, name);
-        final Thread currentThread = Thread.currentThread();
-        final int segmentSize = 3;
-        final int numSegments = 3;
-        final CountDownLatch latch = new CountDownLatch(1);
-        final ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
-        executor.schedule(
-            new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        int txid = 1;
-                        for (long i = 0; i < numSegments; i++) {
-                            long start = txid;
-                            BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-                            for (long j = 1; j <= segmentSize; j++) {
-                                writer.write(DLMTestUtil.getLargeLogRecordInstance(txid++));
-                                if ((i == 0) && (j == 1)) {
-                                    latch.countDown();
-                                }
-                            }
-
-                            if (heartBeatUsingControlRecs) {
-                                // There should be a control record such that
-                                // wait time + commit time (BK) < Idle Reader Threshold
-                                int threadSleepTime = idleReaderErrorThreshold
-                                    - 200 // BK commitTime
-                                    - 100; //safety margin
-
-                                for (int iter = 1; iter <= (2 * idleReaderErrorThreshold / threadSleepTime) ; iter++) {
-                                    Thread.sleep(threadSleepTime);
-                                    writer.write(DLMTestUtil.getLargeLogRecordInstance(txid, true));
-                                    writer.setReadyToFlush();
-                                }
-                                Thread.sleep(threadSleepTime);
-                            }
-
-                            writer.closeAndComplete();
-                            if (!heartBeatUsingControlRecs) {
-                                Thread.sleep(2 * idleReaderErrorThreshold);
-                            }
-                        }
-                    } catch (Exception exc) {
-                        if (!executor.isShutdown()) {
-                            currentThread.interrupt();
-                        }
-                    }
-                }
-            }, 0, TimeUnit.MILLISECONDS);
-
-        latch.await();
-        BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
-        if (simulateReaderStall) {
-            reader.disableProcessingReadRequests();
-        }
-        boolean exceptionEncountered = false;
-        int recordCount = 0;
-        try {
-            while (true) {
-                Future<LogRecordWithDLSN> record = reader.readNext();
-                Await.result(record);
-                recordCount++;
-
-                if (recordCount >= segmentSize * numSegments) {
-                    break;
-                }
-            }
-        } catch (IdleReaderException exc) {
-            exceptionEncountered = true;
-        }
-
-        if (simulateReaderStall) {
-            assertTrue(exceptionEncountered);
-        } else if (heartBeatUsingControlRecs) {
-            assertFalse(exceptionEncountered);
-            Assert.assertEquals(segmentSize * numSegments, recordCount);
-        } else {
-            assertTrue(exceptionEncountered);
-            Assert.assertEquals(segmentSize, recordCount);
-        }
-        assertFalse(currentThread.isInterrupted());
-        Utils.close(reader);
-        executor.shutdown();
-    }
-
-    @Test(timeout = 10000)
-    public void testAsyncReadIdleControlRecord() throws Exception {
-        String name = "distrlog-async-reader-idle-error-control";
-        testAsyncReadIdleErrorInternal(name, 500, true, false);
-    }
-
-    @Test(timeout = 10000)
-    public void testAsyncReadIdleError() throws Exception {
-        String name = "distrlog-async-reader-idle-error";
-        testAsyncReadIdleErrorInternal(name, 1000, false, false);
-    }
-
-    @Test(timeout = 10000)
-    public void testAsyncReadIdleError2() throws Exception {
-        String name = "distrlog-async-reader-idle-error-2";
-        testAsyncReadIdleErrorInternal(name, 1000, true, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testReleaseLockAfterFailedToRecover() throws Exception {
-        String name = "release-lock-after-failed-to-recover";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setLockTimeout(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setOutputBufferSize(0);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer =
-                (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
-
-        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
-        writer.abort();
-
-        for (int i = 0; i < 2; i++) {
-            FailpointUtils.setFailpoint(
-                    FailpointUtils.FailPointName.FP_RecoverIncompleteLogSegments,
-                    FailpointUtils.FailPointActions.FailPointAction_Throw);
-
-            try {
-                dlm.startAsyncLogSegmentNonPartitioned();
-                fail("Should fail during recovering incomplete log segments");
-            } catch (IOException ioe) {
-                // expected;
-            } finally {
-                FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_RecoverIncompleteLogSegments);
-            }
-        }
-
-        writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
-
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(1, segments.size());
-        assertFalse(segments.get(0).isInProgress());
-
-        writer.close();
-        dlm.close();
-    }
-
-    @DistributedLogAnnotations.FlakyTest
-    @Test(timeout = 60000)
-    public void testAsyncReadMissingLogSegmentsNotification() throws Exception {
-        String name = "distrlog-async-reader-missing-zk-notification";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setReadAheadBatchSize(1);
-        confLocal.setReadAheadMaxRecords(1);
-        confLocal.setReadLACLongPollTimeout(49);
-        confLocal.setReaderIdleWarnThresholdMillis(100);
-        confLocal.setReaderIdleErrorThresholdMillis(20000);
-        final DistributedLogManager dlm = createNewDLM(confLocal, name);
-        final Thread currentThread = Thread.currentThread();
-        final int segmentSize = 10;
-        final int numSegments = 3;
-        final CountDownLatch latch = new CountDownLatch(1);
-        final CountDownLatch readLatch = new CountDownLatch(1);
-        final ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
-        executor.schedule(
-                new Runnable() {
-                    @Override
-                    public void run() {
-                        try {
-                            int txid = 1;
-                            for (long i = 0; i < numSegments; i++) {
-                                BKSyncLogWriter writer = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-                                for (long j = 1; j <= segmentSize; j++) {
-                                    writer.write(DLMTestUtil.getLargeLogRecordInstance(txid++));
-                                    if ((i == 0) && (j == 1)) {
-                                        latch.countDown();
-                                    } else {
-                                        // wait for reader to start
-                                        readLatch.await();
-                                    }
-                                }
-                                writer.closeAndComplete();
-                                Thread.sleep(100);
-                            }
-                        } catch (Exception exc) {
-                            if (!executor.isShutdown()) {
-                                currentThread.interrupt();
-                            }
-                        }
-                    }
-                }, 0, TimeUnit.MILLISECONDS);
-
-        latch.await();
-        BKAsyncLogReader reader = (BKAsyncLogReader)dlm.getAsyncLogReader(DLSN.InitialDLSN);
-        reader.disableReadAheadLogSegmentsNotification();
-        boolean exceptionEncountered = false;
-        int recordCount = 0;
-        try {
-            while (true) {
-                Future<LogRecordWithDLSN> record = reader.readNext();
-                Await.result(record);
-                if (recordCount == 0) {
-                    readLatch.countDown();
-                }
-                recordCount++;
-
-                if (recordCount >= segmentSize * numSegments) {
-                    break;
-                }
-            }
-        } catch (IdleReaderException exc) {
-            exceptionEncountered = true;
-        }
-        assertTrue(!exceptionEncountered);
-        Assert.assertEquals(recordCount, segmentSize * numSegments);
-        assertTrue(!currentThread.isInterrupted());
-        Utils.close(reader);
-        executor.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLastTxId() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-
-        int numRecords = 10;
-        for (int i = 0; i < numRecords; i++) {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
-            assertEquals("last tx id should become " + i,
-                    i, writer.getLastTxId());
-        }
-        // open a writer to recover the inprogress log segment
-        AsyncLogWriter recoverWriter = dlm.startAsyncLogSegmentNonPartitioned();
-        assertEquals("recovered last tx id should be " + (numRecords - 1),
-                numRecords - 1, recoverWriter.getLastTxId());
-    }
-
-    @Test(timeout = 60000)
-    public void testMaxReadAheadRecords() throws Exception {
-        int maxRecords = 1;
-        int batchSize = 8;
-        int maxAllowedCachedRecords = maxRecords + batchSize - 1;
-
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(Integer.MAX_VALUE);
-        confLocal.setReadAheadMaxRecords(maxRecords);
-        confLocal.setReadAheadBatchSize(batchSize);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-
-        int numRecords = 40;
-        for (int i = 1; i <= numRecords; i++) {
-            Await.result(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));
-
-        BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
-        record = Await.result(reader.readNext());
-        LOG.info("Read record {}", record);
-        assertEquals(1L, record.getTransactionId());
-
-        assertNotNull(reader.getReadAheadReader());
-        assertTrue(reader.getReadAheadReader().getNumCachedEntries() <= maxAllowedCachedRecords);
-
-        for (int i = 2; i <= numRecords; i++) {
-            record = Await.result(reader.readNext());
-            LOG.info("Read record {}", record);
-            assertEquals((long) i, record.getTransactionId());
-            TimeUnit.MILLISECONDS.sleep(20);
-            int numCachedEntries = reader.getReadAheadReader().getNumCachedEntries();
-            assertTrue("Should cache less than " + batchSize + " records but already found "
-                    + numCachedEntries + " records when reading " + i + "th record",
-                    numCachedEntries <= maxAllowedCachedRecords);
-        }
-        Utils.close(reader);
-    }
-
-    @Test(timeout = 60000)
-    public void testMarkEndOfStream() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-
-        final int NUM_RECORDS = 10;
-        int i = 1;
-        for (; i <= NUM_RECORDS; i++) {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
-            assertEquals("last tx id should become " + i,
-                    i, writer.getLastTxId());
-        }
-
-        Await.result(writer.markEndOfStream());
-
-        // Multiple end of streams are ok.
-        Await.result(writer.markEndOfStream());
-
-        try {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
-            fail("Should have thrown");
-        } catch (EndOfStreamException ex) {
-        }
-
-        BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
-        LogRecord record = null;
-        for (int j = 0; j < NUM_RECORDS; j++) {
-            record = Await.result(reader.readNext());
-            assertEquals(j+1, record.getTransactionId());
-        }
-
-        try {
-            record = Await.result(reader.readNext());
-            fail("Should have thrown");
-        } catch (EndOfStreamException ex) {
-        }
-        Utils.close(reader);
-    }
-
-    @Test(timeout = 60000)
-    public void testMarkEndOfStreamAtBeginningOfSegment() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-        Await.result(writer.markEndOfStream());
-        try {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(1)));
-            fail("Should have thrown");
-        } catch (EndOfStreamException ex) {
-        }
-        writer.close();
-
-        BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
-        try {
-            LogRecord record = Await.result(reader.readNext());
-            fail("Should have thrown");
-        } catch (EndOfStreamException ex) {
-        }
-        Utils.close(reader);
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkReadWaitingMoreRecords() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
-        LogRecord controlRecord = DLMTestUtil.getLogRecordInstance(1L);
-        controlRecord.setControl();
-        FutureUtils.result(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();
-
-        // write another records
-        for (int i = 0; i < 5; i++) {
-            long txid = 2L + i;
-            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txid)));
-            controlRecord = DLMTestUtil.getLogRecordInstance(txid);
-            controlRecord.setControl();
-            FutureUtils.result(writer.write(controlRecord));
-        }
-
-        List<LogRecordWithDLSN> bulkReadRecords = FutureUtils.result(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);
-        assertEquals(3L, record.getTransactionId());
-        DLMTestUtil.verifyLogRecord(record);
-
-        Utils.close(reader);
-        writer.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkReadNotWaitingMoreRecords() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
-        LogRecord controlRecord = DLMTestUtil.getLogRecordInstance(1L);
-        controlRecord.setControl();
-        FutureUtils.result(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();
-
-        List<LogRecordWithDLSN> bulkReadRecords = FutureUtils.result(bulkReadFuture);
-        assertEquals(1, bulkReadRecords.size());
-        assertEquals(1L, bulkReadRecords.get(0).getTransactionId());
-        for (LogRecordWithDLSN record : bulkReadRecords) {
-            DLMTestUtil.verifyLogRecord(record);
-        }
-
-        // write another records
-        for (int i = 0; i < 5; i++) {
-            long txid = 2L + i;
-            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txid)));
-            controlRecord = DLMTestUtil.getLogRecordInstance(txid);
-            controlRecord.setControl();
-            FutureUtils.result(writer.write(controlRecord));
-        }
-
-        LogRecordWithDLSN record = FutureUtils.result(readFuture);
-        assertEquals(2L, record.getTransactionId());
-        DLMTestUtil.verifyLogRecord(record);
-
-        Utils.close(reader);
-        writer.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReadBrokenEntries() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadBatchSize(1);
-        confLocal.setPositionGapDetectionEnabled(false);
-        confLocal.setReadAheadSkipBrokenEntries(true);
-        confLocal.setEIInjectReadAheadBrokenEntries(true);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        int numLogSegments = 3;
-        int numRecordsPerLogSegment = 10;
-
-        long txid = 1L;
-        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false);
-
-        AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
-
-        // 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());
-            assertFalse(record.getDlsn().getEntryId() % 10 == 0);
-        }
-
-        Utils.close(reader);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReadBrokenEntriesWithGapDetection() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadBatchSize(1);
-        confLocal.setPositionGapDetectionEnabled(true);
-        confLocal.setReadAheadSkipBrokenEntries(true);
-        confLocal.setEIInjectReadAheadBrokenEntries(true);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        int numLogSegments = 1;
-        int numRecordsPerLogSegment = 100;
-
-        long txid = 1L;
-        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false);
-
-        AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
-
-        try {
-            // 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());
-                assertFalse(record.getDlsn().getEntryId() % 10 == 0);
-            }
-            fail("should have thrown");
-        } catch (DLIllegalStateException e) {
-        }
-
-        Utils.close(reader);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReadBrokenEntriesAndLargeBatchSize() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadBatchSize(5);
-        confLocal.setPositionGapDetectionEnabled(false);
-        confLocal.setReadAheadSkipBrokenEntries(true);
-        confLocal.setEIInjectReadAheadBrokenEntries(true);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        int numLogSegments = 1;
-        int numRecordsPerLogSegment = 100;
-
-        long txid = 1L;
-        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false);
-
-        AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
-
-        // Every 10th record broken. Reading 5 at once, beginning from 0:
-        // 1. range 0-4 will be corrupted and discarded
-        // 2. ranges 1-5, 2-6, 3-7, 4-8, 5-9 will be ok
-        // 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());
-            assertFalse(record.getDlsn().getEntryId() % 10 == 0);
-        }
-
-        Utils.close(reader);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReadBrokenEntriesAndLargeBatchSizeCrossSegment() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(testConf);
-
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setReadAheadWaitTime(10);
-        confLocal.setReadAheadBatchSize(8);
-        confLocal.setPositionGapDetectionEnabled(false);
-        confLocal.setReadAheadSkipBrokenEntries(true);
-        confLocal.setEIInjectReadAheadBrokenEntries(true);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-
-        int numLogSegments = 3;
-        int numRecordsPerLogSegment = 5;
-
-        long txid = 1L;
-        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false);
-
-        AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
-
-        // Every 10th record broken. Reading 8 at once, beginning from 0:
-        // 1. range 0-7 will be corrupted and discarded
-        // 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());
-            assertFalse(record.getDlsn().getEntryId() % 10 == 0);
-        }
-
-        Utils.close(reader);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogStreamWithDifferentReplicationFactor() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-
-        ConcurrentBaseConfiguration baseConf = new ConcurrentConstConfiguration(confLocal);
-        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(baseConf);
-        dynConf.setProperty(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE,
-                DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT - 1);
-
-        URI uri = createDLMURI("/" + name);
-        ensureURICreated(uri);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.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)));
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(1, segments.size());
-        long ledgerId = segments.get(0).getLogSegmentId();
-        LedgerHandle lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC()
-                .get().openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
-        LedgerMetadata metadata = BookKeeperAccessor.getLedgerMetadata(lh);
-        assertEquals(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT, metadata.getEnsembleSize());
-        lh.close();
-        Utils.close(writer);
-        dlm.close();
-
-        // use customized configuration
-        dlm = namespace.openLog(
-                name + "-custom",
-                Optional.<DistributedLogConfiguration>absent(),
-                Optional.of(dynConf),
-                Optional.<StatsLogger>absent());
-        writer = dlm.startAsyncLogSegmentNonPartitioned();
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
-        segments = dlm.getLogSegments();
-        assertEquals(1, segments.size());
-        ledgerId = segments.get(0).getLogSegmentId();
-        lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC()
-                .get().openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
-        metadata = BookKeeperAccessor.getLedgerMetadata(lh);
-        assertEquals(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT - 1, metadata.getEnsembleSize());
-        lh.close();
-        Utils.close(writer);
-        dlm.close();
-        namespace.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteRecordSet() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-
-        URI uri = createDLMURI("/" + name);
-        ensureURICreated(uri);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-        List<Future<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();
-        // 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>();
-            recordSetWriter.writeRecord(ByteBuffer.wrap(record.getPayload()), writePromise);
-            recordSetFutures.add(writePromise);
-        }
-        final ByteBuffer recordSetBuffer = recordSetWriter.getBuffer();
-        byte[] data = new byte[recordSetBuffer.remaining()];
-        recordSetBuffer.get(data);
-        LogRecord setRecord = new LogRecord(6L, data);
-        setRecord.setRecordSet();
-        Future<DLSN> writeRecordSetFuture = writer.write(setRecord);
-        writeRecordSetFuture.addEventListener(new FutureEventListener<DLSN>() {
-            @Override
-            public void onSuccess(DLSN dlsn) {
-                recordSetWriter.completeTransmit(
-                        dlsn.getLogSegmentSequenceNo(),
-                        dlsn.getEntryId(),
-                        dlsn.getSlotId());
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                recordSetWriter.abortTransmit(cause);
-            }
-        });
-        writeFutures.add(writeRecordSetFuture);
-        FutureUtils.result(writeRecordSetFuture);
-        // write last 5 records
-        for (int i = 0; i < 5; i++) {
-            LogRecord record = DLMTestUtil.getLogRecordInstance(11L + i);
-            Future<DLSN> writeFuture = writer.write(record);
-            writeFutures.add(writeFuture);
-            // make sure get log record count returns the right count
-            if (i == 0) {
-                FutureUtils.result(writeFuture);
-                assertEquals(10, dlm.getLogRecordCount());
-            }
-        }
-
-        List<DLSN> writeResults = FutureUtils.result(Future.collect(writeFutures));
-
-        for (int i = 0; i < 5; i++) {
-            Assert.assertEquals(new DLSN(1L, i, 0L), writeResults.get(i));
-        }
-        Assert.assertEquals(new DLSN(1L, 5L, 0L), writeResults.get(5));
-        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));
-        for (int i = 0; i < 5; i++) {
-            Assert.assertEquals(new DLSN(1L, 5L, i), recordSetWriteResults.get(i));
-        }
-
-        FutureUtils.result(writer.flushAndCommit());
-
-        DistributedLogConfiguration readConf1 = new DistributedLogConfiguration();
-        readConf1.addConfiguration(confLocal);
-        readConf1.setDeserializeRecordSetOnReads(true);
-
-        DistributedLogManager readDLM1 = createNewDLM(readConf1, name);
-        AsyncLogReader reader1 = readDLM1.getAsyncLogReader(DLSN.InitialDLSN);
-        for (int i = 0; i < 15; i++) {
-            LogRecordWithDLSN record = FutureUtils.result(reader1.readNext());
-            if (i < 5) {
-                assertEquals(new DLSN(1L, i, 0L), record.getDlsn());
-                assertEquals(1L + i, record.getTransactionId());
-            } else if (i >= 10) {
-                assertEquals(new DLSN(1L, 6L + i - 10, 0L), record.getDlsn());
-                assertEquals(11L + i - 10, record.getTransactionId());
-            } else {
-                assertEquals(new DLSN(1L, 5L, i - 5), record.getDlsn());
-                assertEquals(6L, record.getTransactionId());
-            }
-            assertEquals(i+1, record.getPositionWithinLogSegment());
-            assertArrayEquals(DLMTestUtil.generatePayload(i+1), record.getPayload());
-        }
-        Utils.close(reader1);
-        readDLM1.close();
-
-        DistributedLogConfiguration readConf2 = new DistributedLogConfiguration();
-        readConf2.addConfiguration(confLocal);
-        readConf2.setDeserializeRecordSetOnReads(false);
-
-        DistributedLogManager readDLM2 = createNewDLM(readConf2, name);
-        AsyncLogReader reader2 = readDLM2.getAsyncLogReader(DLSN.InitialDLSN);
-        for (int i = 0; i < 11; i++) {
-            LogRecordWithDLSN record = FutureUtils.result(reader2.readNext());
-            LOG.info("Read record {}", record);
-            if (i < 5) {
-                assertEquals(new DLSN(1L, i, 0L), record.getDlsn());
-                assertEquals(1L + i, record.getTransactionId());
-                assertEquals(i + 1, record.getPositionWithinLogSegment());
-                assertArrayEquals(DLMTestUtil.generatePayload(i+1), record.getPayload());
-            } else if (i >= 6L) {
-                assertEquals(new DLSN(1L, 6L + i - 6, 0L), record.getDlsn());
-                assertEquals(11L + i - 6, record.getTransactionId());
-                assertEquals(11 + i - 6, record.getPositionWithinLogSegment());
-                assertArrayEquals(DLMTestUtil.generatePayload(11L + i - 6), record.getPayload());
-            } else {
-                assertEquals(new DLSN(1L, 5L, 0), record.getDlsn());
-                assertEquals(6L, record.getTransactionId());
-                assertEquals(6, record.getPositionWithinLogSegment());
-                assertTrue(record.isRecordSet());
-                assertEquals(5, LogRecordSet.numRecords(record));
-            }
-        }
-        Utils.close(reader2);
-        readDLM2.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testIdleReaderExceptionWhenKeepAliveIsDisabled() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setPeriodicKeepAliveMilliSeconds(0);
-        confLocal.setReadLACLongPollTimeout(9);
-        confLocal.setReaderIdleWarnThresholdMillis(20);
-        confLocal.setReaderIdleErrorThresholdMillis(40);
-
-        URI uri = createDLMURI("/" + name);
-        ensureURICreated(uri);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) FutureUtils.result(dlm.openAsyncLogWriter());
-        writer.write(DLMTestUtil.getLogRecordInstance(1L));
-
-        AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(DLSN.InitialDLSN));
-        try {
-            FutureUtils.result(reader.readNext());
-            fail("Should fail when stream is idle");
-        } catch (IdleReaderException ire) {
-            // expected
-        }
-        Utils.close(reader);
-        writer.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testIdleReaderExceptionWhenKeepAliveIsEnabled() throws Exception {
-        String name = runtime.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(testConf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setPeriodicKeepAliveMilliSeconds(1000);
-        confLocal.setReadLACLongPollTimeout(999);
-        confLocal.setReaderIdleWarnThresholdMillis(2000);
-        confLocal.setReaderIdleErrorThresholdMillis(4000);
-
-        URI uri = createDLMURI("/" + name);
-        ensureURICreated(uri);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) FutureUtils.result(dlm.openAsyncLogWriter());
-        writer.write(DLMTestUtil.getLogRecordInstance(1L));
-
-        AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(DLSN.InitialDLSN));
-        LogRecordWithDLSN record = FutureUtils.result(reader.readNext());
-        assertEquals(1L, record.getTransactionId());
-        DLMTestUtil.verifyLogRecord(record);
-
-        Utils.close(reader);
-        writer.close();
-        dlm.close();
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogManager.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogManager.java
deleted file mode 100644
index f7d587d..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogManager.java
+++ /dev/null
@@ -1,1201 +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 com.twitter.distributedlog;
-
-import java.net.URI;
-import java.util.Collection;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Random;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-import com.twitter.distributedlog.exceptions.AlreadyTruncatedTransactionException;
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.exceptions.LogEmptyException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.LogReadException;
-import com.twitter.distributedlog.impl.ZKLogSegmentMetadataStore;
-import com.twitter.distributedlog.io.Abortables;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Utils;
-import org.apache.bookkeeper.client.BKException;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.twitter.distributedlog.callback.LogSegmentListener;
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.TransactionIdOutOfOrderException;
-import com.twitter.distributedlog.metadata.LogMetadata;
-import com.twitter.distributedlog.metadata.MetadataUpdater;
-import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.subscription.SubscriptionsStore;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-
-import static org.junit.Assert.*;
-import static org.junit.Assert.assertEquals;
-
-public class TestBKDistributedLogManager extends TestDistributedLogBase {
-    static final Logger LOG = LoggerFactory.getLogger(TestBKDistributedLogManager.class);
-
-    private static final Random RAND = new Random(System.currentTimeMillis());
-
-    @Rule
-    public TestName testNames = new TestName();
-
-    private static final long DEFAULT_SEGMENT_SIZE = 1000;
-
-    private void testNonPartitionedWritesInternal(String name, DistributedLogConfiguration conf) throws Exception {
-        BKDistributedLogManager dlm = createNewDLM(conf, name);
-
-        long txid = 1;
-        for (long i = 0; i < 3; i++) {
-            long start = txid;
-            BKSyncLogWriter writer = dlm.startLogSegmentNonPartitioned();
-            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
-                writer.write(DLMTestUtil.getLogRecordInstance(txid++));
-            }
-            BKLogSegmentWriter perStreamLogWriter = writer.getCachedLogWriter();
-            writer.closeAndComplete();
-            BKLogWriteHandler blplm = dlm.createWriteHandler(true);
-            assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
-                                                                perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-            FutureUtils.result(blplm.asyncClose());
-        }
-
-        LogWriter writer = dlm.startLogSegmentNonPartitioned();
-        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
-            writer.write(DLMTestUtil.getLogRecordInstance(txid++));
-        }
-        writer.setReadyToFlush();
-        writer.flushAndSync();
-        writer.close();
-
-        LogReader reader = dlm.getInputStream(1);
-        long numTrans = 0;
-        LogRecord record = reader.readNext(false);
-        long lastTxId = -1;
-        while (null != record) {
-            DLMTestUtil.verifyLogRecord(record);
-            assert (lastTxId < record.getTransactionId());
-            lastTxId = record.getTransactionId();
-            numTrans++;
-            record = reader.readNext(false);
-        }
-        reader.close();
-        assertEquals((txid - 1), numTrans);
-    }
-
-    @Test(timeout = 60000)
-    public void testSimpleWrite() throws Exception {
-        BKDistributedLogManager dlm = createNewDLM(conf, "distrlog-simplewrite");
-        BKSyncLogWriter out = dlm.startLogSegmentNonPartitioned();
-        for (long i = 1; i <= 100; i++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(i);
-            out.write(op);
-        }
-        BKLogSegmentWriter perStreamLogWriter = out.getCachedLogWriter();
-        out.closeAndComplete();
-
-        BKLogWriteHandler blplm = dlm.createWriteHandler(true);
-        assertNotNull(zkc.exists(blplm.completedLedgerZNode(1, 100,
-                perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-        FutureUtils.result(blplm.asyncClose());
-    }
-
-    @Test(timeout = 60000)
-    public void testNumberOfTransactions() throws Exception {
-        String name = "distrlog-txncount";
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-        for (long i = 1; i <= 100; i++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(i);
-            out.write(op);
-        }
-        out.closeAndComplete();
-
-        long numTrans = DLMTestUtil.getNumberofLogRecords(createNewDLM(conf, name), 1);
-        assertEquals(100, numTrans);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testContinuousReaders() throws Exception {
-        String name = "distrlog-continuous";
-        BKDistributedLogManager dlm = createNewDLM(conf, name);
-        long txid = 1;
-        for (long i = 0; i < 3; i++) {
-            long start = txid;
-            BKSyncLogWriter out = dlm.startLogSegmentNonPartitioned();
-            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
-                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                out.write(op);
-            }
-            BKLogSegmentWriter perStreamLogWriter = out.getCachedLogWriter();
-            out.closeAndComplete();
-            BKLogWriteHandler blplm = dlm.createWriteHandler(true);
-
-            assertNotNull(
-                zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
-                                                      perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-            FutureUtils.result(blplm.asyncClose());
-        }
-
-        BKSyncLogWriter out = dlm.startLogSegmentNonPartitioned();
-        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-            out.write(op);
-        }
-        out.setReadyToFlush();
-        out.flushAndSync();
-        out.close();
-        dlm.close();
-
-        dlm = createNewDLM(conf, name);
-
-        LogReader reader = dlm.getInputStream(1);
-        long numTrans = 0;
-        LogRecord record = reader.readNext(false);
-        while (null != record) {
-            DLMTestUtil.verifyLogRecord(record);
-            numTrans++;
-            record = reader.readNext(false);
-        }
-        assertEquals((txid - 1), numTrans);
-        assertEquals(txid - 1, dlm.getLogRecordCount());
-        reader.close();
-        dlm.close();
-    }
-
-    /**
-     * Create a bkdlm namespace, write a journal from txid 1, close stream.
-     * Try to create a new journal from txid 1. Should throw an exception.
-     */
-    @Test(timeout = 60000)
-    public void testWriteRestartFrom1() throws Exception {
-        DistributedLogManager dlm = createNewDLM(conf, "distrlog-restartFrom1");
-        long txid = 1;
-        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-            out.write(op);
-        }
-        out.closeAndComplete();
-
-        txid = 1;
-        try {
-            out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-            out.write(DLMTestUtil.getLogRecordInstance(txid));
-            fail("Shouldn't be able to start another journal from " + txid
-                + " when one already exists");
-        } catch (Exception ioe) {
-            LOG.info("Caught exception as expected", ioe);
-        } finally {
-            out.close();
-        }
-
-        // test border case
-        txid = DEFAULT_SEGMENT_SIZE - 1;
-        try {
-            out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-            out.write(DLMTestUtil.getLogRecordInstance(txid));
-            fail("Shouldn't be able to start another journal from " + txid
-                + " when one already exists");
-        } catch (TransactionIdOutOfOrderException rste) {
-            LOG.info("Caught exception as expected", rste);
-        } finally {
-            out.close();
-        }
-
-        // open journal continuing from before
-        txid = DEFAULT_SEGMENT_SIZE + 1;
-        out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-        assertNotNull(out);
-
-        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-            out.write(op);
-        }
-        out.closeAndComplete();
-
-        // open journal arbitarily far in the future
-        txid = DEFAULT_SEGMENT_SIZE * 4;
-        out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-        out.write(DLMTestUtil.getLogRecordInstance(txid));
-        out.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testTwoWritersOnLockDisabled() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        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)));
-
-        // write a record to writer 1 again
-        try {
-            FutureUtils.result(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());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testSimpleRead() throws Exception {
-        String name = "distrlog-simpleread";
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        final long numTransactions = 10000;
-        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-        for (long i = 1; i <= numTransactions; i++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(i);
-            out.write(op);
-        }
-        out.closeAndComplete();
-
-        assertEquals(numTransactions, DLMTestUtil.getNumberofLogRecords(createNewDLM(conf, name), 1));
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testNumberOfTransactionsWithInprogressAtEnd() throws Exception {
-        String name = "distrlog-inprogressAtEnd";
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        long txid = 1;
-        for (long i = 0; i < 3; i++) {
-            long start = txid;
-            BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
-                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                out.write(op);
-            }
-            BKLogSegmentWriter perStreamLogWriter = out.getCachedLogWriter();
-            out.closeAndComplete();
-            BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
-            assertNotNull(
-                zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
-                                                      perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-            FutureUtils.result(blplm.asyncClose());
-        }
-        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-            out.write(op);
-        }
-        out.setReadyToFlush();
-        out.flushAndSync();
-        out.close();
-
-        long numTrans = DLMTestUtil.getNumberofLogRecords(createNewDLM(conf, name), 1);
-        assertEquals((txid - 1), numTrans);
-    }
-
-    @Test(timeout = 60000)
-    public void testContinuousReaderBulk() throws Exception {
-        String name = "distrlog-continuous-bulk";
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        long txid = 1;
-        for (long i = 0; i < 3; i++) {
-            BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
-                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                out.write(op);
-            }
-            out.closeAndComplete();
-        }
-
-        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-            out.write(op);
-        }
-        out.setReadyToFlush();
-        out.flushAndSync();
-        out.close();
-        dlm.close();
-
-        dlm = createNewDLM(conf, name);
-
-        LogReader reader = dlm.getInputStream(1);
-        long numTrans = 0;
-        List<LogRecordWithDLSN> recordList = reader.readBulk(false, 13);
-        long lastTxId = -1;
-        while (!recordList.isEmpty()) {
-            for (LogRecord record : recordList) {
-                assert (lastTxId < record.getTransactionId());
-                lastTxId = record.getTransactionId();
-                DLMTestUtil.verifyLogRecord(record);
-                numTrans++;
-            }
-            recordList = reader.readBulk(false, 13);
-        }
-        reader.close();
-        assertEquals((txid - 1), numTrans);
-    }
-
-    @Test(timeout = 60000)
-    public void testContinuousReadersWithEmptyLedgers() throws Exception {
-        String name = "distrlog-continuous-emptyledgers";
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        long txid = 1;
-        for (long i = 0; i < 3; i++) {
-            long start = txid;
-            BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
-                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                out.write(op);
-            }
-            BKLogSegmentWriter writer = out.getCachedLogWriter();
-            out.closeAndComplete();
-            BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
-
-            assertNotNull(
-                zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
-                                                      writer.getLogSegmentSequenceNumber()), false));
-            BKLogSegmentWriter perStreamLogWriter = blplm.startLogSegment(txid - 1);
-            blplm.completeAndCloseLogSegment(perStreamLogWriter.getLogSegmentSequenceNumber(),
-                    perStreamLogWriter.getLogSegmentId(), txid - 1, txid - 1, 0);
-            assertNotNull(
-                zkc.exists(blplm.completedLedgerZNode(txid - 1, txid - 1,
-                                                      perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-            FutureUtils.result(blplm.asyncClose());
-        }
-
-        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-            out.write(op);
-        }
-        out.setReadyToFlush();
-        out.flushAndSync();
-        out.close();
-        dlm.close();
-
-        dlm = createNewDLM(conf, name);
-
-        AsyncLogReader asyncreader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
-        long numTrans = 0;
-        LogRecordWithDLSN record = Await.result(asyncreader.readNext());
-        while (null != record) {
-            DLMTestUtil.verifyLogRecord(record);
-            numTrans++;
-            if (numTrans >= (txid - 1)) {
-                break;
-            }
-            record = Await.result(asyncreader.readNext());
-        }
-        assertEquals((txid - 1), numTrans);
-        Utils.close(asyncreader);
-
-        LogReader reader = dlm.getInputStream(1);
-        numTrans = 0;
-        record = reader.readNext(false);
-        while (null != record) {
-            DLMTestUtil.verifyLogRecord(record);
-            numTrans++;
-            record = reader.readNext(false);
-        }
-        assertEquals((txid - 1), numTrans);
-        reader.close();
-        assertEquals(txid - 1, dlm.getLogRecordCount());
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testNonPartitionedWrites() throws Exception {
-        String name = "distrlog-non-partitioned-bulk";
-        testNonPartitionedWritesInternal(name, conf);
-    }
-
-    @Test(timeout = 60000)
-    public void testCheckLogExists() throws Exception {
-        String name = "distrlog-check-log-exists";
-        DistributedLogManager dlm = createNewDLM(conf, name);
-
-        long txid = 1;
-        LogWriter writer = dlm.startLogSegmentNonPartitioned();
-        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
-            writer.write(DLMTestUtil.getLogRecordInstance(txid++));
-        }
-        writer.setReadyToFlush();
-        writer.flushAndSync();
-        writer.close();
-        dlm.close();
-
-        URI uri = createDLMURI("/" + name);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.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()
-                .conf(conf).uri(nonExistentUri).build();
-        assertFalse(nonExistentNS.logExists(name));
-
-        int logCount = 0;
-        Iterator<String> logIter = namespace.getLogs();
-        while(logIter.hasNext()) {
-            String log = logIter.next();
-            logCount++;
-            assertEquals(name, log);
-        }
-        assertEquals(1, logCount);
-
-        namespace.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testMetadataAccessor() throws Exception {
-        String name = "distrlog-metadata-accessor";
-        MetadataAccessor metadata = DLMTestUtil.createNewMetadataAccessor(conf, name, createDLMURI("/" + name));
-        assertEquals(name, metadata.getStreamName());
-        metadata.createOrUpdateMetadata(name.getBytes());
-        assertEquals(name, new String(metadata.getMetadata()));
-        metadata.deleteMetadata();
-        assertEquals(null, metadata.getMetadata());
-    }
-
-    @Test(timeout = 60000)
-    public void testSubscriptionsStore() throws Exception {
-        String name = "distrlog-subscriptions-store";
-        String subscriber0 = "subscriber-0";
-        String subscriber1 = "subscriber-1";
-        String subscriber2 = "subscriber-2";
-
-        DLSN commitPosition0 = new DLSN(4, 33, 5);
-        DLSN commitPosition1 = new DLSN(4, 34, 5);
-        DLSN commitPosition2 = new DLSN(5, 34, 5);
-        DLSN commitPosition3 = new DLSN(6, 35, 6);
-
-        DistributedLogManager dlm = createNewDLM(conf, name);
-
-        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);
-        // empty
-        assertTrue(Await.result(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());
-        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());
-        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());
-        assertEquals(3, committedPositions.size());
-        assertEquals(commitPosition0, committedPositions.get(subscriber0));
-        assertEquals(commitPosition1, committedPositions.get(subscriber1));
-        assertEquals(commitPosition2, committedPositions.get(subscriber2));
-
-        // subscriber 2 advance again
-        DistributedLogManager newDLM = createNewDLM(conf, name);
-        SubscriptionsStore newStore = newDLM.getSubscriptionsStore();
-        Await.result(newStore.advanceCommitPosition(subscriber2, commitPosition3));
-        newStore.close();
-        newDLM.close();
-
-        committedPositions = Await.result(store.getLastCommitPositions());
-        assertEquals(3, committedPositions.size());
-        assertEquals(commitPosition0, committedPositions.get(subscriber0));
-        assertEquals(commitPosition1, committedPositions.get(subscriber1));
-        assertEquals(commitPosition3, committedPositions.get(subscriber2));
-
-        dlm.close();
-
-    }
-
-    private long writeAndMarkEndOfStream(DistributedLogManager dlm, long txid) throws Exception {
-        for (long i = 0; i < 3; i++) {
-            long start = txid;
-            BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
-                writer.write(DLMTestUtil.getLogRecordInstance(txid++));
-            }
-
-            BKLogSegmentWriter perStreamLogWriter = writer.getCachedLogWriter();
-
-            if (i < 2) {
-                writer.closeAndComplete();
-                BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
-                assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
-                                                                    perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-                FutureUtils.result(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());
-            }
-        }
-        return txid;
-    }
-
-    @Test(timeout = 60000)
-    public void testMarkEndOfStream() throws Exception {
-        String name = "distrlog-mark-end-of-stream";
-        DistributedLogManager dlm = createNewDLM(conf, name);
-
-        long txid = 1;
-        txid = writeAndMarkEndOfStream(dlm, txid);
-
-        LogReader reader = dlm.getInputStream(1);
-        long numTrans = 0;
-        boolean exceptionEncountered = false;
-        LogRecord record = null;
-        try {
-            record = reader.readNext(false);
-            long expectedTxId = 1;
-            while (null != record) {
-                DLMTestUtil.verifyLogRecord(record);
-                assertEquals(expectedTxId, record.getTransactionId());
-                expectedTxId++;
-                numTrans++;
-                record = reader.readNext(false);
-            }
-        } catch (EndOfStreamException exc) {
-            LOG.info("Encountered EndOfStream on reading records after {}", record);
-            exceptionEncountered = true;
-        }
-        assertEquals((txid - 1), numTrans);
-        assertTrue(exceptionEncountered);
-        exceptionEncountered = false;
-        try {
-            reader.readNext(false);
-        } catch (EndOfStreamException exc) {
-            exceptionEncountered = true;
-        }
-        assertTrue(exceptionEncountered);
-        reader.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteFailsAfterMarkEndOfStream() throws Exception {
-        String name = "distrlog-mark-end-failure";
-        DistributedLogManager dlm = createNewDLM(conf, name);
-
-        long txid = 1;
-        txid = writeAndMarkEndOfStream(dlm, txid);
-
-        assertEquals(txid - 1, dlm.getLastTxId());
-        LogRecord last = dlm.getLastLogRecord();
-        assertEquals(txid - 1, last.getTransactionId());
-        DLMTestUtil.verifyLogRecord(last);
-        assertTrue(dlm.isEndOfStreamMarked());
-
-        LogWriter writer = null;
-        boolean exceptionEncountered = false;
-        try {
-            writer = dlm.startLogSegmentNonPartitioned();
-            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
-                writer.write(DLMTestUtil.getLogRecordInstance(txid++));
-            }
-        } catch (EndOfStreamException exc) {
-            exceptionEncountered = true;
-        }
-        writer.close();
-        assertTrue(exceptionEncountered);
-    }
-
-    @Test(timeout = 60000)
-    public void testMarkEndOfStreamOnEmptyStream() throws Exception {
-        markEndOfStreamOnEmptyLogSegment(0);
-    }
-
-    @Test(timeout = 60000)
-    public void testMarkEndOfStreamOnClosedStream() throws Exception {
-        markEndOfStreamOnEmptyLogSegment(3);
-    }
-
-    private void markEndOfStreamOnEmptyLogSegment(int numCompletedSegments) throws Exception {
-        String name = "distrlog-mark-end-empty-" + numCompletedSegments;
-
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        DLMTestUtil.generateCompletedLogSegments(dlm, conf, numCompletedSegments, DEFAULT_SEGMENT_SIZE);
-
-        BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-        writer.markEndOfStream();
-
-        LogReader reader = dlm.getInputStream(1);
-        long numTrans = 0;
-        boolean exceptionEncountered = false;
-        try {
-            LogRecord record = reader.readNext(false);
-            long lastTxId = -1;
-            while (null != record) {
-                DLMTestUtil.verifyLogRecord(record);
-                assert (lastTxId < record.getTransactionId());
-                lastTxId = record.getTransactionId();
-                numTrans++;
-                record = reader.readNext(false);
-            }
-        } catch (EndOfStreamException exc) {
-            exceptionEncountered = true;
-        }
-        assertEquals(numCompletedSegments * DEFAULT_SEGMENT_SIZE, numTrans);
-        assertTrue(exceptionEncountered);
-        exceptionEncountered = false;
-        try {
-            reader.readNext(false);
-        } catch (EndOfStreamException exc) {
-            exceptionEncountered = true;
-        }
-        assertTrue(exceptionEncountered);
-        reader.close();
-    }
-
-    @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(
-                                DLMTestUtil.repeatString("abcdefgh", 256), 512).getBytes())));
-    }
-
-    @Test(timeout = 60000)
-    public void testMaxTransmissionSize() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setOutputBufferSize(1024 * 1024);
-        BKDistributedLogManager dlm =
-                createNewDLM(confLocal, "distrlog-transmissionSize");
-        AsyncLogWriter out = FutureUtils.result(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);
-            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);
-        } catch (LogRecordTooLongException exc) {
-            exceptionEncountered = true;
-        } finally {
-            FutureUtils.result(out.asyncClose());
-        }
-        assertFalse(exceptionEncountered);
-        Abortables.abortQuietly(out);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void deleteDuringRead() throws Exception {
-        String name = "distrlog-delete-with-reader";
-        DistributedLogManager dlm = createNewDLM(conf, name);
-
-        long txid = 1;
-        for (long i = 0; i < 3; i++) {
-            long start = txid;
-            BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
-                writer.write(DLMTestUtil.getLogRecordInstance(txid++));
-            }
-
-            BKLogSegmentWriter perStreamLogWriter = writer.getCachedLogWriter();
-
-            writer.closeAndComplete();
-            BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
-            assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
-                                                                perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-            FutureUtils.result(blplm.asyncClose());
-        }
-
-        LogReader reader = dlm.getInputStream(1);
-        long numTrans = 1;
-        LogRecord record = reader.readNext(false);
-        assert (null != record);
-        DLMTestUtil.verifyLogRecord(record);
-        long lastTxId = record.getTransactionId();
-
-        dlm.delete();
-
-        boolean exceptionEncountered = false;
-        try {
-            record = reader.readNext(false);
-            while (null != record) {
-                DLMTestUtil.verifyLogRecord(record);
-                assert (lastTxId < record.getTransactionId());
-                lastTxId = record.getTransactionId();
-                numTrans++;
-                record = reader.readNext(false);
-            }
-            // make sure the exception is thrown from readahead
-            while (true) {
-                reader.readNext(false);
-            }
-        } catch (LogReadException readexc) {
-            exceptionEncountered = true;
-        } catch (LogNotFoundException exc) {
-            exceptionEncountered = true;
-        }
-        assertTrue(exceptionEncountered);
-        reader.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testImmediateFlush() throws Exception {
-        String name = "distrlog-immediate-flush";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setOutputBufferSize(0);
-        testNonPartitionedWritesInternal(name, confLocal);
-    }
-
-    @Test(timeout = 60000)
-    public void testLastLogRecordWithEmptyLedgers() throws Exception {
-        String name = "distrlog-lastLogRec-emptyledgers";
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        long txid = 1;
-        for (long i = 0; i < 3; i++) {
-            long start = txid;
-            BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
-                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                out.write(op);
-            }
-            BKLogSegmentWriter perStreamLogWriter = out.getCachedLogWriter();
-            out.closeAndComplete();
-            BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
-
-            assertNotNull(
-                zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
-                                                      perStreamLogWriter.getLogSegmentSequenceNumber()), false));
-            BKLogSegmentWriter writer = blplm.startLogSegment(txid - 1);
-            blplm.completeAndCloseLogSegment(writer.getLogSegmentSequenceNumber(),
-                    writer.getLogSegmentId(), txid - 1, txid - 1, 0);
-            assertNotNull(
-                zkc.exists(blplm.completedLedgerZNode(txid - 1, txid - 1,
-                                                      writer.getLogSegmentSequenceNumber()), false));
-            FutureUtils.result(blplm.asyncClose());
-        }
-
-        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-        LogRecord op = DLMTestUtil.getLogRecordInstance(txid);
-        op.setControl();
-        out.write(op);
-        out.setReadyToFlush();
-        out.flushAndSync();
-        out.abort();
-        dlm.close();
-
-        dlm = createNewDLM(conf, name);
-
-        assertEquals(txid - 1, dlm.getLastTxId());
-        LogRecord last = dlm.getLastLogRecord();
-        assertEquals(txid - 1, last.getTransactionId());
-        DLMTestUtil.verifyLogRecord(last);
-        assertEquals(txid - 1, dlm.getLogRecordCount());
-
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testLogSegmentListener() throws Exception {
-        String name = "distrlog-logsegment-listener";
-        int numSegments = 3;
-        final CountDownLatch[] latches = new CountDownLatch[numSegments + 1];
-        for (int i = 0; i < numSegments + 1; i++) {
-            latches[i] = new CountDownLatch(1);
-        }
-
-        final AtomicInteger numFailures = new AtomicInteger(0);
-        final AtomicReference<Collection<LogSegmentMetadata>> receivedStreams =
-                new AtomicReference<Collection<LogSegmentMetadata>>();
-
-        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
-
-        FutureUtils.result(dlm.getWriterMetadataStore().getLog(dlm.getUri(), name, true, true));
-        dlm.registerListener(new LogSegmentListener() {
-            @Override
-            public void onSegmentsUpdated(List<LogSegmentMetadata> segments) {
-                int updates = segments.size();
-                boolean hasIncompletedLogSegments = false;
-                for (LogSegmentMetadata l : segments) {
-                    if (l.isInProgress()) {
-                        hasIncompletedLogSegments = true;
-                        break;
-                    }
-                }
-                if (hasIncompletedLogSegments) {
-                    return;
-                }
-                if (updates >= 1) {
-                    if (segments.get(segments.size() - 1).getLogSegmentSequenceNumber() != updates) {
-                        numFailures.incrementAndGet();
-                    }
-                }
-                receivedStreams.set(segments);
-                latches[updates].countDown();
-            }
-
-            @Override
-            public void onLogStreamDeleted() {
-                // no-op
-            }
-        });
-        long txid = 1;
-        for (int i = 0; i < numSegments; i++) {
-            LOG.info("Waiting for creating log segment {}.", i);
-            latches[i].await();
-            LOG.info("Creating log segment {}.", i);
-            BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
-            LOG.info("Created log segment {}.", i);
-            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
-                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                out.write(op);
-            }
-            out.closeAndComplete();
-            LOG.info("Completed log segment {}.", i);
-        }
-        latches[numSegments].await();
-        assertEquals(0, numFailures.get());
-        assertNotNull(receivedStreams.get());
-        assertEquals(numSegments, receivedStreams.get().size());
-        int seqno = 1;
-        for (LogSegmentMetadata m : receivedStreams.get()) {
-            assertEquals(seqno, m.getLogSegmentSequenceNumber());
-            assertEquals((seqno - 1) * DEFAULT_SEGMENT_SIZE + 1, m.getFirstTxId());
-            assertEquals(seqno * DEFAULT_SEGMENT_SIZE, m.getLastTxId());
-            ++seqno;
-        }
-
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLastDLSN() throws Exception {
-        String name = "distrlog-get-last-dlsn";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setFirstNumEntriesPerReadLastRecordScan(2);
-        confLocal.setMaxNumEntriesPerReadLastRecordScan(4);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setOutputBufferSize(0);
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-        long txid = 1;
-        LOG.info("Writing 10 control records");
-        for (int i = 0; i < 10; i++) {
-            LogRecord record = DLMTestUtil.getLogRecordInstance(txid++);
-            record.setControl();
-            Await.result(writer.writeControlRecord(record));
-        }
-        LOG.info("10 control records are written");
-
-        try {
-            dlm.getLastDLSN();
-            fail("Should fail on getting last dlsn from an empty log.");
-        } catch (LogEmptyException lee) {
-            // expected
-        }
-
-        writer.closeAndComplete();
-        LOG.info("Completed first log segment");
-
-        writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-        Await.result(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));
-        }
-
-        assertEquals(new DLSN(2, 0, 0), dlm.getLastDLSN());
-
-        writer.closeAndComplete();
-        LOG.info("Completed third log segment");
-
-        assertEquals(new DLSN(2, 0, 0), dlm.getLastDLSN());
-
-        writer.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordCountAsync() throws Exception {
-        DistributedLogManager dlm = createNewDLM(conf, testNames.getMethodName());
-        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));
-        assertEquals(20, count.longValue());
-
-        writer.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testInvalidStreamFromInvalidZkPath() throws Exception {
-        String baseName = testNames.getMethodName();
-        String streamName = "\0blah";
-        URI uri = createDLMURI("/" + baseName);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf).uri(uri).build();
-
-        DistributedLogManager dlm = null;
-        AsyncLogWriter writer = null;
-        try {
-            dlm = namespace.openLog(streamName);
-            writer = dlm.startAsyncLogSegmentNonPartitioned();
-            fail("should have thrown");
-        } catch (InvalidStreamNameException e) {
-        } finally {
-            if (null != writer) {
-                Utils.close(writer);
-            }
-            if (null != dlm) {
-                dlm.close();
-            }
-            namespace.close();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testTruncationValidation() throws Exception {
-        String name = "distrlog-truncation-validation";
-        URI uri = createDLMURI("/" + name);
-        ZooKeeperClient zookeeperClient = TestZooKeeperClientBuilder.newBuilder()
-            .uri(uri)
-            .build();
-        OrderedScheduler scheduler = OrderedScheduler.newBuilder()
-                .name("test-truncation-validation")
-                .corePoolSize(1)
-                .build();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setDLLedgerMetadataLayoutVersion(LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setLogSegmentCacheEnabled(false);
-
-        LogSegmentMetadataStore metadataStore = new ZKLogSegmentMetadataStore(confLocal, zookeeperClient, scheduler);
-
-        BKDistributedLogManager dlm = createNewDLM(confLocal, name);
-        DLSN truncDLSN = DLSN.InitialDLSN;
-        DLSN beyondTruncDLSN = DLSN.InitialDLSN;
-        long beyondTruncTxId = 1;
-        long txid = 1;
-        for (long i = 0; i < 3; i++) {
-            long start = txid;
-            BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-            for (long j = 1; j <= 10; j++) {
-                LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
-                Future<DLSN> dlsn = writer.write(record);
-
-                if (i == 1 && j == 2) {
-                    truncDLSN = Await.result(dlsn);
-                } else if (i == 2 && j == 3) {
-                    beyondTruncDLSN = Await.result(dlsn);
-                    beyondTruncTxId = record.getTransactionId();
-                } else if (j == 10) {
-                    Await.ready(dlsn);
-                }
-            }
-
-            writer.close();
-        }
-
-        {
-            LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
-            LogRecordWithDLSN record = reader.readNext(false);
-            assertTrue((record != null) && (record.getDlsn().compareTo(DLSN.InitialDLSN) == 0));
-            reader.close();
-        }
-
-        Map<Long, LogSegmentMetadata> segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
-                LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName()));
-
-        LOG.info("Read segments before truncating first segment : {}", segmentList);
-
-        MetadataUpdater updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(
-                confLocal, metadataStore);
-        FutureUtils.result(updater.setLogSegmentTruncated(segmentList.get(1L)));
-
-        segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
-                LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName()));
-
-        LOG.info("Read segments after truncated first segment : {}", segmentList);
-
-        {
-            LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
-            LogRecordWithDLSN record = reader.readNext(false);
-            assertTrue("Unexpected record : " + record,
-                    (record != null) && (record.getDlsn().compareTo(new DLSN(2, 0, 0)) == 0));
-            reader.close();
-        }
-
-        {
-            LogReader reader = dlm.getInputStream(1);
-            LogRecordWithDLSN record = reader.readNext(false);
-            assertTrue((record != null) && (record.getDlsn().compareTo(new DLSN(2, 0, 0)) == 0));
-            reader.close();
-        }
-
-        updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, metadataStore);
-        FutureUtils.result(updater.setLogSegmentActive(segmentList.get(1L)));
-
-        segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
-                LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName()));
-
-        LOG.info("Read segments after marked first segment as active : {}", segmentList);
-
-        updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, metadataStore);
-        FutureUtils.result(updater.setLogSegmentTruncated(segmentList.get(2L)));
-
-        segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
-                LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName()));
-
-        LOG.info("Read segments after truncated second segment : {}", segmentList);
-
-        {
-            AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN);
-            long expectedTxId = 1L;
-            boolean exceptionEncountered = false;
-            try {
-                for (int i = 0; i < 3 * 10; i++) {
-                    LogRecordWithDLSN record = Await.result(reader.readNext());
-                    DLMTestUtil.verifyLargeLogRecord(record);
-                    assertEquals(expectedTxId, record.getTransactionId());
-                    expectedTxId++;
-                }
-            } catch (AlreadyTruncatedTransactionException exc) {
-                exceptionEncountered = true;
-            }
-            assertTrue(exceptionEncountered);
-            Utils.close(reader);
-        }
-
-        updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
-        FutureUtils.result(updater.setLogSegmentActive(segmentList.get(2L)));
-
-        BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-        Assert.assertTrue(Await.result(writer.truncate(truncDLSN)));
-        BKLogWriteHandler handler = writer.getCachedWriteHandler();
-        List<LogSegmentMetadata> cachedSegments = handler.getCachedLogSegments(LogSegmentMetadata.COMPARATOR);
-        for (LogSegmentMetadata segment: cachedSegments) {
-            if (segment.getLastDLSN().compareTo(truncDLSN) < 0) {
-                Assert.assertTrue(segment.isTruncated());
-                Assert.assertTrue(!segment.isPartiallyTruncated());
-            } else if (segment.getFirstDLSN().compareTo(truncDLSN) < 0) {
-                Assert.assertTrue(!segment.isTruncated());
-                Assert.assertTrue(segment.isPartiallyTruncated());
-            } else {
-                Assert.assertTrue(!segment.isTruncated());
-                Assert.assertTrue(!segment.isPartiallyTruncated());
-            }
-        }
-
-        segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
-                LogMetadata.getLogSegmentsPath(uri, name, conf.getUnpartitionedStreamName()));
-
-        Assert.assertTrue(segmentList.get(truncDLSN.getLogSegmentSequenceNo()).getMinActiveDLSN().compareTo(truncDLSN) == 0);
-
-        {
-            LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
-            LogRecordWithDLSN record = reader.readNext(false);
-            assertTrue(record != null);
-            assertEquals(truncDLSN, record.getDlsn());
-            reader.close();
-        }
-
-        {
-            LogReader reader = dlm.getInputStream(1);
-            LogRecordWithDLSN record = reader.readNext(false);
-            assertTrue(record != null);
-            assertEquals(truncDLSN, record.getDlsn());
-            reader.close();
-        }
-
-        {
-            AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN);
-            LogRecordWithDLSN record = Await.result(reader.readNext());
-            assertTrue(record != null);
-            assertEquals(truncDLSN, record.getDlsn());
-            Utils.close(reader);
-        }
-
-
-        {
-            LogReader reader = dlm.getInputStream(beyondTruncDLSN);
-            LogRecordWithDLSN record = reader.readNext(false);
-            assertTrue(record != null);
-            assertEquals(beyondTruncDLSN, record.getDlsn());
-            reader.close();
-        }
-
-        {
-            LogReader reader = dlm.getInputStream(beyondTruncTxId);
-            LogRecordWithDLSN record = reader.readNext(false);
-            assertTrue(record != null);
-            assertEquals(beyondTruncDLSN, record.getDlsn());
-            assertEquals(beyondTruncTxId, record.getTransactionId());
-            reader.close();
-        }
-
-        {
-            AsyncLogReader reader = dlm.getAsyncLogReader(beyondTruncDLSN);
-            LogRecordWithDLSN record = Await.result(reader.readNext());
-            assertTrue(record != null);
-            assertEquals(beyondTruncDLSN, record.getDlsn());
-            Utils.close(reader);
-        }
-
-        zookeeperClient.close();
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogNamespace.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogNamespace.java
deleted file mode 100644
index a8a82fa..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKDistributedLogNamespace.java
+++ /dev/null
@@ -1,440 +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 com.twitter.distributedlog;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.exceptions.AlreadyClosedException;
-import com.twitter.distributedlog.exceptions.InvalidStreamNameException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.util.DLUtils;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.data.Stat;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-import static org.hamcrest.core.IsNot.not;
-import static org.junit.Assert.*;
-
-public class TestBKDistributedLogNamespace extends TestDistributedLogBase {
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    static final Logger LOG = LoggerFactory.getLogger(TestBKDistributedLogNamespace.class);
-
-    protected static DistributedLogConfiguration conf =
-            new DistributedLogConfiguration().setLockTimeout(10)
-                .setEnableLedgerAllocatorPool(true).setLedgerAllocatorPoolName("test");
-
-    private ZooKeeperClient zooKeeperClient;
-
-    @Before
-    public void setup() throws Exception {
-        zooKeeperClient =
-            TestZooKeeperClientBuilder.newBuilder()
-                .uri(createDLMURI("/"))
-                .build();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        zooKeeperClient.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateIfNotExists() throws Exception {
-        URI uri = createDLMURI("/" + runtime.getMethodName());
-        ensureURICreated(zooKeeperClient.get(), uri);
-        DistributedLogConfiguration newConf = new DistributedLogConfiguration();
-        newConf.addConfiguration(conf);
-        newConf.setCreateStreamIfNotExists(false);
-        String streamName = "test-stream";
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(newConf).uri(uri).build();
-        DistributedLogManager dlm = namespace.openLog(streamName);
-        LogWriter writer;
-        try {
-            writer = dlm.startLogSegmentNonPartitioned();
-            writer.write(DLMTestUtil.getLogRecordInstance(1L));
-            fail("Should fail to write data if stream doesn't exist.");
-        } catch (IOException ioe) {
-            // expected
-        }
-        dlm.close();
-
-        // create the stream
-        namespace.createLog(streamName);
-
-        DistributedLogManager newDLM = namespace.openLog(streamName);
-        LogWriter newWriter = newDLM.startLogSegmentNonPartitioned();
-        newWriter.write(DLMTestUtil.getLogRecordInstance(1L));
-        newWriter.close();
-        newDLM.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testInvalidStreamName() throws Exception {
-        assertFalse(DLUtils.isReservedStreamName("test"));
-        assertTrue(DLUtils.isReservedStreamName(".test"));
-
-        URI uri = createDLMURI("/" + runtime.getMethodName());
-
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf).uri(uri).build();
-
-        try {
-            namespace.openLog(".test1");
-            fail("Should fail to create invalid stream .test");
-        } catch (InvalidStreamNameException isne) {
-            // expected
-        }
-
-        DistributedLogManager dlm = namespace.openLog("test1");
-        LogWriter writer = dlm.startLogSegmentNonPartitioned();
-        writer.write(DLMTestUtil.getLogRecordInstance(1));
-        writer.close();
-        dlm.close();
-
-        try {
-            namespace.openLog(".test2");
-            fail("Should fail to create invalid stream .test2");
-        } catch (InvalidStreamNameException isne) {
-            // expected
-        }
-
-        try {
-            namespace.openLog("/test2");
-            fail("should fail to create invalid stream /test2");
-        } catch (InvalidStreamNameException isne) {
-            // expected
-        }
-
-        try {
-            char[] chars = new char[6];
-            for (int i = 0; i < chars.length; i++) {
-                chars[i] = 'a';
-            }
-            chars[0] = 0;
-            String streamName = new String(chars);
-            namespace.openLog(streamName);
-            fail("should fail to create invalid stream " + streamName);
-        } catch (InvalidStreamNameException isne) {
-            // expected
-        }
-
-        try {
-            char[] chars = new char[6];
-            for (int i = 0; i < chars.length; i++) {
-                chars[i] = 'a';
-            }
-            chars[3] = '\u0010';
-            String streamName = new String(chars);
-            namespace.openLog(streamName);
-            fail("should fail to create invalid stream " + streamName);
-        } catch (InvalidStreamNameException isne) {
-            // expected
-        }
-
-        DistributedLogManager newDLM =
-                namespace.openLog("test_2-3");
-        LogWriter newWriter = newDLM.startLogSegmentNonPartitioned();
-        newWriter.write(DLMTestUtil.getLogRecordInstance(1));
-        newWriter.close();
-        newDLM.close();
-
-        Iterator<String> streamIter = namespace.getLogs();
-        Set<String> streamSet = Sets.newHashSet(streamIter);
-
-        assertEquals(2, streamSet.size());
-        assertTrue(streamSet.contains("test1"));
-        assertTrue(streamSet.contains("test_2-3"));
-
-        namespace.close();
-    }
-
-    @Test(timeout = 60000)
-    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()
-                .conf(conf).uri(uri).build();
-        final CountDownLatch[] latches = new CountDownLatch[3];
-        for (int i = 0; i < 3; i++) {
-            latches[i] = new CountDownLatch(1);
-        }
-        final AtomicInteger numUpdates = new AtomicInteger(0);
-        final AtomicInteger numFailures = new AtomicInteger(0);
-        final AtomicReference<Collection<String>> receivedStreams = new AtomicReference<Collection<String>>(null);
-        namespace.registerNamespaceListener(new NamespaceListener() {
-            @Override
-            public void onStreamsChanged(Iterator<String> streams) {
-                Set<String> streamSet = Sets.newHashSet(streams);
-                int updates = numUpdates.incrementAndGet();
-                if (streamSet.size() != updates - 1) {
-                    numFailures.incrementAndGet();
-                }
-
-                receivedStreams.set(streamSet);
-                latches[updates - 1].countDown();
-            }
-        });
-        latches[0].await();
-        namespace.createLog("test1");
-        latches[1].await();
-        namespace.createLog("test2");
-        latches[2].await();
-        assertEquals(0, numFailures.get());
-        assertNotNull(receivedStreams.get());
-        Set<String> streamSet = new HashSet<String>();
-        streamSet.addAll(receivedStreams.get());
-        assertEquals(2, receivedStreams.get().size());
-        assertEquals(2, streamSet.size());
-        assertTrue(streamSet.contains("test1"));
-        assertTrue(streamSet.contains("test2"));
-    }
-
-    private void initDlogMeta(String dlNamespace, String un, String streamName) throws Exception {
-        URI uri = createDLMURI(dlNamespace);
-        DistributedLogConfiguration newConf = new DistributedLogConfiguration();
-        newConf.addConfiguration(conf);
-        newConf.setCreateStreamIfNotExists(true);
-        newConf.setZkAclId(un);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(newConf).uri(uri).build();
-        DistributedLogManager dlm = namespace.openLog(streamName);
-        LogWriter writer = dlm.startLogSegmentNonPartitioned();
-        for (int i = 0; i < 10; i++) {
-            writer.write(DLMTestUtil.getLogRecordInstance(1L));
-        }
-        writer.close();
-        dlm.close();
-        namespace.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testAclPermsZkAccessConflict() throws Exception {
-
-        String namespace = "/" + runtime.getMethodName();
-        initDlogMeta(namespace, "test-un", "test-stream");
-        URI uri = createDLMURI(namespace);
-
-        ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder()
-            .name("unpriv")
-            .uri(uri)
-            .build();
-
-        try {
-            zkc.get().create(uri.getPath() + "/test-stream/test-garbage",
-                new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-            fail("write should have failed due to perms");
-        } catch (KeeperException.NoAuthException ex) {
-            LOG.info("caught exception trying to write with no perms", ex);
-        }
-
-        try {
-            zkc.get().setData(uri.getPath() + "/test-stream", new byte[0], 0);
-            fail("write should have failed due to perms");
-        } catch (KeeperException.NoAuthException ex) {
-            LOG.info("caught exception trying to write with no perms", ex);
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testAclPermsZkAccessNoConflict() throws Exception {
-
-        String namespace = "/" + runtime.getMethodName();
-        initDlogMeta(namespace, "test-un", "test-stream");
-        URI uri = createDLMURI(namespace);
-
-        ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder()
-            .name("unpriv")
-            .uri(uri)
-            .build();
-
-        zkc.get().getChildren(uri.getPath() + "/test-stream", false, new Stat());
-        zkc.get().getData(uri.getPath() + "/test-stream", false, new Stat());
-    }
-
-    @Test(timeout = 60000)
-    public void testAclModifyPermsDlmConflict() throws Exception {
-        String streamName = "test-stream";
-
-        // Reopening and writing again with the same un will succeed.
-        initDlogMeta("/" + runtime.getMethodName(), "test-un", streamName);
-
-        try {
-            // Reopening and writing again with a different un will fail.
-            initDlogMeta("/" + runtime.getMethodName(), "not-test-un", streamName);
-            fail("write should have failed due to perms");
-        } catch (ZKException ex) {
-            LOG.info("caught exception trying to write with no perms {}", ex);
-            assertEquals(KeeperException.Code.NOAUTH, ex.getKeeperExceptionCode());
-        } catch (Exception ex) {
-            LOG.info("caught wrong exception trying to write with no perms {}", ex);
-            fail("wrong exception " + ex.getClass().getName() + " expected " + LockingException.class.getName());
-        }
-
-        // Should work again.
-        initDlogMeta("/" + runtime.getMethodName(), "test-un", streamName);
-    }
-
-    @Test(timeout = 60000)
-    public void testAclModifyPermsDlmNoConflict() throws Exception {
-        String streamName = "test-stream";
-
-        // Establish the uri.
-        initDlogMeta("/" + runtime.getMethodName(), "test-un", streamName);
-
-        // Reopening and writing again with the same un will succeed.
-        initDlogMeta("/" + runtime.getMethodName(), "test-un", streamName);
-    }
-
-    static void validateBadAllocatorConfiguration(DistributedLogConfiguration conf, URI uri) throws Exception {
-        try {
-            BKNamespaceDriver.validateAndGetFullLedgerAllocatorPoolPath(conf, uri);
-            fail("Should throw exception when bad allocator configuration provided");
-        } catch (IOException ioe) {
-            // expected
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testValidateAndGetFullLedgerAllocatorPoolPath() throws Exception {
-        DistributedLogConfiguration testConf = new DistributedLogConfiguration();
-        testConf.setEnableLedgerAllocatorPool(true);
-
-        String namespace = "/" + runtime.getMethodName();
-        URI uri = createDLMURI(namespace);
-
-        testConf.setLedgerAllocatorPoolName("test");
-
-        testConf.setLedgerAllocatorPoolPath("test");
-        validateBadAllocatorConfiguration(testConf, uri);
-
-        testConf.setLedgerAllocatorPoolPath(".");
-        validateBadAllocatorConfiguration(testConf, uri);
-
-        testConf.setLedgerAllocatorPoolPath("..");
-        validateBadAllocatorConfiguration(testConf, uri);
-
-        testConf.setLedgerAllocatorPoolPath("./");
-        validateBadAllocatorConfiguration(testConf, uri);
-
-        testConf.setLedgerAllocatorPoolPath(".test/");
-        validateBadAllocatorConfiguration(testConf, uri);
-
-        testConf.setLedgerAllocatorPoolPath(".test");
-        testConf.setLedgerAllocatorPoolName(null);
-        validateBadAllocatorConfiguration(testConf, uri);
-    }
-
-    @Test(timeout = 60000)
-    public void testUseNamespaceAfterCloseShouldFailFast() throws Exception {
-        URI uri = createDLMURI("/" + runtime.getMethodName());
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-            .conf(conf)
-            .uri(uri)
-            .build();
-        // before closing the namespace, no exception should be thrown
-        String logName = "test-stream";
-        // create a log
-        namespace.createLog(logName);
-        // log exists
-        Assert.assertTrue(namespace.logExists(logName));
-        // create a dlm
-        DistributedLogManager dlm = namespace.openLog(logName);
-        // do some writes
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
-        for (long i = 0; i < 3; i++) {
-            LogRecord record = DLMTestUtil.getLargeLogRecordInstance(i);
-            writer.write(record);
-        }
-        writer.closeAndComplete();
-        // do some reads
-        LogReader reader = dlm.getInputStream(0);
-        for (long i = 0; i < 3; i++) {
-            Assert.assertEquals(reader.readNext(false).getTransactionId(), i);
-        }
-        namespace.deleteLog(logName);
-        Assert.assertFalse(namespace.logExists(logName));
-
-        // now try to close the namespace
-        namespace.close();
-        try {
-            namespace.createLog(logName);
-            fail("Should throw exception after namespace is closed");
-        } catch (AlreadyClosedException e) {
-            // No-ops
-        }
-        try {
-            namespace.openLog(logName);
-            fail("Should throw exception after namespace is closed");
-        } catch (AlreadyClosedException e) {
-            // No-ops
-        }
-        try {
-            namespace.logExists(logName);
-            fail("Should throw exception after namespace is closed");
-        } catch (AlreadyClosedException e) {
-            // No-ops
-        }
-        try {
-            namespace.getLogs();
-            fail("Should throw exception after namespace is closed");
-        } catch (AlreadyClosedException e) {
-            // No-ops
-        }
-        try {
-            namespace.deleteLog(logName);
-            fail("Should throw exception after namespace is closed");
-        } catch (AlreadyClosedException e) {
-            // No-ops
-        }
-        try {
-            namespace.createAccessControlManager();
-            fail("Should throw exception after namespace is closed");
-        } catch (AlreadyClosedException e) {
-            // No-ops
-        }
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogReadHandler.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogReadHandler.java
deleted file mode 100644
index 854cb74..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogReadHandler.java
+++ /dev/null
@@ -1,398 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.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;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.junit.Assert.*;
-
-/**
- * Test {@link BKLogReadHandler}
- */
-public class TestBKLogReadHandler extends TestDistributedLogBase {
-
-    static final Logger LOG = LoggerFactory.getLogger(TestBKLogReadHandler.class);
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    private void prepareLogSegmentsNonPartitioned(String name, int numSegments, int numEntriesPerSegment) throws Exception {
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        long txid = 1;
-        for (int sid = 0; sid < numSegments; ++sid) {
-            LogWriter out = dlm.startLogSegmentNonPartitioned();
-            for (int eid = 0; eid < numEntriesPerSegment; ++eid) {
-                LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
-                out.write(record);
-                ++txid;
-            }
-            out.close();
-        }
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testGetFirstDLSNWithOpenLedger() throws Exception {
-        String dlName = runtime.getMethodName();
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(0);
-
-        int numEntriesPerSegment = 10;
-        DistributedLogManager dlm1 = createNewDLM(confLocal, dlName);
-        long txid = 1;
-
-        ArrayList<Future<DLSN>> futures = new ArrayList<Future<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));
-        // commit
-        LogRecord controlRecord = new LogRecord(txid, DistributedLogConstants.CONTROL_RECORD_CONTENT);
-        controlRecord.setControl();
-        FutureUtils.result(out.write(controlRecord));
-
-        DLSN last = dlm1.getLastDLSN();
-        assertEquals(new DLSN(1,9,0), last);
-        DLSN first = Await.result(dlm1.getFirstDLSNAsync());
-        assertEquals(new DLSN(1,0,0), first);
-        Utils.close(out);
-    }
-
-    @Test(timeout = 60000)
-    public void testGetFirstDLSNNoLogSegments() throws Exception {
-        String dlName = runtime.getMethodName();
-        BKDistributedLogManager dlm = createNewDLM(conf, dlName);
-        BKLogReadHandler readHandler = dlm.createReadHandler();
-        Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
-        try {
-            Await.result(futureRecord);
-            fail("should have thrown exception");
-        } catch (LogNotFoundException ex) {
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testGetFirstDLSNWithLogSegments() throws Exception {
-        String dlName = runtime.getMethodName();
-        BKDistributedLogManager dlm = createNewDLM(conf, dlName);
-        DLMTestUtil.generateCompletedLogSegments(dlm, conf, 3, 3);
-        BKLogReadHandler readHandler = dlm.createReadHandler();
-        Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
-        try {
-            LogRecordWithDLSN record = Await.result(futureRecord);
-            assertEquals(new DLSN(1, 0, 0), record.getDlsn());
-        } catch (Exception ex) {
-            fail("should not have thrown exception: " + ex);
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testGetFirstDLSNAfterCleanTruncation() throws Exception {
-        String dlName = runtime.getMethodName();
-        prepareLogSegmentsNonPartitioned(dlName, 3, 10);
-        DistributedLogManager dlm = createNewDLM(conf, dlName);
-        BKLogReadHandler readHandler =
-            ((BKDistributedLogManager) dlm).createReadHandler();
-        AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-        Future<Boolean> futureSuccess = writer.truncate(new DLSN(2, 0, 0));
-        Boolean success = Await.result(futureSuccess);
-        assertTrue(success);
-        Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
-        LogRecordWithDLSN record = Await.result(futureRecord);
-        assertEquals(new DLSN(2, 0, 0), record.getDlsn());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetFirstDLSNAfterPartialTruncation() throws Exception {
-        String dlName = runtime.getMethodName();
-        prepareLogSegmentsNonPartitioned(dlName, 3, 10);
-        DistributedLogManager dlm = createNewDLM(conf, dlName);
-        BKLogReadHandler readHandler =
-            ((BKDistributedLogManager) dlm).createReadHandler();
-        AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-
-        // Only truncates at ledger boundary.
-        Future<Boolean> futureSuccess = writer.truncate(new DLSN(2, 5, 0));
-        Boolean success = Await.result(futureSuccess);
-        assertTrue(success);
-        Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
-        LogRecordWithDLSN record = Await.result(futureRecord);
-        assertEquals(new DLSN(2, 0, 0), record.getDlsn());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordCountEmptyLedger() throws Exception {
-        String dlName = runtime.getMethodName();
-        DistributedLogManager dlm = createNewDLM(conf, dlName);
-        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
-        count = readHandler.asyncGetLogRecordCount(DLSN.InitialDLSN);
-        try {
-            Await.result(count);
-            fail("log is empty, should have returned log empty ex");
-        } catch (LogNotFoundException ex) {
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordCountTotalCount() throws Exception {
-        String dlName = runtime.getMethodName();
-        prepareLogSegmentsNonPartitioned(dlName, 11, 3);
-        DistributedLogManager dlm = createNewDLM(conf, dlName);
-        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
-        count = readHandler.asyncGetLogRecordCount(DLSN.InitialDLSN);
-        assertEquals(33, Await.result(count).longValue());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordCountAtLedgerBoundary() throws Exception {
-        String dlName = runtime.getMethodName();
-        prepareLogSegmentsNonPartitioned(dlName, 11, 3);
-        DistributedLogManager dlm = createNewDLM(conf, dlName);
-        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
-        count = readHandler.asyncGetLogRecordCount(new DLSN(2, 0, 0));
-        assertEquals(30, Await.result(count).longValue());
-        count = readHandler.asyncGetLogRecordCount(new DLSN(3, 0, 0));
-        assertEquals(27, Await.result(count).longValue());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordCountPastEnd() throws Exception {
-        String dlName = runtime.getMethodName();
-        prepareLogSegmentsNonPartitioned(dlName, 11, 3);
-        DistributedLogManager dlm = createNewDLM(conf, dlName);
-        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
-        count = readHandler.asyncGetLogRecordCount(new DLSN(12, 0, 0));
-        assertEquals(0, Await.result(count).longValue());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordCountLastRecord() throws Exception {
-        String dlName = runtime.getMethodName();
-        prepareLogSegmentsNonPartitioned(dlName, 11, 3);
-        DistributedLogManager dlm = createNewDLM(conf, dlName);
-        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
-        count = readHandler.asyncGetLogRecordCount(new DLSN(11, 2, 0));
-        assertEquals(1, Await.result(count).longValue());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordCountInteriorRecords() throws Exception {
-        String dlName = runtime.getMethodName();
-        prepareLogSegmentsNonPartitioned(dlName, 5, 10);
-        DistributedLogManager dlm = createNewDLM(conf, dlName);
-        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
-        count = readHandler.asyncGetLogRecordCount(new DLSN(3, 5, 0));
-        assertEquals(25, Await.result(count).longValue());
-        count = readHandler.asyncGetLogRecordCount(new DLSN(2, 5, 0));
-        assertEquals(35, Await.result(count).longValue());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordCountWithControlRecords() throws Exception {
-        DistributedLogManager dlm = createNewDLM(conf, runtime.getMethodName());
-        long txid = 1;
-        txid += DLMTestUtil.generateLogSegmentNonPartitioned(dlm, 5, 5, txid);
-        txid += DLMTestUtil.generateLogSegmentNonPartitioned(dlm, 0, 10, txid);
-        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
-        count = readHandler.asyncGetLogRecordCount(new DLSN(1, 0, 0));
-        assertEquals(15, Await.result(count).longValue());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordCountWithAllControlRecords() throws Exception {
-        DistributedLogManager dlm = createNewDLM(conf, runtime.getMethodName());
-        long txid = 1;
-        txid += DLMTestUtil.generateLogSegmentNonPartitioned(dlm, 5, 0, txid);
-        txid += DLMTestUtil.generateLogSegmentNonPartitioned(dlm, 10, 0, txid);
-        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
-        Future<Long> count = null;
-        count = readHandler.asyncGetLogRecordCount(new DLSN(1, 0, 0));
-        assertEquals(0, Await.result(count).longValue());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordCountWithSingleInProgressLedger() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
-
-        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)));
-
-        BKLogReadHandler readHandler = bkdlm.createReadHandler();
-        List<LogSegmentMetadata> ledgerList = FutureUtils.result(
-                readHandler.readLogSegmentsFromStore(
-                        LogSegmentMetadata.COMPARATOR,
-                        LogSegmentFilter.DEFAULT_FILTER,
-                        null
-                )
-        ).getValue();
-        assertEquals(1, ledgerList.size());
-        assertTrue(ledgerList.get(0).isInProgress());
-
-        Future<Long> count = null;
-        count = readHandler.asyncGetLogRecordCount(new DLSN(1, 0, 0));
-        assertEquals(2, Await.result(count).longValue());
-
-        Utils.close(out);
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordCountWithCompletedAndInprogressLedgers() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
-
-        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)));
-
-        BKLogReadHandler readHandler = bkdlm.createReadHandler();
-        List<LogSegmentMetadata> ledgerList = FutureUtils.result(
-                readHandler.readLogSegmentsFromStore(
-                        LogSegmentMetadata.COMPARATOR,
-                        LogSegmentFilter.DEFAULT_FILTER,
-                        null)
-        ).getValue();
-        assertEquals(2, ledgerList.size());
-        assertFalse(ledgerList.get(0).isInProgress());
-        assertTrue(ledgerList.get(1).isInProgress());
-
-        Future<Long> count = null;
-        count = readHandler.asyncGetLogRecordCount(new DLSN(1, 0, 0));
-        assertEquals(7, Await.result(count).longValue());
-
-        Utils.close(out);
-    }
-
-    @Test(timeout = 60000)
-    public void testLockStreamWithMissingLog() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
-        BKLogReadHandler readHandler = bkdlm.createReadHandler();
-        try {
-            Await.result(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());
-            fail("Subscriber should fail lock stream if log not found");
-        } catch (LogNotFoundException ex) {
-            // expected
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testLockStreamDifferentSubscribers() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
-        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, 1);
-        BKLogReadHandler readHandler = bkdlm.createReadHandler();
-        Await.result(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());
-        BKDistributedLogManager bkdlm20 = createNewDLM(conf, streamName);
-        BKLogReadHandler s20Handler =
-                bkdlm20.createReadHandler(Optional.of("s2"));
-        Await.result(s20Handler.lockStream());
-
-        readHandler.asyncClose();
-        bkdlm.close();
-        s10Handler.asyncClose();
-        bkdlm10.close();
-        s20Handler.asyncClose();
-        bkdlm20.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testLockStreamSameSubscriber() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
-        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, 1);
-        BKLogReadHandler readHandler = bkdlm.createReadHandler();
-        Await.result(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());
-
-        BKDistributedLogManager bkdlm11 = createNewDLM(conf, streamName);
-        BKLogReadHandler s11Handler =
-                bkdlm11.createReadHandler(Optional.of("s1"));
-        try {
-            Await.result(s11Handler.lockStream(), Duration.apply(10000, TimeUnit.MILLISECONDS));
-            fail("Should fail lock stream using same subscriber id");
-        } catch (OwnershipAcquireFailedException oafe) {
-            // expected
-        } catch (TimeoutException te) {
-            // expected.
-        }
-
-        readHandler.asyncClose();
-        bkdlm.close();
-        s10Handler.asyncClose();
-        bkdlm10.close();
-        s11Handler.asyncClose();
-        bkdlm11.close();
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogSegmentWriter.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogSegmentWriter.java
deleted file mode 100644
index 8f86192..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogSegmentWriter.java
+++ /dev/null
@@ -1,790 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.exceptions.BKTransmitException;
-import com.twitter.distributedlog.exceptions.EndOfStreamException;
-import com.twitter.distributedlog.exceptions.WriteCancelledException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
-import com.twitter.distributedlog.io.Abortables;
-import com.twitter.distributedlog.lock.SessionLockFactory;
-import com.twitter.distributedlog.lock.ZKDistributedLock;
-import com.twitter.distributedlog.lock.ZKSessionLockFactory;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.PermitLimiter;
-import com.twitter.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;
-import org.apache.bookkeeper.feature.SettableFeatureProvider;
-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;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
-
-/**
- * Test Case for BookKeeper Based Log Segment Writer
- */
-public class TestBKLogSegmentWriter extends TestDistributedLogBase {
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    private OrderedScheduler scheduler;
-    private OrderedScheduler lockStateExecutor;
-    private ZooKeeperClient zkc;
-    private ZooKeeperClient zkc0;
-    private BookKeeperClient bkc;
-
-    @Before
-    @Override
-    public void setup() throws Exception {
-        super.setup();
-        scheduler = OrderedScheduler.newBuilder().corePoolSize(1).build();
-        lockStateExecutor = OrderedScheduler.newBuilder().corePoolSize(1).build();
-        // build zookeeper client
-        URI uri = createDLMURI("");
-        zkc = TestZooKeeperClientBuilder.newBuilder(conf)
-                .name("test-zkc")
-                .uri(uri)
-                .build();
-        zkc0 = TestZooKeeperClientBuilder.newBuilder(conf)
-                .name("test-zkc0")
-                .uri(uri)
-                .build();
-        // build bookkeeper client
-        BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
-        bkc = BookKeeperClientBuilder.newBuilder()
-                .dlConfig(conf)
-                .name("test-bkc")
-                .ledgersPath(bkdlConfig.getBkLedgersPath())
-                .zkServers(BKNamespaceDriver.getZKServersFromDLUri(uri))
-                .build();
-    }
-
-    @After
-    @Override
-    public void teardown() throws Exception {
-        if (null != bkc) {
-            bkc.close();
-        }
-        if (null != zkc) {
-            zkc.close();
-        }
-        if (null != lockStateExecutor) {
-            lockStateExecutor.shutdown();
-        }
-        if (null != scheduler) {
-            scheduler.shutdown();
-        }
-        super.teardown();
-    }
-
-    private DistributedLogConfiguration newLocalConf() {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        return confLocal;
-    }
-
-    private ZKDistributedLock createLock(String path,
-                                         ZooKeeperClient zkClient,
-                                         boolean acquireLock)
-            throws Exception {
-        try {
-            Await.result(Utils.zkAsyncCreateFullPathOptimistic(zkClient, path, new byte[0],
-                    ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT));
-        } catch (KeeperException.NodeExistsException nee) {
-            // node already exists
-        }
-        SessionLockFactory lockFactory = new ZKSessionLockFactory(
-                zkClient,
-                "test-lock",
-                lockStateExecutor,
-                0,
-                Long.MAX_VALUE,
-                conf.getZKSessionTimeoutMilliseconds(),
-                NullStatsLogger.INSTANCE
-        );
-        ZKDistributedLock lock = new ZKDistributedLock(
-                lockStateExecutor,
-                lockFactory,
-                path,
-                Long.MAX_VALUE,
-                NullStatsLogger.INSTANCE);
-        if (acquireLock) {
-            return FutureUtils.result(lock.asyncAcquire());
-        } else {
-            return lock;
-        }
-    }
-
-    private void closeWriterAndLock(BKLogSegmentWriter writer,
-                                    ZKDistributedLock lock)
-            throws IOException {
-        try {
-            FutureUtils.result(writer.asyncClose());
-        } finally {
-            Utils.closeQuietly(lock);
-        }
-    }
-
-    private void abortWriterAndLock(BKLogSegmentWriter writer,
-                                    ZKDistributedLock lock)
-            throws IOException {
-        try {
-            Abortables.abort(writer, false);
-        } finally {
-            Utils.closeQuietly(lock);
-        }
-    }
-
-    private BKLogSegmentWriter createLogSegmentWriter(DistributedLogConfiguration conf,
-                                                      long logSegmentSequenceNumber,
-                                                      long startTxId,
-                                                      ZKDistributedLock lock) throws Exception {
-        LedgerHandle lh = bkc.get().createLedger(3, 2, 2,
-                BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
-        return new BKLogSegmentWriter(
-                runtime.getMethodName(),
-                runtime.getMethodName(),
-                conf,
-                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION,
-                new BKLogSegmentEntryWriter(lh),
-                lock,
-                startTxId,
-                logSegmentSequenceNumber,
-                scheduler,
-                NullStatsLogger.INSTANCE,
-                NullStatsLogger.INSTANCE,
-                new AlertStatsLogger(NullStatsLogger.INSTANCE, "test"),
-                PermitLimiter.NULL_PERMIT_LIMITER,
-                new SettableFeatureProvider("", 0),
-                ConfUtils.getConstDynConf(conf));
-    }
-
-    private LedgerHandle openLedgerNoRecovery(LedgerHandle lh) throws Exception {
-        return bkc.get().openLedgerNoRecovery(lh.getId(),
-                BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
-    }
-
-    private LedgerHandle openLedger(LedgerHandle lh) throws Exception {
-        return bkc.get().openLedger(lh.getId(),
-                BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
-    }
-
-    private void fenceLedger(LedgerHandle lh) throws Exception {
-        bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32,
-                conf.getBKDigestPW().getBytes(UTF_8));
-    }
-
-    /**
-     * Close a segment log writer should flush buffered data.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testCloseShouldFlush() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
-        BKLogSegmentWriter writer =
-                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();
-        // add 10 records
-        int numRecords = 10;
-        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
-        for (int i = 0; i < numRecords; i++) {
-            futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
-        }
-        assertEquals("Last tx id should be " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxId());
-        assertEquals("Last acked tx id should be -1",
-                -1L, writer.getLastTxIdAcknowledged());
-        assertEquals("Last DLSN should be " + DLSN.InvalidDLSN,
-                DLSN.InvalidDLSN, writer.getLastDLSN());
-        assertEquals("Position should be " + numRecords,
-                10, writer.getPositionWithinLogSegment());
-        // close the writer should flush buffered data and release lock
-        closeWriterAndLock(writer, lock);
-        Await.result(lockFuture0);
-        lock0.checkOwnership();
-        assertEquals("Last tx id should still be " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxId());
-        assertEquals("Last acked tx id should become " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxIdAcknowledged());
-        assertEquals("Position should still be " + numRecords,
-                10, writer.getPositionWithinLogSegment());
-        List<DLSN> dlsns = Await.result(Future.collect(futureList));
-        assertEquals("All records should be written",
-                numRecords, dlsns.size());
-        for (int i = 0; i < numRecords; i++) {
-            DLSN dlsn = dlsns.get(i);
-            assertEquals("Incorrent ledger sequence number",
-                    0L, dlsn.getLogSegmentSequenceNo());
-            assertEquals("Incorrent entry id",
-                    0L, dlsn.getEntryId());
-            assertEquals("Inconsistent slot id",
-                    i, dlsn.getSlotId());
-        }
-        assertEquals("Last DLSN should be " + dlsns.get(dlsns.size() - 1),
-                dlsns.get(dlsns.size() - 1), writer.getLastDLSN());
-        LedgerHandle lh = getLedgerHandle(writer);
-        LedgerHandle readLh = openLedgerNoRecovery(lh);
-        assertTrue("Ledger " + lh.getId() + " should be closed", readLh.isClosed());
-        assertEquals("There should be two entries in ledger " + lh.getId(),
-                1L, readLh.getLastAddConfirmed());
-    }
-
-    /**
-     * Abort a segment log writer should just abort pending writes and not flush buffered data.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testAbortShouldNotFlush() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
-        BKLogSegmentWriter writer =
-                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();
-        // add 10 records
-        int numRecords = 10;
-        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
-        for (int i = 0; i < numRecords; i++) {
-            futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
-        }
-        assertEquals("Last tx id should be " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxId());
-        assertEquals("Last acked tx id should be -1",
-                -1L, writer.getLastTxIdAcknowledged());
-        assertEquals("Last DLSN should be " + DLSN.InvalidDLSN,
-                DLSN.InvalidDLSN, writer.getLastDLSN());
-        assertEquals("Position should be " + numRecords,
-                10, writer.getPositionWithinLogSegment());
-        // close the writer should flush buffered data and release lock
-        abortWriterAndLock(writer, lock);
-        Await.result(lockFuture0);
-        lock0.checkOwnership();
-        assertEquals("Last tx id should still be " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxId());
-        assertEquals("Last acked tx id should still be " + (numRecords - 1),
-                -1L, writer.getLastTxIdAcknowledged());
-        assertEquals("Last DLSN should still be " + DLSN.InvalidDLSN,
-                DLSN.InvalidDLSN, writer.getLastDLSN());
-        assertEquals("Position should still be " + numRecords,
-                10, writer.getPositionWithinLogSegment());
-
-        for (int i = 0; i < numRecords; i++) {
-            try {
-                Await.result(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",
-                        wce.getCause() instanceof BKTransmitException);
-                BKTransmitException bkte = (BKTransmitException) wce.getCause();
-                assertEquals("Record " + i + " should be aborted",
-                        BKException.Code.InterruptedException, bkte.getBKResultCode());
-            }
-        }
-
-        // check no entries were written
-        LedgerHandle lh = getLedgerHandle(writer);
-        LedgerHandle readLh = openLedgerNoRecovery(lh);
-        assertTrue("Ledger " + lh.getId() + " should not be closed", readLh.isClosed());
-        assertEquals("There should be no entries in ledger " + lh.getId(),
-                LedgerHandle.INVALID_ENTRY_ID, readLh.getLastAddConfirmed());
-    }
-
-
-    /**
-     * Close a log segment writer that already detect ledger fenced, should not flush buffered data.
-     * And should throw exception on closing.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testCloseShouldNotFlushIfLedgerFenced() throws Exception {
-        testCloseShouldNotFlushIfInErrorState(BKException.Code.LedgerFencedException);
-    }
-
-    /**
-     * Close a log segment writer that is already in error state, should not flush buffered data.
-     *
-     * @throws Exception
-     */
-    void testCloseShouldNotFlushIfInErrorState(int rcToFailComplete) throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
-        BKLogSegmentWriter writer =
-                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();
-        // add 10 records
-        int numRecords = 10;
-        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
-        for (int i = 0; i < numRecords; i++) {
-            futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
-        }
-        assertEquals("Last tx id should be " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxId());
-        assertEquals("Last acked tx id should be -1",
-                -1L, writer.getLastTxIdAcknowledged());
-        assertEquals("Last DLSN should be " + DLSN.InvalidDLSN,
-                DLSN.InvalidDLSN, writer.getLastDLSN());
-        assertEquals("Position should be " + numRecords,
-                10, writer.getPositionWithinLogSegment());
-        writer.setTransmitResult(rcToFailComplete);
-        // close the writer should release lock but not flush data
-        try {
-            closeWriterAndLock(writer, lock);
-            fail("Close a log segment writer in error state should throw exception");
-        } catch (BKTransmitException bkte) {
-            assertEquals("Inconsistent rc is thrown",
-                    rcToFailComplete, bkte.getBKResultCode());
-        }
-        Await.result(lockFuture0);
-        lock0.checkOwnership();
-        assertEquals("Last tx id should still be " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxId());
-        assertEquals("Last acked tx id should still be " + (numRecords - 1),
-                -1L, writer.getLastTxIdAcknowledged());
-        assertEquals("Last DLSN should still be " + DLSN.InvalidDLSN,
-                DLSN.InvalidDLSN, writer.getLastDLSN());
-        assertEquals("Position should still be " + numRecords,
-                10, writer.getPositionWithinLogSegment());
-
-        for (int i = 0; i < numRecords; i++) {
-            try {
-                Await.result(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",
-                        wce.getCause() instanceof BKTransmitException);
-                BKTransmitException bkte = (BKTransmitException) wce.getCause();
-                assertEquals("Record " + i + " should be aborted",
-                        rcToFailComplete, bkte.getBKResultCode());
-            }
-        }
-
-        // check no entries were written
-        LedgerHandle lh = getLedgerHandle(writer);
-        LedgerHandle readLh = openLedgerNoRecovery(lh);
-        assertFalse("Ledger " + lh.getId() + " should not be closed", readLh.isClosed());
-        assertEquals("There should be no entries in ledger " + lh.getId(),
-                LedgerHandle.INVALID_ENTRY_ID, readLh.getLastAddConfirmed());
-    }
-
-    /**
-     * Close the writer when ledger is fenced: it should release the lock, fail on flushing data and throw exception
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testCloseShouldFailIfLedgerFenced() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
-        BKLogSegmentWriter writer =
-                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();
-        // add 10 records
-        int numRecords = 10;
-        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
-        for (int i = 0; i < numRecords; i++) {
-            futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
-        }
-        assertEquals("Last tx id should be " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxId());
-        assertEquals("Last acked tx id should be -1",
-                -1L, writer.getLastTxIdAcknowledged());
-        assertEquals("Last DLSN should be " + DLSN.InvalidDLSN,
-                DLSN.InvalidDLSN, writer.getLastDLSN());
-        assertEquals("Position should be " + numRecords,
-                10, writer.getPositionWithinLogSegment());
-        // fence the ledger
-        fenceLedger(getLedgerHandle(writer));
-        // close the writer: it should release the lock, fail on flushing data and throw exception
-        try {
-            closeWriterAndLock(writer, lock);
-            fail("Close a log segment writer when ledger is fenced should throw exception");
-        } catch (BKTransmitException bkte) {
-            assertEquals("Inconsistent rc is thrown",
-                    BKException.Code.LedgerFencedException, bkte.getBKResultCode());
-        }
-
-        Await.result(lockFuture0);
-        lock0.checkOwnership();
-
-        assertEquals("Last tx id should still be " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxId());
-        assertEquals("Last acked tx id should still be " + (numRecords - 1),
-                -1L, writer.getLastTxIdAcknowledged());
-        assertEquals("Last DLSN should still be " + DLSN.InvalidDLSN,
-                DLSN.InvalidDLSN, writer.getLastDLSN());
-        assertEquals("Position should still be " + numRecords,
-                10, writer.getPositionWithinLogSegment());
-
-        for (int i = 0; i < numRecords; i++) {
-            try {
-                Await.result(futureList.get(i));
-                fail("Should be aborted record " + i + " with transmit exception");
-            } catch (BKTransmitException bkte) {
-                assertEquals("Record " + i + " should be aborted",
-                        BKException.Code.LedgerFencedException, bkte.getBKResultCode());
-            }
-        }
-
-        // check no entries were written
-        LedgerHandle lh = getLedgerHandle(writer);
-        LedgerHandle readLh = openLedgerNoRecovery(lh);
-        assertTrue("Ledger " + lh.getId() + " should be closed", readLh.isClosed());
-        assertEquals("There should be no entries in ledger " + lh.getId(),
-                LedgerHandle.INVALID_ENTRY_ID, readLh.getLastAddConfirmed());
-    }
-
-    /**
-     * Abort should wait for outstanding transmits to be completed and cancel buffered data.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testAbortShouldFailAllWrites() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
-        BKLogSegmentWriter writer =
-                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();
-        // add 10 records
-        int numRecords = 10;
-        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
-        for (int i = 0; i < numRecords; i++) {
-            futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
-        }
-        assertEquals("Last tx id should be " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxId());
-        assertEquals("Last acked tx id should be -1",
-                -1L, writer.getLastTxIdAcknowledged());
-        assertEquals("Last DLSN should be " + DLSN.InvalidDLSN,
-                DLSN.InvalidDLSN, writer.getLastDLSN());
-        assertEquals("Position should be " + numRecords,
-                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;
-            }
-        });
-
-        // transmit the buffered data
-        FutureUtils.result(writer.flush());
-
-        // add another 10 records
-        List<Future<DLSN>> anotherFutureList = new ArrayList<Future<DLSN>>(numRecords);
-        for (int i = numRecords; i < 2 * numRecords; i++) {
-            anotherFutureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
-        }
-        assertEquals("Last tx id should become " + (2 * numRecords - 1),
-                2 * numRecords - 1, writer.getLastTxId());
-        assertEquals("Last acked tx id should become " + (numRecords - 1),
-                (long) (numRecords - 1), writer.getLastTxIdAcknowledged());
-        assertEquals("Last DLSN should still be " + DLSN.InvalidDLSN,
-                DLSN.InvalidDLSN, writer.getLastDLSN());
-        assertEquals("Position should become " + (2 * numRecords),
-                2 * numRecords, writer.getPositionWithinLogSegment());
-
-        // abort the writer: it waits for outstanding transmits and abort buffered data
-        abortWriterAndLock(writer, lock);
-
-        Await.result(lockFuture0);
-        lock0.checkOwnership();
-
-        // release defer latch so completion would go through
-        deferLatch.countDown();
-
-        List<DLSN> dlsns = Await.result(Future.collect(futureList));
-        assertEquals("All first 10 records should be written",
-                numRecords, dlsns.size());
-        for (int i = 0; i < numRecords; i++) {
-            DLSN dlsn = dlsns.get(i);
-            assertEquals("Incorrent ledger sequence number",
-                    0L, dlsn.getLogSegmentSequenceNo());
-            assertEquals("Incorrent entry id",
-                    0L, dlsn.getEntryId());
-            assertEquals("Inconsistent slot id",
-                    i, dlsn.getSlotId());
-        }
-        for (int i = 0; i < numRecords; i++) {
-            try {
-                Await.result(anotherFutureList.get(i));
-                fail("Should be aborted record " + (numRecords + i) + " with transmit exception");
-            } catch (WriteCancelledException wce) {
-                // writes should be cancelled.
-            }
-        }
-
-        assertEquals("Last tx id should still be " + (2 * numRecords - 1),
-                2 * numRecords - 1, writer.getLastTxId());
-        assertEquals("Last acked tx id should be still " + (numRecords - 1),
-                (long) (numRecords - 1), writer.getLastTxIdAcknowledged());
-        assertEquals("Last DLSN should become " + futureList.get(futureList.size() - 1),
-                dlsns.get(futureList.size() - 1), writer.getLastDLSN());
-        assertEquals("Position should become " + 2 * numRecords,
-                2 * numRecords, writer.getPositionWithinLogSegment());
-
-        // check only 1 entry were written
-        LedgerHandle lh = getLedgerHandle(writer);
-        LedgerHandle readLh = openLedgerNoRecovery(lh);
-        assertTrue("Ledger " + lh.getId() + " should not be closed", readLh.isClosed());
-        assertEquals("Only one entry is written for ledger " + lh.getId(),
-                0L, lh.getLastAddPushed());
-        assertEquals("Only one entry is written for ledger " + lh.getId(),
-                0L, readLh.getLastAddConfirmed());
-    }
-
-    /**
-     * Log Segment Writer should only update last tx id only for user records.
-     */
-    @Test(timeout = 60000)
-    public void testUpdateLastTxIdForUserRecords() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
-        BKLogSegmentWriter writer =
-                createLogSegmentWriter(confLocal, 0L, -1L, lock);
-        // add 10 records
-        int numRecords = 10;
-        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
-        for (int i = 0; i < numRecords; i++) {
-            futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
-        }
-        LogRecord controlRecord = DLMTestUtil.getLogRecordInstance(9999L);
-        controlRecord.setControl();
-        futureList.add(writer.asyncWrite(controlRecord));
-        assertEquals("Last tx id should be " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxId());
-        assertEquals("Last DLSN should be " + DLSN.InvalidDLSN,
-                DLSN.InvalidDLSN, writer.getLastDLSN());
-        assertEquals("Position should be " + numRecords,
-                numRecords, writer.getPositionWithinLogSegment());
-
-        // close the writer to flush the output buffer
-        closeWriterAndLock(writer, lock);
-
-        List<DLSN> dlsns = Await.result(Future.collect(futureList));
-        assertEquals("All 11 records should be written",
-                numRecords + 1, dlsns.size());
-        for (int i = 0; i < numRecords; i++) {
-            DLSN dlsn = dlsns.get(i);
-            assertEquals("Incorrent ledger sequence number",
-                    0L, dlsn.getLogSegmentSequenceNo());
-            assertEquals("Incorrent entry id",
-                    0L, dlsn.getEntryId());
-            assertEquals("Inconsistent slot id",
-                    i, dlsn.getSlotId());
-        }
-        DLSN dlsn = dlsns.get(numRecords);
-        assertEquals("Incorrent ledger sequence number",
-                0L, dlsn.getLogSegmentSequenceNo());
-        assertEquals("Incorrent entry id",
-                1L, dlsn.getEntryId());
-        assertEquals("Inconsistent slot id",
-                0L, dlsn.getSlotId());
-
-        assertEquals("Last tx id should be " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxId());
-        assertEquals("Last acked tx id should be " + (numRecords - 1),
-                numRecords - 1, writer.getLastTxIdAcknowledged());
-        assertEquals("Position should be " + numRecords,
-                numRecords, writer.getPositionWithinLogSegment());
-        assertEquals("Last DLSN should be " + dlsn,
-                dlsns.get(numRecords - 1), writer.getLastDLSN());
-    }
-
-    /**
-     * Non durable write should fail if writer is closed.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testNondurableWriteAfterWriterIsClosed() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setDurableWriteEnabled(false);
-        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
-        BKLogSegmentWriter writer =
-                createLogSegmentWriter(confLocal, 0L, -1L, lock);
-
-        // close the writer
-        closeWriterAndLock(writer, lock);
-        FutureUtils.result(writer.asyncClose());
-
-        try {
-            Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(1)));
-            fail("Should fail the write if the writer is closed");
-        } catch (WriteException we) {
-            // expected
-        }
-    }
-
-    /**
-     * Non durable write should fail if writer is marked as end of stream.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testNondurableWriteAfterEndOfStream() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setDurableWriteEnabled(false);
-        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
-        BKLogSegmentWriter writer =
-                createLogSegmentWriter(confLocal, 0L, -1L, lock);
-
-        FutureUtils.result(writer.markEndOfStream());
-
-        try {
-            Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(1)));
-            fail("Should fail the write if the writer is marked as end of stream");
-        } catch (EndOfStreamException we) {
-            // expected
-        }
-
-        closeWriterAndLock(writer, lock);
-    }
-
-    /**
-     * Non durable write should fail if the log segment is fenced.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testNondurableWriteAfterLedgerIsFenced() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setDurableWriteEnabled(false);
-        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
-        BKLogSegmentWriter writer =
-                createLogSegmentWriter(confLocal, 0L, -1L, lock);
-
-        // fence the ledger
-        fenceLedger(getLedgerHandle(writer));
-
-        LogRecord record = DLMTestUtil.getLogRecordInstance(1);
-        record.setControl();
-        try {
-            Await.result(writer.asyncWrite(record));
-            fail("Should fail the writer if the log segment is already fenced");
-        } catch (BKTransmitException bkte) {
-            // expected
-            assertEquals(BKException.Code.LedgerFencedException, bkte.getBKResultCode());
-        }
-
-        try {
-            Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(2)));
-            fail("Should fail the writer if the log segment is already fenced");
-        } catch (WriteException we) {
-            // expected
-        }
-
-        abortWriterAndLock(writer, lock);
-    }
-
-    /**
-     * Non durable write should fail if writer is marked as end of stream.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testNondurableWrite() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setDurableWriteEnabled(false);
-        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
-        BKLogSegmentWriter writer =
-                createLogSegmentWriter(confLocal, 0L, -1L, lock);
-
-        assertEquals(DLSN.InvalidDLSN,
-                Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(2))));
-        assertEquals(-1L, ((BKLogSegmentEntryWriter) writer.getEntryWriter())
-                .getLedgerHandle().getLastAddPushed());
-
-        closeWriterAndLock(writer, lock);
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogWriteHandler.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogWriteHandler.java
deleted file mode 100644
index a0485bd..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKLogWriteHandler.java
+++ /dev/null
@@ -1,90 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.bk.LedgerAllocator;
-import com.twitter.distributedlog.bk.LedgerAllocatorPool;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Utils;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import java.io.IOException;
-import java.net.URI;
-
-import static org.junit.Assert.*;
-
-/**
- * Test {@link BKLogWriteHandler}
- */
-public class TestBKLogWriteHandler extends TestDistributedLogBase {
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    /**
-     * Testcase: when write handler encounters exceptions on starting log segment
-     * it should abort the transaction and return the ledger to the pool.
-     */
-    @Test(timeout = 60000)
-    public void testAbortTransactionOnStartLogSegment() throws Exception {
-        URI uri = createDLMURI("/" + runtime.getMethodName());
-        ensureURICreated(zkc, uri);
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setEnableLedgerAllocatorPool(true);
-        confLocal.setLedgerAllocatorPoolCoreSize(1);
-        confLocal.setLedgerAllocatorPoolName("test-allocator-pool");
-
-        BKDistributedLogNamespace namespace = (BKDistributedLogNamespace)
-                DistributedLogNamespaceBuilder.newBuilder()
-                        .conf(confLocal)
-                        .uri(uri)
-                        .build();
-        DistributedLogManager dlm = namespace.openLog("test-stream");
-        FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_StartLogSegmentOnAssignLogSegmentSequenceNumber,
-                FailpointUtils.FailPointActions.FailPointAction_Throw);
-        try {
-            AsyncLogWriter writer =  FutureUtils.result(dlm.openAsyncLogWriter());
-            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
-            fail("Should fail opening the writer");
-        } catch (IOException ioe) {
-            // expected
-        } finally {
-            FailpointUtils.removeFailpoint(
-                    FailpointUtils.FailPointName.FP_StartLogSegmentOnAssignLogSegmentSequenceNumber);
-        }
-
-        LedgerAllocator allocator = ((BKNamespaceDriver) namespace.getNamespaceDriver())
-                .getLedgerAllocator();
-        assertTrue(allocator instanceof LedgerAllocatorPool);
-        LedgerAllocatorPool allocatorPool = (LedgerAllocatorPool) allocator;
-        assertEquals(0, allocatorPool.obtainMapSize());
-
-        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
-        writer.write(DLMTestUtil.getLogRecordInstance(1L));
-        Utils.close(writer);
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKSyncLogReader.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKSyncLogReader.java
deleted file mode 100644
index d28b62c..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestBKSyncLogReader.java
+++ /dev/null
@@ -1,306 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static org.junit.Assert.*;
-
-/**
- * Test Sync Log Reader
- */
-public class TestBKSyncLogReader extends TestDistributedLogBase {
-
-    static final Logger logger = LoggerFactory.getLogger(TestBKSyncLogReader.class);
-
-    @Rule
-    public TestName testName = new TestName();
-
-    @Test(timeout = 60000)
-    public void testCreateReaderBeyondLastTransactionId() throws Exception {
-        String name = testName.getMethodName();
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-        for (long i = 1; i < 10; i++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(i);
-            out.write(op);
-        }
-        out.closeAndComplete();
-
-        LogReader reader = dlm.getInputStream(20L);
-        assertNull(reader.readNext(false));
-
-        // write another 20 records
-        out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-        for (long i = 10; i < 30; i++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(i);
-            out.write(op);
-        }
-        out.closeAndComplete();
-
-        for (int i = 0; i < 10; i++) {
-            LogRecord record = waitForNextRecord(reader);
-            assertEquals(20L + i, record.getTransactionId());
-        }
-        assertNull(reader.readNext(false));
-    }
-
-    @Test(timeout = 60000)
-    public void testDeletingLogWhileReading() throws Exception {
-        String name = testName.getMethodName();
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-        for (long i = 1; i < 10; i++) {
-            LogRecord op = DLMTestUtil.getLogRecordInstance(i);
-            out.write(op);
-        }
-        out.closeAndComplete();
-
-        LogReader reader = dlm.getInputStream(1L);
-        for (int i = 1; i < 10; i++) {
-            LogRecord record = waitForNextRecord(reader);
-            assertEquals((long) i, record.getTransactionId());
-        }
-
-        DistributedLogManager deleteDLM = createNewDLM(conf, name);
-        deleteDLM.delete();
-
-        LogRecord record;
-        try {
-            record = reader.readNext(false);
-            while (null == record) {
-                record = reader.readNext(false);
-            }
-            fail("Should fail reading next with LogNotFound");
-        } catch (LogNotFoundException lnfe) {
-            // expected
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testReadingFromEmptyLog() throws Exception {
-        String name = testName.getMethodName();
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(Integer.MAX_VALUE);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-        // write a record but not commit
-        LogRecord op = DLMTestUtil.getLogRecordInstance(1L);
-        out.write(op);
-
-        LogReader reader = dlm.getInputStream(1L);
-        assertNull(reader.readNext(true));
-        assertNull(reader.readNext(false));
-
-        op = DLMTestUtil.getLogRecordInstance(2L);
-        out.write(op);
-
-        // reader is able to read first record
-        LogRecord record = waitForNextRecord(reader);
-        assertNotNull(record);
-        assertEquals(1L, record.getTransactionId());
-        DLMTestUtil.verifyLogRecord(record);
-
-        assertNull(reader.readNext(true));
-
-        out.close();
-        reader.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReadRecordsAfterReadAheadCaughtUp() throws Exception {
-        String name = testName.getMethodName();
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(Integer.MAX_VALUE);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-        for (long i = 1L; i <= 10L; i++) {
-            LogRecord record = DLMTestUtil.getLogRecordInstance(i);
-            out.write(record);
-        }
-        out.setReadyToFlush();
-        out.flushAndSync();
-
-        logger.info("Write first 10 records");
-
-        // all 10 records are added to the stream
-        // then open a reader to read
-        BKSyncLogReader reader = (BKSyncLogReader) dlm.getInputStream(1L);
-
-        // wait until readahead caught up
-        while (!reader.getReadAheadReader().isReadAheadCaughtUp()) {
-            TimeUnit.MILLISECONDS.sleep(20);
-        }
-
-        logger.info("ReadAhead is caught up with first 10 records");
-
-        for (long i = 11L; i <= 20L; i++) {
-            LogRecord record = DLMTestUtil.getLogRecordInstance(i);
-            out.write(record);
-        }
-        out.setReadyToFlush();
-        out.flushAndSync();
-
-        logger.info("Write another 10 records");
-
-        // wait until readahead move on
-        while (reader.getReadAheadReader().getNextEntryPosition().getEntryId() < 21) {
-            TimeUnit.MILLISECONDS.sleep(20);
-        }
-
-        logger.info("ReadAhead is caught up with another 10 records");
-
-        // resume reading from sync reader. so it should be able to read all 20 records
-        // and return null to claim it as caughtup
-        LogRecord record = reader.readNext(false);
-        int numReads = 0;
-        long expectedTxId = 1L;
-        while (null != record) {
-            ++numReads;
-            assertEquals(expectedTxId, record.getTransactionId());
-            DLMTestUtil.verifyLogRecord(record);
-            ++expectedTxId;
-            record = reader.readNext(false);
-        }
-        assertEquals(20, numReads);
-
-        out.close();
-        reader.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReadRecordsWhenReadAheadCatchingUp() throws Exception {
-        String name = testName.getMethodName();
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(Integer.MAX_VALUE);
-        confLocal.setReadAheadMaxRecords(1);
-        confLocal.setReadAheadBatchSize(1);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-        for (long i = 1L; i <= 10L; i++) {
-            LogRecord record = DLMTestUtil.getLogRecordInstance(i);
-            out.write(record);
-        }
-        out.setReadyToFlush();
-        out.flushAndSync();
-
-        logger.info("Write first 10 records");
-
-        // open a reader to read
-        BKSyncLogReader reader = (BKSyncLogReader) dlm.getInputStream(1L);
-        // resume reading from sync reader. so it should be able to read all 10 records
-        // and return null to claim it as caughtup
-        LogRecord record = reader.readNext(false);
-        int numReads = 0;
-        long expectedTxId = 1L;
-        while (null != record) {
-            ++numReads;
-            assertEquals(expectedTxId, record.getTransactionId());
-            DLMTestUtil.verifyLogRecord(record);
-            ++expectedTxId;
-            record = reader.readNext(false);
-        }
-        assertEquals(10, numReads);
-
-        out.close();
-        reader.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReadRecordsWhenReadAheadCatchingUp2() throws Exception {
-        String name = testName.getMethodName();
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(Integer.MAX_VALUE);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        final BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-        for (long i = 1L; i <= 10L; i++) {
-            LogRecord record = DLMTestUtil.getLogRecordInstance(i);
-            out.write(record);
-        }
-        out.setReadyToFlush();
-        out.flushAndSync();
-        final AtomicLong nextTxId = new AtomicLong(11L);
-
-        logger.info("Write first 10 records");
-
-        ScheduledExecutorService executorService =
-                Executors.newSingleThreadScheduledExecutor();
-        executorService.scheduleAtFixedRate(new Runnable() {
-            @Override
-            public void run() {
-                long txid = nextTxId.getAndIncrement();
-                LogRecord record = DLMTestUtil.getLogRecordInstance(txid);
-                try {
-                    out.write(record);
-                } catch (IOException e) {
-                    // ignore the ioe
-                }
-            }
-        }, 0, 400, TimeUnit.MILLISECONDS);
-
-        // open a reader to read
-        BKSyncLogReader reader = (BKSyncLogReader) dlm.getInputStream(1L);
-        // resume reading from sync reader. so it should be able to read all 10 records
-        // and return null to claim it as caughtup
-        LogRecord record = reader.readNext(false);
-        int numReads = 0;
-        long expectedTxId = 1L;
-        while (null != record) {
-            ++numReads;
-            assertEquals(expectedTxId, record.getTransactionId());
-            DLMTestUtil.verifyLogRecord(record);
-            ++expectedTxId;
-            record = reader.readNext(false);
-        }
-        assertTrue(numReads >= 10);
-
-        executorService.shutdown();
-        out.close();
-        reader.close();
-        dlm.close();
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDLMTestUtil.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDLMTestUtil.java
deleted file mode 100644
index c92aa6e..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDLMTestUtil.java
+++ /dev/null
@@ -1,62 +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 com.twitter.distributedlog;
-
-import java.io.File;
-
-import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TestDLMTestUtil {
-    static final Logger LOG = LoggerFactory.getLogger(TestDLMTestUtil.class);
-
-    @Rule
-    public TestName testNames = new TestName();
-
-    @Test(timeout = 60000)
-    public void testRunZookeeperOnAnyPort() throws Exception {
-        Pair<ZooKeeperServerShim, Integer> serverAndPort1 = null;
-        Pair<ZooKeeperServerShim, Integer> serverAndPort2 = null;
-        Pair<ZooKeeperServerShim, Integer> serverAndPort3 = null;
-        try {
-            File zkTmpDir1 = IOUtils.createTempDir("zookeeper1", "distrlog");
-            serverAndPort1 = LocalDLMEmulator.runZookeeperOnAnyPort(7000, zkTmpDir1);
-            File zkTmpDir2 = IOUtils.createTempDir("zookeeper2", "distrlog");
-            serverAndPort2 = LocalDLMEmulator.runZookeeperOnAnyPort(7000, zkTmpDir2);
-            File zkTmpDir3 = IOUtils.createTempDir("zookeeper3", "distrlog");
-            serverAndPort3 = LocalDLMEmulator.runZookeeperOnAnyPort(7000, zkTmpDir3);
-        } catch (Exception ex) {
-            if (null != serverAndPort1) {
-                serverAndPort1.getLeft().stop();
-            }
-            if (null != serverAndPort2) {
-                serverAndPort2.getLeft().stop();
-            }
-            if (null != serverAndPort3) {
-                serverAndPort3.getLeft().stop();
-            }
-        }
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogBase.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogBase.java
deleted file mode 100644
index d850db4..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogBase.java
+++ /dev/null
@@ -1,244 +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 com.twitter.distributedlog;
-
-import static org.junit.Assert.assertTrue;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Ticker;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.injector.AsyncRandomFailureInjector;
-import com.twitter.distributedlog.io.AsyncCloseable;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryWriter;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataCache;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.namespace.NamespaceDriver;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.PermitLimiter;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.feature.SettableFeatureProvider;
-import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.ZooKeeper;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-public class TestDistributedLogBase {
-    static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogBase.class);
-
-    // Num worker threads should be one, since the exec service is used for the ordered
-    // future pool in test cases, and setting to > 1 will therefore result in unordered
-    // write ops.
-    protected static DistributedLogConfiguration conf =
-        new DistributedLogConfiguration()
-                .setEnableReadAhead(true)
-                .setReadAheadMaxRecords(1000)
-                .setReadAheadBatchSize(10)
-                .setLockTimeout(1)
-                .setNumWorkerThreads(1)
-                .setReadAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis(20)
-                .setSchedulerShutdownTimeoutMs(0)
-                .setDLLedgerMetadataLayoutVersion(LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
-    protected ZooKeeper zkc;
-    protected static LocalDLMEmulator bkutil;
-    protected static ZooKeeperServerShim zks;
-    protected static String zkServers;
-    protected static int zkPort;
-    protected static int numBookies = 3;
-    protected static final List<File> tmpDirs = new ArrayList<File>();
-
-    @BeforeClass
-    public static void setupCluster() throws Exception {
-        File zkTmpDir = IOUtils.createTempDir("zookeeper", "distrlog");
-        tmpDirs.add(zkTmpDir);
-        Pair<ZooKeeperServerShim, Integer> serverAndPort = LocalDLMEmulator.runZookeeperOnAnyPort(zkTmpDir);
-        zks = serverAndPort.getLeft();
-        zkPort = serverAndPort.getRight();
-        bkutil = LocalDLMEmulator.newBuilder()
-                .numBookies(numBookies)
-                .zkHost("127.0.0.1")
-                .zkPort(zkPort)
-                .serverConf(DLMTestUtil.loadTestBkConf())
-                .shouldStartZK(false)
-                .build();
-        bkutil.start();
-        zkServers = "127.0.0.1:" + zkPort;
-        Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
-            @Override
-            public void uncaughtException(Thread t, Throwable e) {
-                LOG.warn("Uncaught exception at Thread {} : ", t.getName(), e);
-            }
-        });
-    }
-
-    @AfterClass
-    public static void teardownCluster() throws Exception {
-        bkutil.teardown();
-        zks.stop();
-        for (File dir : tmpDirs) {
-            FileUtils.deleteDirectory(dir);
-        }
-    }
-
-    @Before
-    public void setup() throws Exception {
-        try {
-            zkc = LocalDLMEmulator.connectZooKeeper("127.0.0.1", zkPort);
-        } catch (Exception ex) {
-            LOG.error("hit exception connecting to zookeeper at {}:{}", new Object[] { "127.0.0.1", zkPort, ex });
-            throw ex;
-        }
-    }
-
-    @After
-    public void teardown() throws Exception {
-        if (null != zkc) {
-            zkc.close();
-        }
-    }
-
-    protected LogRecord waitForNextRecord(LogReader reader) throws Exception {
-        LogRecord record = reader.readNext(false);
-        while (null == record) {
-            record = reader.readNext(false);
-        }
-        return record;
-    }
-
-    public URI createDLMURI(String path) throws Exception {
-        return DLMTestUtil.createDLMURI(zkPort, path);
-    }
-
-    protected void ensureURICreated(URI uri) throws Exception {
-        ensureURICreated(zkc, uri);
-    }
-
-    protected void ensureURICreated(ZooKeeper zkc, URI uri) throws Exception {
-        try {
-            zkc.create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        } catch (KeeperException.NodeExistsException nee) {
-            // ignore
-        }
-    }
-
-    public BKDistributedLogManager createNewDLM(DistributedLogConfiguration conf,
-                                                String name) throws Exception {
-        return createNewDLM(conf, name, PermitLimiter.NULL_PERMIT_LIMITER);
-    }
-
-    public BKDistributedLogManager createNewDLM(DistributedLogConfiguration conf,
-                                                String name,
-                                                PermitLimiter writeLimiter)
-            throws Exception {
-        URI uri = createDLMURI("/" + name);
-        ensureURICreated(uri);
-        final DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .uri(uri)
-                .conf(conf)
-                .build();
-        final OrderedScheduler scheduler = OrderedScheduler.newBuilder()
-                .corePoolSize(1)
-                .name("test-scheduler")
-                .build();
-        AsyncCloseable resourcesCloseable = new AsyncCloseable() {
-            @Override
-            public Future<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();
-            }
-        };
-        AsyncFailureInjector failureInjector = AsyncRandomFailureInjector.newBuilder()
-                .injectDelays(conf.getEIInjectReadAheadDelay(),
-                        conf.getEIInjectReadAheadDelayPercent(),
-                        conf.getEIInjectMaxReadAheadDelayMs())
-                .injectErrors(false, 10)
-                .injectStops(conf.getEIInjectReadAheadStall(), 10)
-                .injectCorruption(conf.getEIInjectReadAheadBrokenEntries())
-                .build();
-        return new BKDistributedLogManager(
-                name,
-                conf,
-                ConfUtils.getConstDynConf(conf),
-                uri,
-                namespace.getNamespaceDriver(),
-                new LogSegmentMetadataCache(conf, Ticker.systemTicker()),
-                scheduler,
-                DistributedLogConstants.UNKNOWN_CLIENT_ID,
-                DistributedLogConstants.LOCAL_REGION_ID,
-                writeLimiter,
-                new SettableFeatureProvider("", 0),
-                failureInjector,
-                NullStatsLogger.INSTANCE,
-                NullStatsLogger.INSTANCE,
-                Optional.of(resourcesCloseable));
-    }
-
-    protected LogSegmentMetadataStore getLogSegmentMetadataStore(DistributedLogNamespace namespace)
-            throws IOException {
-        return namespace.getNamespaceDriver().getLogStreamMetadataStore(NamespaceDriver.Role.READER)
-                .getLogSegmentMetadataStore();
-    }
-
-    protected ZooKeeperClient getZooKeeperClient(DistributedLogNamespace namespace) throws Exception {
-        NamespaceDriver driver = namespace.getNamespaceDriver();
-        assertTrue(driver instanceof BKNamespaceDriver);
-        return ((BKNamespaceDriver) driver).getWriterZKC();
-    }
-
-    @SuppressWarnings("deprecation")
-    protected BookKeeperClient getBookKeeperClient(DistributedLogNamespace namespace) throws Exception {
-        NamespaceDriver driver = namespace.getNamespaceDriver();
-        assertTrue(driver instanceof BKNamespaceDriver);
-        return ((BKNamespaceDriver) driver).getReaderBKC();
-    }
-
-    protected LedgerHandle getLedgerHandle(BKLogSegmentWriter segmentWriter) {
-        LogSegmentEntryWriter entryWriter = segmentWriter.getEntryWriter();
-        assertTrue(entryWriter instanceof BKLogSegmentEntryWriter);
-        return ((BKLogSegmentEntryWriter) entryWriter).getLedgerHandle();
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogConfiguration.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogConfiguration.java
deleted file mode 100644
index 19b9863..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestDistributedLogConfiguration.java
+++ /dev/null
@@ -1,132 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Optional;
-
-import com.twitter.distributedlog.net.DNSResolverForRacks;
-import com.twitter.distributedlog.net.DNSResolverForRows;
-import org.apache.bookkeeper.net.DNSToSwitchMapping;
-import org.apache.commons.configuration.StrictConfigurationComparator;
-import org.junit.Test;
-
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-public class TestDistributedLogConfiguration {
-
-    static final class TestDNSResolver implements DNSToSwitchMapping {
-
-        public TestDNSResolver() {}
-
-        @Override
-        public List<String> resolve(List<String> list) {
-            return list;
-        }
-
-        @Override
-        public void reloadCachedMappings() {
-            // no-op
-        }
-    }
-
-    @Test(timeout = 20000)
-    public void loadStreamConfGoodOverrideAccepted() throws Exception {
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        assertEquals(conf.getPeriodicFlushFrequencyMilliSeconds(),
-            DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT);
-        assertEquals(conf.getReaderIdleErrorThresholdMillis(),
-            DistributedLogConfiguration.BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT);
-        DistributedLogConfiguration override = new DistributedLogConfiguration();
-        override.setPeriodicFlushFrequencyMilliSeconds(
-            DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT+1);
-        override.setReaderIdleErrorThresholdMillis(
-            DistributedLogConfiguration.BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT - 1);
-        conf.loadStreamConf(Optional.of(override));
-        assertEquals(conf.getPeriodicFlushFrequencyMilliSeconds(),
-            DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT+1);
-        assertEquals(conf.getReaderIdleErrorThresholdMillis(),
-            DistributedLogConfiguration.BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT - 1);
-    }
-
-    @SuppressWarnings("deprecation")
-    @Test(timeout = 20000)
-    public void loadStreamConfBadOverrideIgnored() throws Exception {
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        assertEquals(conf.getBKClientWriteTimeout(),
-            DistributedLogConfiguration.BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT);
-        DistributedLogConfiguration override = new DistributedLogConfiguration();
-        override.setBKClientWriteTimeout(
-            DistributedLogConfiguration.BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT+1);
-        conf.loadStreamConf(Optional.of(override));
-        assertEquals(conf.getBKClientWriteTimeout(),
-            DistributedLogConfiguration.BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT);
-    }
-
-    @Test(timeout = 20000)
-    public void loadStreamConfNullOverrides() throws Exception {
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        DistributedLogConfiguration confClone = (DistributedLogConfiguration)conf.clone();
-        Optional<DistributedLogConfiguration> streamConfiguration = Optional.absent();
-        conf.loadStreamConf(streamConfiguration);
-
-        StrictConfigurationComparator comp = new StrictConfigurationComparator();
-        assertTrue(comp.compare(conf, confClone));
-    }
-
-    @Test(timeout = 200000)
-    public void getEnsemblePlacementResolverClass() throws Exception {
-        DistributedLogConfiguration conf1 = new DistributedLogConfiguration();
-        assertEquals(DNSResolverForRacks.class, conf1.getEnsemblePlacementDnsResolverClass());
-        DistributedLogConfiguration conf2 = new DistributedLogConfiguration()
-                .setRowAwareEnsemblePlacementEnabled(true);
-        assertEquals(DNSResolverForRows.class, conf2.getEnsemblePlacementDnsResolverClass());
-        DistributedLogConfiguration conf3 = new DistributedLogConfiguration()
-                .setRowAwareEnsemblePlacementEnabled(true)
-                .setEnsemblePlacementDnsResolverClass(TestDNSResolver.class);
-        assertEquals(TestDNSResolver.class, conf3.getEnsemblePlacementDnsResolverClass());
-    }
-
-    @Test(timeout = 200000)
-    public void validateConfiguration(){
-        boolean exceptionThrown=false;
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        // validate default configuration
-        conf.validate();
-        // test equal, should not throw exception
-        conf.setReadLACLongPollTimeout(conf.getBKClientReadTimeout() * 1000);
-        try {
-            conf.validate();
-        } catch (IllegalArgumentException e){
-            exceptionThrown=true;
-        }
-        assertFalse(exceptionThrown);
-        // test invalid case, should throw exception
-        exceptionThrown=false;
-        conf.setReadLACLongPollTimeout(conf.getBKClientReadTimeout() * 1000 * 2);
-        try {
-            conf.validate();
-        } catch (IllegalArgumentException e){
-            exceptionThrown=true;
-        }
-        assertTrue(exceptionThrown);
-    }
-
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestEntry.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestEntry.java
deleted file mode 100644
index 0e4737b..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestEntry.java
+++ /dev/null
@@ -1,345 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.Entry.Reader;
-import com.twitter.distributedlog.Entry.Writer;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.io.Buffer;
-import com.twitter.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.junit.Assert;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-import static org.junit.Assert.*;
-
-/**
- * Test Case of {@link Entry}
- */
-public class TestEntry {
-
-    @Test(timeout = 20000)
-    public void testEmptyRecordSet() throws Exception {
-        Writer writer = Entry.newEntry(
-                "test-empty-record-set",
-                1024,
-                true,
-                CompressionCodec.Type.NONE,
-                NullStatsLogger.INSTANCE);
-        assertEquals("zero bytes", 0, writer.getNumBytes());
-        assertEquals("zero records", 0, writer.getNumRecords());
-
-        Buffer buffer = writer.getBuffer();
-        Entry recordSet = Entry.newBuilder()
-                .setData(buffer.getData(), 0, buffer.size())
-                .setLogSegmentInfo(1L, 0L)
-                .setEntryId(0L)
-                .build();
-        Reader reader = recordSet.reader();
-        Assert.assertNull("Empty record set should return null",
-                reader.nextRecord());
-    }
-
-    @Test(timeout = 20000)
-    public void testWriteTooLongRecord() throws Exception {
-        Writer writer = Entry.newEntry(
-                "test-write-too-long-record",
-                1024,
-                false,
-                CompressionCodec.Type.NONE,
-                NullStatsLogger.INSTANCE);
-        assertEquals("zero bytes", 0, writer.getNumBytes());
-        assertEquals("zero records", 0, writer.getNumRecords());
-
-        LogRecord largeRecord = new LogRecord(1L, new byte[MAX_LOGRECORD_SIZE + 1]);
-        try {
-            writer.writeRecord(largeRecord, new Promise<DLSN>());
-            Assert.fail("Should fail on writing large record");
-        } catch (LogRecordTooLongException lrtle) {
-            // expected
-        }
-        assertEquals("zero bytes", 0, writer.getNumBytes());
-        assertEquals("zero records", 0, writer.getNumRecords());
-
-        Buffer buffer = writer.getBuffer();
-        Assert.assertEquals("zero bytes", 0, buffer.size());
-    }
-
-    @Test(timeout = 20000)
-    public void testWriteRecords() throws Exception {
-        Writer writer = Entry.newEntry(
-                "test-write-records",
-                1024,
-                true,
-                CompressionCodec.Type.NONE,
-                NullStatsLogger.INSTANCE);
-        assertEquals("zero bytes", 0, writer.getNumBytes());
-        assertEquals("zero records", 0, writer.getNumRecords());
-
-        List<Future<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>();
-            writer.writeRecord(record, writePromise);
-            writePromiseList.add(writePromise);
-            assertEquals((i + 1) + " records", (i + 1), writer.getNumRecords());
-        }
-
-        // write large record
-        LogRecord largeRecord = new LogRecord(1L, new byte[MAX_LOGRECORD_SIZE + 1]);
-        try {
-            writer.writeRecord(largeRecord, new Promise<DLSN>());
-            Assert.fail("Should fail on writing large record");
-        } catch (LogRecordTooLongException lrtle) {
-            // expected
-        }
-        assertEquals("5 records", 5, writer.getNumRecords());
-
-        // write another 5 records
-        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>();
-            writer.writeRecord(record, writePromise);
-            writePromiseList.add(writePromise);
-            assertEquals((i + 6) + " records", (i + 6), writer.getNumRecords());
-        }
-
-        Buffer buffer = writer.getBuffer();
-
-        // Test transmit complete
-        writer.completeTransmit(1L, 1L);
-        List<DLSN> writeResults = Await.result(Future.collect(writePromiseList));
-        for (int i = 0; i < 10; i++) {
-            Assert.assertEquals(new DLSN(1L, 1L, i), writeResults.get(i));
-        }
-
-        // Test reading from buffer
-        Entry recordSet = Entry.newBuilder()
-                .setData(buffer.getData(), 0, buffer.size())
-                .setLogSegmentInfo(1L, 1L)
-                .setEntryId(0L)
-                .build();
-        Reader reader = recordSet.reader();
-        LogRecordWithDLSN record = reader.nextRecord();
-        int numReads = 0;
-        long expectedTxid = 0L;
-        while (null != record) {
-            Assert.assertEquals(expectedTxid, record.getTransactionId());
-            Assert.assertEquals(expectedTxid, record.getSequenceId());
-            Assert.assertEquals(new DLSN(1L, 0L, expectedTxid), record.getDlsn());
-            ++numReads;
-            ++expectedTxid;
-            record = reader.nextRecord();
-        }
-        Assert.assertEquals(10, numReads);
-    }
-
-    @Test(timeout = 20000)
-    public void testWriteRecordSet() throws Exception {
-        Writer writer = Entry.newEntry(
-                "test-write-recordset",
-                1024,
-                true,
-                CompressionCodec.Type.NONE,
-                NullStatsLogger.INSTANCE);
-        assertEquals("zero bytes", 0, writer.getNumBytes());
-        assertEquals("zero records", 0, writer.getNumRecords());
-
-        List<Future<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>();
-            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();
-        // 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>();
-            recordSetWriter.writeRecord(record, writePromise);
-            recordSetPromiseList.add(writePromise);
-            assertEquals((i + 1) + " records", (i + 1), recordSetWriter.getNumRecords());
-        }
-        final ByteBuffer recordSetBuffer = recordSetWriter.getBuffer();
-        byte[] data = new byte[recordSetBuffer.remaining()];
-        recordSetBuffer.get(data);
-        LogRecord setRecord = new LogRecord(5L, data);
-        setRecord.setPositionWithinLogSegment(5);
-        setRecord.setRecordSet();
-        Promise<DLSN> writePromise = new Promise<DLSN>();
-        writePromise.addEventListener(new FutureEventListener<DLSN>() {
-            @Override
-            public void onSuccess(DLSN dlsn) {
-                recordSetWriter.completeTransmit(
-                        dlsn.getLogSegmentSequenceNo(),
-                        dlsn.getEntryId(),
-                        dlsn.getSlotId());
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                recordSetWriter.abortTransmit(cause);
-            }
-        });
-        writer.writeRecord(setRecord, writePromise);
-        writePromiseList.add(writePromise);
-
-        // write last 5 records
-        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>();
-            writer.writeRecord(record, writePromise);
-            writePromiseList.add(writePromise);
-            assertEquals((i + 11) + " records", (i + 11), writer.getNumRecords());
-        }
-
-        Buffer buffer = writer.getBuffer();
-
-        // Test transmit complete
-        writer.completeTransmit(1L, 1L);
-        List<DLSN> writeResults = Await.result(Future.collect(writePromiseList));
-        for (int i = 0; i < 5; i++) {
-            Assert.assertEquals(new DLSN(1L, 1L, i), writeResults.get(i));
-        }
-        Assert.assertEquals(new DLSN(1L, 1L, 5), writeResults.get(5));
-        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));
-        for (int i = 0; i < 5; i++) {
-            Assert.assertEquals(new DLSN(1L, 1L, (5 + i)), recordSetWriteResults.get(i));
-        }
-
-        // Test reading from buffer
-        verifyReadResult(buffer, 1L, 1L, 1L, true,
-                new DLSN(1L, 1L, 2L), 3, 5, 5,
-                new DLSN(1L, 1L, 2L), 2L);
-        verifyReadResult(buffer, 1L, 1L, 1L, true,
-                new DLSN(1L, 1L, 7L), 0, 3, 5,
-                new DLSN(1L, 1L, 7L), 7L);
-        verifyReadResult(buffer, 1L, 1L, 1L, true,
-                new DLSN(1L, 1L, 12L), 0, 0, 3,
-                new DLSN(1L, 1L, 12L), 12L);
-        verifyReadResult(buffer, 1L, 1L, 1L, false,
-                new DLSN(1L, 1L, 2L), 3, 5, 5,
-                new DLSN(1L, 1L, 2L), 2L);
-        verifyReadResult(buffer, 1L, 1L, 1L, false,
-                new DLSN(1L, 1L, 7L), 0, 3, 5,
-                new DLSN(1L, 1L, 7L), 7L);
-        verifyReadResult(buffer, 1L, 1L, 1L, false,
-                new DLSN(1L, 1L, 12L), 0, 0, 3,
-                new DLSN(1L, 1L, 12L), 12L);
-    }
-
-    void verifyReadResult(Buffer data,
-                          long lssn, long entryId, long startSequenceId,
-                          boolean deserializeRecordSet,
-                          DLSN skipTo,
-                          int firstNumRecords,
-                          int secondNumRecords,
-                          int lastNumRecords,
-                          DLSN expectedDLSN,
-                          long expectedTxId) throws Exception {
-        Entry recordSet = Entry.newBuilder()
-                .setData(data.getData(), 0, data.size())
-                .setLogSegmentInfo(lssn, startSequenceId)
-                .setEntryId(entryId)
-                .deserializeRecordSet(deserializeRecordSet)
-                .skipTo(skipTo)
-                .build();
-        Reader reader = recordSet.reader();
-
-        LogRecordWithDLSN record;
-        for (int i = 0; i < firstNumRecords; i++) { // first
-            record = reader.nextRecord();
-            assertNotNull(record);
-            assertEquals(expectedDLSN, record.getDlsn());
-            assertEquals(expectedTxId, record.getTransactionId());
-            assertNotNull("record " + record + " payload is null",
-                    record.getPayload());
-            assertEquals("record-" + expectedTxId, new String(record.getPayload(), UTF_8));
-            expectedDLSN = expectedDLSN.getNextDLSN();
-            ++expectedTxId;
-        }
-
-        boolean verifyDeserializedRecords = true;
-        if (firstNumRecords > 0) {
-            verifyDeserializedRecords = deserializeRecordSet;
-        }
-        if (verifyDeserializedRecords) {
-            long txIdOfRecordSet = 5;
-            for (int i = 0; i < secondNumRecords; i++) {
-                record = reader.nextRecord();
-                assertNotNull(record);
-                assertEquals(expectedDLSN, record.getDlsn());
-                assertEquals(txIdOfRecordSet, record.getTransactionId());
-                assertNotNull("record " + record + " payload is null",
-                        record.getPayload());
-                assertEquals("record-" + expectedTxId, new String(record.getPayload(), UTF_8));
-                expectedDLSN = expectedDLSN.getNextDLSN();
-                ++expectedTxId;
-            }
-        } else {
-            record = reader.nextRecord();
-            assertNotNull(record);
-            assertEquals(expectedDLSN, record.getDlsn());
-            assertEquals(expectedTxId, record.getTransactionId());
-            for (int i = 0; i < secondNumRecords; i++) {
-                expectedDLSN = expectedDLSN.getNextDLSN();
-                ++expectedTxId;
-            }
-        }
-
-        for (int i = 0; i < lastNumRecords; i++) {
-            record = reader.nextRecord();
-            assertNotNull(record);
-            assertEquals(expectedDLSN, record.getDlsn());
-            assertEquals(expectedTxId, record.getTransactionId());
-            assertNotNull("record " + record + " payload is null",
-                    record.getPayload());
-            assertEquals("record-" + expectedTxId, new String(record.getPayload(), UTF_8));
-            expectedDLSN = expectedDLSN.getNextDLSN();
-            ++expectedTxId;
-        }
-
-    }
-
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestEntryPosition.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestEntryPosition.java
deleted file mode 100644
index 384d1e8..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestEntryPosition.java
+++ /dev/null
@@ -1,59 +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 com.twitter.distributedlog;
-
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-/**
- * Test Case for {@link EntryPosition}
- */
-public class TestEntryPosition {
-
-    private void checkPosition(EntryPosition position,
-                               long lssn,
-                               long entryId) {
-        assertEquals(position.getLogSegmentSequenceNumber(), lssn);
-        assertEquals(position.getEntryId(), entryId);
-    }
-
-    @Test
-    public void testAdvance() {
-        EntryPosition position = new EntryPosition(9L, 99L);
-
-        checkPosition(position, 9L, 99L);
-
-        // advance (8L, 100L) takes no effect
-        assertFalse(position.advance(8L, 100L));
-        checkPosition(position, 9L, 99L);
-        // advance (9L, 98L) takes no effect
-        assertFalse(position.advance(9L, 98L));
-        checkPosition(position, 9L, 99L);
-        // advance (9L, 99L) takes no effect
-        assertFalse(position.advance(9L, 99L));
-        checkPosition(position, 9L, 99L);
-        // advance (9L, 100L) takes effects
-        assertTrue(position.advance(9L, 100L));
-        checkPosition(position, 9L, 100L);
-        // advance (10L, 0L) takes effects
-        assertTrue(position.advance(10L, 0L));
-        checkPosition(position, 10L, 0L);
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestEnvelopedEntry.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestEnvelopedEntry.java
deleted file mode 100644
index 37f261d..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestEnvelopedEntry.java
+++ /dev/null
@@ -1,81 +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 com.twitter.distributedlog;
-
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-
-import com.twitter.distributedlog.io.Buffer;
-import com.twitter.distributedlog.io.CompressionCodec;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.junit.Assert;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-public class TestEnvelopedEntry {
-
-    static final Logger LOG = LoggerFactory.getLogger(TestEnvelopedEntry.class);
-
-    private String getString(boolean compressible) {
-        if (compressible) {
-            StringBuilder builder = new StringBuilder();
-            for(int i = 0; i < 1000; i++) {
-                builder.append('A');
-            }
-            return builder.toString();
-        }
-        return "DistributedLogEnvelopedEntry";
-    }
-
-    @Test(timeout = 20000)
-    public void testEnvelope() throws Exception {
-        byte[] data = getString(false).getBytes();
-        EnvelopedEntry writeEntry = new EnvelopedEntry(EnvelopedEntry.CURRENT_VERSION,
-                                                  CompressionCodec.Type.NONE,
-                                                  data,
-                                                  data.length,
-                                                  new NullStatsLogger());
-        Buffer outBuf = new Buffer(2 * data.length);
-        writeEntry.writeFully(new DataOutputStream(outBuf));
-        EnvelopedEntry readEntry = new EnvelopedEntry(EnvelopedEntry.CURRENT_VERSION,
-                                                      new NullStatsLogger());
-        readEntry.readFully(new DataInputStream(new ByteArrayInputStream(outBuf.getData())));
-        byte[] newData = readEntry.getDecompressedPayload();
-        Assert.assertEquals("Written data should equal read data", new String(data), new String(newData));
-    }
-
-    @Test(timeout = 20000)
-    public void testLZ4Compression() throws Exception {
-        byte[] data = getString(true).getBytes();
-        EnvelopedEntry writeEntry = new EnvelopedEntry(EnvelopedEntry.CURRENT_VERSION,
-                                                       CompressionCodec.Type.LZ4,
-                                                       data,
-                                                       data.length,
-                                                       new NullStatsLogger());
-        Buffer outBuf = new Buffer(data.length);
-        writeEntry.writeFully(new DataOutputStream(outBuf));
-        Assert.assertTrue(data.length > outBuf.size());
-        EnvelopedEntry readEntry = new EnvelopedEntry(EnvelopedEntry.CURRENT_VERSION,
-                                                      new NullStatsLogger());
-        readEntry.readFully(new DataInputStream(new ByteArrayInputStream(outBuf.getData())));
-        byte[] newData = readEntry.getDecompressedPayload();
-        Assert.assertEquals("Written data should equal read data", new String(data), new String(newData));
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestInterleavedReaders.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestInterleavedReaders.java
deleted file mode 100644
index 830e059..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestInterleavedReaders.java
+++ /dev/null
@@ -1,338 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.util.FutureUtils;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-public class TestInterleavedReaders extends TestDistributedLogBase {
-    static final Logger LOG = LoggerFactory.getLogger(TestInterleavedReaders.class);
-
-    static {
-        conf.setOutputBufferSize(0);
-        conf.setImmediateFlushEnabled(true);
-    }
-
-    private int drainStreams(LogReader reader0, int num0, LogReader reader1, int num1)
-            throws Exception {
-        // Allow time for watches to fire
-        Thread.sleep(15);
-        int numTrans = 0;
-        LogRecord record;
-        int i = 0;
-        while (i < num0) {
-            record = reader0.readNext(false);
-            if (null != record) {
-                assertTrue((record.getTransactionId() % 2 == 0));
-                DLMTestUtil.verifyLogRecord(record);
-                numTrans++;
-                i++;
-                LOG.info("Read record {}", record);
-            }
-        }
-        i = 0;
-        while (i < num1) {
-            record = reader1.readNext(false);
-            if (null != record) {
-                assertTrue((record.getTransactionId() % 2 == 1));
-                DLMTestUtil.verifyLogRecord(record);
-                numTrans++;
-                i++;
-                LOG.info("Read record {}", record);
-            }
-        }
-        return numTrans;
-    }
-
-    @Test(timeout = 60000)
-    public void testInterleavedReaders() throws Exception {
-        String name = "distrlog-interleaved";
-        BKDistributedLogManager dlmwrite0 = createNewDLM(conf, name + "-0");
-        BKDistributedLogManager dlmreader0 = createNewDLM(conf, name + "-0");
-        BKDistributedLogManager dlmwrite1 = createNewDLM(conf, name + "-1");
-        BKDistributedLogManager dlmreader1 = createNewDLM(conf, name + "-1");
-
-        LogReader reader0 = null;
-        LogReader reader1 = null;
-        long txid = 1;
-        int numTrans = 0;
-
-        BKAsyncLogWriter writer0 = dlmwrite0.startAsyncLogSegmentNonPartitioned();
-        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++)));
-            }
-            FutureUtils.result(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
-            FutureUtils.result(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
-            if (null == reader0) {
-                reader0 = dlmreader0.getInputStream(1);
-            }
-            if (null == reader1) {
-                reader1 = dlmreader1.getInputStream(1);
-            }
-            numTrans += drainStreams(reader0, 10, reader1, 10);
-            assertEquals((txid - 1), numTrans);
-        }
-        reader0.close();
-        reader1.close();
-        dlmreader0.close();
-        dlmwrite0.close();
-        dlmreader1.close();
-        dlmwrite1.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testInterleavedReadersWithRollingEdge() throws Exception {
-        String name = "distrlog-interleaved-rolling-edge";
-        BKDistributedLogManager dlmwrite0 = createNewDLM(conf, name + "-0");
-        BKDistributedLogManager dlmreader0 = createNewDLM(conf, name + "-0");
-        BKDistributedLogManager dlmwrite1 = createNewDLM(conf, name + "-1");
-        BKDistributedLogManager dlmreader1 = createNewDLM(conf, name + "-1");
-
-        LogReader reader0 = null;
-        LogReader reader1 = null;
-        long txid = 1;
-        int numTrans = 0;
-
-        BKAsyncLogWriter writer0 = dlmwrite0.startAsyncLogSegmentNonPartitioned();
-        BKAsyncLogWriter writer1 = dlmwrite1.startAsyncLogSegmentNonPartitioned();
-        for (long j = 1; j <= 4; j++) {
-            if (j > 1) {
-                writer0.setForceRolling(true);
-                writer1.setForceRolling(true);
-            }
-            for (int k = 1; k <= 2; k++) {
-                FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
-                FutureUtils.result(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)));
-            LOG.info("Completed {} write", j);
-            if (null == reader0) {
-                reader0 = dlmreader0.getInputStream(1);
-            }
-            if (null == reader1) {
-                reader1 = dlmreader1.getInputStream(1);
-            }
-            numTrans += drainStreams(reader0, 2, reader1, 2);
-            assertEquals((txid - 1), numTrans);
-        }
-        reader0.close();
-        reader1.close();
-        dlmreader0.close();
-        dlmwrite0.close();
-        dlmreader1.close();
-        dlmwrite1.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testInterleavedReadersWithRolling() throws Exception {
-        String name = "distrlog-interleaved-rolling";
-        BKDistributedLogManager dlmwrite0 = createNewDLM(conf, name + "-0");
-        BKDistributedLogManager dlmreader0 = createNewDLM(conf, name + "-0");
-        BKDistributedLogManager dlmwrite1 = createNewDLM(conf, name + "-1");
-        BKDistributedLogManager dlmreader1 = createNewDLM(conf, name + "-1");
-
-        LogReader reader0 = null;
-        LogReader reader1 = null;
-        long txid = 1;
-        int numTrans = 0;
-
-        BKAsyncLogWriter writer0 = dlmwrite0.startAsyncLogSegmentNonPartitioned();
-        BKAsyncLogWriter writer1 = dlmwrite1.startAsyncLogSegmentNonPartitioned();
-        for (long j = 1; j <= 2; j++) {
-            for (int k = 1; k <= 6; k++) {
-                if (k == 3) {
-                    writer0.setForceRolling(true);
-                    writer1.setForceRolling(true);
-                }
-                FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
-                FutureUtils.result(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)));
-            if (null == reader0) {
-                reader0 = dlmreader0.getInputStream(1);
-            }
-            if (null == reader1) {
-                reader1 = dlmreader1.getInputStream(1);
-            }
-            numTrans += drainStreams(reader0, 6, reader1, 6);
-            assertEquals((txid - 1), numTrans);
-        }
-        reader0.close();
-        reader1.close();
-        dlmreader0.close();
-        dlmwrite0.close();
-        dlmreader1.close();
-        dlmwrite1.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testInterleavedReadersWithCleanup() throws Exception {
-        String name = "distrlog-interleaved-cleanup";
-        BKDistributedLogManager dlmwrite0 = createNewDLM(conf, name + "-0");
-        BKDistributedLogManager dlmwrite1 = createNewDLM(conf, name + "-1");
-        long txid = 1;
-        Long retentionPeriodOverride = null;
-
-        BKAsyncLogWriter writer0 = dlmwrite0.startAsyncLogSegmentNonPartitioned();
-        BKAsyncLogWriter writer1 = dlmwrite1.startAsyncLogSegmentNonPartitioned();
-        for (long j = 1; j <= 4; j++) {
-            for (int k = 1; k <= 10; k++) {
-                if (k == 5) {
-                    writer0.setForceRolling(true);
-                    writer0.overRideMinTimeStampToKeep(retentionPeriodOverride);
-                    writer1.setForceRolling(true);
-                    writer1.overRideMinTimeStampToKeep(retentionPeriodOverride);
-                }
-                DLSN dlsn1 = FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
-                LOG.info("writer1 write record {}", dlsn1);
-                DLSN dlsn0 = FutureUtils.result(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
-                LOG.info("writer0 write record {}", dlsn0);
-                if (k == 5) {
-                    writer0.setForceRolling(false);
-                    writer1.setForceRolling(false);
-                    retentionPeriodOverride = System.currentTimeMillis();
-                }
-                Thread.sleep(5);
-            }
-            FutureUtils.result(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
-            FutureUtils.result(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
-        }
-        writer0.close();
-        writer1.close();
-
-        DistributedLogManager dlmreader0 = createNewDLM(conf, name + "-0");
-        DistributedLogManager dlmreader1 = createNewDLM(conf, name + "-1");
-        LogReader reader0 = dlmreader0.getInputStream(1);
-        LogReader reader1 = dlmreader1.getInputStream(1);
-        int numTrans = drainStreams(reader0, 15, reader1, 15);
-        assertEquals(30, numTrans);
-        reader0.close();
-        reader1.close();
-        dlmreader0.close();
-        dlmwrite0.close();
-        dlmreader1.close();
-        dlmwrite1.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testInterleavedReadersWithRecovery() throws Exception {
-        String name = "distrlog-interleaved-recovery";
-        BKDistributedLogManager dlmwrite0 = createNewDLM(conf, name + "-0");
-        BKDistributedLogManager dlmreader0 = createNewDLM(conf, name + "-0");
-        BKDistributedLogManager dlmwrite1 = createNewDLM(conf, name + "-1");
-        BKDistributedLogManager dlmreader1 = createNewDLM(conf, name + "-1");
-
-        LogReader reader0 = null;
-        LogReader reader1 = null;
-        long txid = 1;
-        int numTrans = 0;
-
-        BKAsyncLogWriter writer0 = dlmwrite0.startAsyncLogSegmentNonPartitioned();
-        BKAsyncLogWriter writer1 = dlmwrite1.startAsyncLogSegmentNonPartitioned();
-        for (long j = 1; j <= 2; j++) {
-            for (int k = 1; k <= 6; k++) {
-                if (k == 3) {
-                    writer0.setForceRecovery(true);
-                    writer1.setForceRecovery(true);
-                }
-                DLSN dlsn1 = FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
-                LOG.info("writer1 write record {} - txid = {}", dlsn1, txid-1);
-                DLSN dlsn0 = FutureUtils.result(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)));
-            if (null == reader0) {
-                reader0 = dlmreader0.getInputStream(1);
-            }
-            if (null == reader1) {
-                reader1 = dlmreader1.getInputStream(1);
-            }
-            numTrans += drainStreams(reader0, 6, reader1, 6);
-            assertEquals((txid - 1), numTrans);
-        }
-        reader0.close();
-        reader1.close();
-        assertEquals(txid - 1,
-            dlmreader0.getLogRecordCount() + dlmreader1.getLogRecordCount());
-        dlmreader0.close();
-        dlmwrite0.close();
-        dlmreader1.close();
-        dlmwrite1.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testInterleavedReadersWithRollingEdgeUnPartitioned() throws Exception {
-        String name = "distrlog-interleaved-rolling-edge-unpartitioned";
-        BKDistributedLogManager dlmwrite0 = createNewDLM(conf, name + "-0");
-        BKDistributedLogManager dlmreader0 = createNewDLM(conf, name + "-0");
-        BKDistributedLogManager dlmwrite1 = createNewDLM(conf, name + "-1");
-        BKDistributedLogManager dlmreader1 = createNewDLM(conf, name + "-1");
-
-        LogReader reader0 = null;
-        LogReader reader1 = null;
-        long txid = 1;
-        int numTrans = 0;
-
-        BKAsyncLogWriter writer0 = dlmwrite0.startAsyncLogSegmentNonPartitioned();
-        BKAsyncLogWriter writer1 = dlmwrite1.startAsyncLogSegmentNonPartitioned();
-        for (long j = 1; j <= 4; j++) {
-            if (j > 1) {
-                writer0.setForceRolling(true);
-                writer1.setForceRolling(true);
-            }
-            for (int k = 1; k <= 2; k++) {
-                FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
-                FutureUtils.result(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)));
-            if (null == reader0) {
-                reader0 = dlmreader0.getInputStream(1);
-            }
-            if (null == reader1) {
-                reader1 = dlmreader1.getInputStream(1);
-            }
-            numTrans += drainStreams(reader0, 2, reader1, 2);
-            assertEquals((txid - 1), numTrans);
-        }
-        reader0.close();
-        reader1.close();
-        dlmreader0.close();
-        dlmreader1.close();
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentCreation.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentCreation.java
deleted file mode 100644
index 42b3ed5..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentCreation.java
+++ /dev/null
@@ -1,106 +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 com.twitter.distributedlog;
-
-import java.net.URI;
-import java.util.List;
-
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.util.Await;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.junit.Assert.*;
-
-public class TestLogSegmentCreation extends TestDistributedLogBase {
-
-    static Logger LOG = LoggerFactory.getLogger(TestLogSegmentCreation.class);
-
-    @Test(timeout = 60000)
-    public void testCreateLogSegmentAfterLoseLock() throws Exception {
-        URI uri = createDLMURI("/LogSegmentCreation");
-        String name = "distrlog-createlogsegment-afterloselock";
-        DistributedLogConfiguration conf = new DistributedLogConfiguration()
-                .setLockTimeout(99999)
-                .setOutputBufferSize(0)
-                .setImmediateFlushEnabled(true)
-                .setEnableLedgerAllocatorPool(true)
-                .setLedgerAllocatorPoolName("test");
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf).uri(uri).build();
-        DistributedLogManager dlm = namespace.openLog(name);
-        final int numSegments = 3;
-        for (int i = 0; i < numSegments; i++) {
-            BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-            out.write(DLMTestUtil.getLogRecordInstance(i));
-            out.closeAndComplete();
-        }
-
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        LOG.info("Segments : {}", segments);
-        assertEquals(3, segments.size());
-
-        final DistributedLogManager dlm1 = namespace.openLog(name);
-        final DistributedLogManager dlm2 = namespace.openLog(name);
-
-        BKAsyncLogWriter writer1 = (BKAsyncLogWriter) dlm1.startAsyncLogSegmentNonPartitioned();
-        LOG.info("Created writer 1.");
-        BKSyncLogWriter writer2 = (BKSyncLogWriter) dlm2.startLogSegmentNonPartitioned();
-        LOG.info("Created writer 2.");
-        writer2.write(DLMTestUtil.getLogRecordInstance(numSegments));
-        writer2.closeAndComplete();
-
-        try {
-            Await.result(writer1.write(DLMTestUtil.getLogRecordInstance(numSegments + 1)));
-            fail("Should fail on writing new log records.");
-        } catch (Throwable t) {
-            LOG.error("Failed to write entry : ", t);
-        }
-
-        segments = dlm.getLogSegments();
-
-        boolean hasInprogress = false;
-        boolean hasDuplicatedSegment = false;
-        long nextSeqNo = segments.get(0).getLogSegmentSequenceNumber();
-        for (int i = 1; i < segments.size(); i++) {
-            LogSegmentMetadata segment = segments.get(i);
-            assertTrue(segment.getLogSegmentSequenceNumber() >= nextSeqNo);
-            if (segment.getLogSegmentSequenceNumber() == nextSeqNo) {
-                hasDuplicatedSegment = true;
-            }
-            nextSeqNo = segment.getLogSegmentSequenceNumber();
-            if (segment.isInProgress()) {
-                hasInprogress = true;
-            }
-        }
-        assertEquals(4, segments.size());
-        assertFalse(hasInprogress);
-        assertFalse(hasDuplicatedSegment);
-
-        LOG.info("Segments : duplicated = {}, inprogress = {}, {}",
-                 new Object[] { hasDuplicatedSegment, hasInprogress, segments });
-
-        dlm1.close();
-        dlm2.close();
-        dlm.close();
-
-        namespace.close();
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentMetadata.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentMetadata.java
deleted file mode 100644
index 9e2c22e..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentMetadata.java
+++ /dev/null
@@ -1,159 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.LogSegmentMetadata.LogSegmentMetadataBuilder;
-import com.twitter.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion;
-import com.twitter.distributedlog.LogSegmentMetadata.TruncationStatus;
-import com.twitter.distributedlog.exceptions.UnsupportedMetadataVersionException;
-
-import com.twitter.distributedlog.util.FutureUtils;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Test {@link LogSegmentMetadata}
- */
-public class TestLogSegmentMetadata extends ZooKeeperClusterTestCase {
-
-    static final Logger LOG = LoggerFactory.getLogger(TestLogSegmentMetadata.class);
-
-    static final int TEST_REGION_ID = 0xf - 1;
-
-    private ZooKeeperClient zkc;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .zkServers(zkServers)
-                .build();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        zkc.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testReadMetadata() throws Exception {
-        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"));
-        assertEquals(metadata1, read1);
-        assertEquals(TEST_REGION_ID, read1.getRegionId());
-    }
-
-    @Test(timeout = 60000)
-    public void testReadMetadataCrossVersion() throws Exception {
-        LogSegmentMetadata metadata1 = new LogSegmentMetadataBuilder("/metadata2",
-            1, 1000, 1).setRegionId(TEST_REGION_ID).build();
-        metadata1.write(zkc);
-        // synchronous read
-        LogSegmentMetadata read1 = FutureUtils.result(LogSegmentMetadata.read(zkc, "/metadata2", true));
-        assertEquals(read1.getLogSegmentId(), metadata1.getLogSegmentId());
-        assertEquals(read1.getFirstTxId(), metadata1.getFirstTxId());
-        assertEquals(read1.getLastTxId(), metadata1.getLastTxId());
-        assertEquals(read1.getLogSegmentSequenceNumber(), metadata1.getLogSegmentSequenceNumber());
-        assertEquals(DistributedLogConstants.LOCAL_REGION_ID, read1.getRegionId());
-    }
-
-    @Test(timeout = 60000)
-    public void testReadMetadataCrossVersionFailure() throws Exception {
-        LogSegmentMetadata metadata1 = new LogSegmentMetadataBuilder("/metadata-failure",
-            1, 1000, 1).setRegionId(TEST_REGION_ID).build();
-        metadata1.write(zkc);
-        // synchronous read
-        try {
-            LogSegmentMetadata read1 = FutureUtils.result(LogSegmentMetadata.read(zkc, "/metadata-failure"));
-            fail("The previous statement should throw an exception");
-        } catch (UnsupportedMetadataVersionException e) {
-            // Expected
-        }
-    }
-
-
-    @Test(timeout = 60000)
-    public void testMutateTruncationStatus() {
-        LogSegmentMetadata metadata =
-                new LogSegmentMetadataBuilder(
-                        "/metadata", LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES, 1L, 0L)
-                        .setRegionId(0).setLogSegmentSequenceNo(1L).build();
-        metadata = metadata.completeLogSegment("/completed-metadata", 1000L, 1000, 1000L, 0L, 0L);
-
-        LogSegmentMetadata partiallyTruncatedSegment =
-                metadata.mutator()
-                        .setTruncationStatus(TruncationStatus.PARTIALLY_TRUNCATED)
-                        .setMinActiveDLSN(new DLSN(1L, 500L, 0L))
-                        .build();
-
-        LogSegmentMetadata fullyTruncatedSegment =
-                partiallyTruncatedSegment.mutator()
-                        .setTruncationStatus(TruncationStatus.TRUNCATED)
-                        .build();
-
-        assertEquals(new DLSN(1L, 500L, 0L), fullyTruncatedSegment.getMinActiveDLSN());
-    }
-
-    @Test(timeout = 60000)
-    public void testParseInvalidMetadata() throws Exception {
-        try {
-            LogSegmentMetadata.parseData("/metadata1", new byte[0], false);
-            fail("Should fail to parse invalid metadata");
-        } catch (IOException ioe) {
-            // expected
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testReadLogSegmentWithSequenceId() throws Exception {
-        LogSegmentMetadata metadata =
-                new LogSegmentMetadataBuilder(
-                        "/metadata", LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID, 1L, 0L)
-                        .setRegionId(0)
-                        .setLogSegmentSequenceNo(1L)
-                        .setStartSequenceId(999L)
-                        .build();
-        // write inprogress log segment with v5
-        String data = metadata.getFinalisedData();
-        LogSegmentMetadata parsedMetadata = LogSegmentMetadata.parseData("/metadatav5", data.getBytes(UTF_8), false);
-        assertEquals(999L, parsedMetadata.getStartSequenceId());
-
-        LogSegmentMetadata metadatav4 =
-                new LogSegmentMetadataBuilder(
-                        "/metadata", LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES, 1L, 0L)
-                        .setRegionId(0)
-                        .setLogSegmentSequenceNo(1L)
-                        .setStartSequenceId(999L)
-                        .build();
-        String datav4 = metadatav4.getFinalisedData();
-        LogSegmentMetadata parsedMetadatav4 =
-                LogSegmentMetadata.parseData("/metadatav4", datav4.getBytes(UTF_8), false);
-        assertTrue(parsedMetadatav4.getStartSequenceId() < 0);
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentsZK.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentsZK.java
deleted file mode 100644
index 06c7bba..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestLogSegmentsZK.java
+++ /dev/null
@@ -1,245 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.metadata.LogMetadata;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.util.DLUtils;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.zookeeper.data.Stat;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.List;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
-
-public class TestLogSegmentsZK extends TestDistributedLogBase {
-
-    static Logger LOG = LoggerFactory.getLogger(TestLogSegmentsZK.class);
-
-    private static MaxLogSegmentSequenceNo getMaxLogSegmentSequenceNo(ZooKeeperClient zkc, URI uri, String streamName,
-                                                                      DistributedLogConfiguration conf) throws Exception {
-        Stat stat = new Stat();
-        String logSegmentsPath = LogMetadata.getLogSegmentsPath(
-                uri, streamName, conf.getUnpartitionedStreamName());
-        byte[] data = zkc.get().getData(logSegmentsPath, false, stat);
-        Versioned<byte[]> maxLSSNData = new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
-        return new MaxLogSegmentSequenceNo(maxLSSNData);
-    }
-
-    private static void updateMaxLogSegmentSequenceNo(ZooKeeperClient zkc, URI uri, String streamName,
-                                                      DistributedLogConfiguration conf, byte[] data) throws Exception {
-        String logSegmentsPath = LogMetadata.getLogSegmentsPath(
-                uri, streamName, conf.getUnpartitionedStreamName());
-        zkc.get().setData(logSegmentsPath, data, -1);
-    }
-
-    @Rule
-    public TestName testName = new TestName();
-
-    private URI createURI() throws Exception {
-        return createDLMURI("/" + testName.getMethodName());
-    }
-
-    /**
-     * Create Log Segment for an pre-create stream. No max ledger sequence number recorded.
-     */
-    @Test(timeout = 60000)
-    public void testCreateLogSegmentOnPrecreatedStream() throws Exception {
-        URI uri = createURI();
-        String streamName = testName.getMethodName();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration()
-                .setLockTimeout(99999)
-                .setOutputBufferSize(0)
-                .setImmediateFlushEnabled(true)
-                .setEnableLedgerAllocatorPool(true)
-                .setLedgerAllocatorPoolName("test");
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
-
-        namespace.createLog(streamName);
-        MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
-        assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO, max1.getSequenceNumber());
-        DistributedLogManager dlm = namespace.openLog(streamName);
-        final int numSegments = 3;
-        for (int i = 0; i < numSegments; i++) {
-            BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-            out.write(DLMTestUtil.getLogRecordInstance(i));
-            out.closeAndComplete();
-        }
-        MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
-        assertEquals(3, max2.getSequenceNumber());
-        dlm.close();
-        namespace.close();
-    }
-
-    /**
-     * Create Log Segment when no max sequence number recorded in /ledgers. e.g. old version.
-     */
-    @Test(timeout = 60000)
-    public void testCreateLogSegmentMissingMaxSequenceNumber() throws Exception {
-        URI uri = createURI();
-        String streamName = testName.getMethodName();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration()
-                .setLockTimeout(99999)
-                .setOutputBufferSize(0)
-                .setImmediateFlushEnabled(true)
-                .setEnableLedgerAllocatorPool(true)
-                .setLedgerAllocatorPoolName("test");
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
-
-        namespace.createLog(streamName);
-        MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
-        assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO, max1.getSequenceNumber());
-        DistributedLogManager dlm = namespace.openLog(streamName);
-        final int numSegments = 3;
-        for (int i = 0; i < numSegments; i++) {
-            BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-            out.write(DLMTestUtil.getLogRecordInstance(i));
-            out.closeAndComplete();
-        }
-        MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
-        assertEquals(3, max2.getSequenceNumber());
-
-        // nuke the max ledger sequence number
-        updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf, new byte[0]);
-        DistributedLogManager dlm1 = namespace.openLog(streamName);
-        try {
-            dlm1.startLogSegmentNonPartitioned();
-            fail("Should fail with unexpected exceptions");
-        } catch (UnexpectedException ue) {
-            // expected
-        } finally {
-            dlm1.close();
-        }
-
-        // invalid max ledger sequence number
-        updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf, "invalid-max".getBytes(UTF_8));
-        DistributedLogManager dlm2 = namespace.openLog(streamName);
-        try {
-            dlm2.startLogSegmentNonPartitioned();
-            fail("Should fail with unexpected exceptions");
-        } catch (UnexpectedException ue) {
-            // expected
-        } finally {
-            dlm2.close();
-        }
-
-        dlm.close();
-        namespace.close();
-    }
-
-    /**
-     * Create Log Segment while max sequence number isn't match with list of log segments.
-     */
-    @Test(timeout = 60000)
-    public void testCreateLogSegmentUnmatchMaxSequenceNumber() throws Exception {
-        URI uri = createURI();
-        String streamName = testName.getMethodName();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration()
-                .setLockTimeout(99999)
-                .setOutputBufferSize(0)
-                .setImmediateFlushEnabled(true)
-                .setEnableLedgerAllocatorPool(true)
-                .setLedgerAllocatorPoolName("test");
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
-
-        namespace.createLog(streamName);
-        MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
-        assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO, max1.getSequenceNumber());
-        DistributedLogManager dlm = namespace.openLog(streamName);
-        final int numSegments = 3;
-        for (int i = 0; i < numSegments; i++) {
-            BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-            out.write(DLMTestUtil.getLogRecordInstance(i));
-            out.closeAndComplete();
-        }
-        MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
-        assertEquals(3, max2.getSequenceNumber());
-
-        // update the max ledger sequence number
-        updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf,
-                DLUtils.serializeLogSegmentSequenceNumber(99));
-
-        DistributedLogManager dlm1 = namespace.openLog(streamName);
-        try {
-            BKSyncLogWriter out1 = (BKSyncLogWriter) dlm1.startLogSegmentNonPartitioned();
-            out1.write(DLMTestUtil.getLogRecordInstance(numSegments+1));
-            out1.closeAndComplete();
-            fail("Should fail creating new log segment when encountered unmatch max ledger sequence number");
-        } catch (DLIllegalStateException lse) {
-            // expected
-        } finally {
-            dlm1.close();
-        }
-
-        DistributedLogManager dlm2 = namespace.openLog(streamName);
-        List<LogSegmentMetadata> segments = dlm2.getLogSegments();
-        try {
-            assertEquals(3, segments.size());
-            assertEquals(1L, segments.get(0).getLogSegmentSequenceNumber());
-            assertEquals(2L, segments.get(1).getLogSegmentSequenceNumber());
-            assertEquals(3L, segments.get(2).getLogSegmentSequenceNumber());
-        } finally {
-            dlm2.close();
-        }
-
-        dlm.close();
-        namespace.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testCompleteLogSegmentConflicts() throws Exception {
-        URI uri = createURI();
-        String streamName = testName.getMethodName();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration()
-                .setLockTimeout(99999)
-                .setOutputBufferSize(0)
-                .setImmediateFlushEnabled(true)
-                .setEnableLedgerAllocatorPool(true)
-                .setLedgerAllocatorPoolName("test");
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
-
-        namespace.createLog(streamName);
-        DistributedLogManager dlm1 = namespace.openLog(streamName);
-        DistributedLogManager dlm2 = namespace.openLog(streamName);
-
-        // dlm1 is writing
-        BKSyncLogWriter out1 = (BKSyncLogWriter) dlm1.startLogSegmentNonPartitioned();
-        out1.write(DLMTestUtil.getLogRecordInstance(1));
-        // before out1 complete, out2 is in on recovery
-        // it completed the log segments which bump the version of /ledgers znode
-        BKAsyncLogWriter out2 = (BKAsyncLogWriter) dlm2.startAsyncLogSegmentNonPartitioned();
-
-        try {
-            out1.closeAndComplete();
-            fail("Should fail closeAndComplete since other people already completed it.");
-        } catch (IOException ioe) {
-        }
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReads.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReads.java
deleted file mode 100644
index 9553637..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReads.java
+++ /dev/null
@@ -1,348 +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 com.twitter.distributedlog;
-
-import java.util.concurrent.ScheduledFuture;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import java.util.concurrent.TimeUnit;
-
-import com.twitter.distributedlog.annotations.DistributedLogAnnotations;
-import com.twitter.distributedlog.exceptions.IdleReaderException;
-import com.twitter.distributedlog.util.FutureUtils;
-import org.junit.Assert;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static com.twitter.distributedlog.NonBlockingReadsTestUtil.*;
-import static org.junit.Assert.*;
-
-/**
- * {@link https://issues.apache.org/jira/browse/DL-12}
- */
-@DistributedLogAnnotations.FlakyTest
-@Ignore
-public class TestNonBlockingReads extends TestDistributedLogBase {
-    static final Logger LOG = LoggerFactory.getLogger(TestNonBlockingReads.class);
-
-    static {
-        conf.setOutputBufferSize(0);
-        conf.setImmediateFlushEnabled(true);
-    }
-
-    @Test(timeout = 100000)
-    public void testNonBlockingRead() throws Exception {
-        String name = "distrlog-non-blocking-reader";
-        final DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setReadAheadBatchSize(1);
-        confLocal.setReadAheadMaxRecords(1);
-        confLocal.setReaderIdleWarnThresholdMillis(100);
-        confLocal.setReadLACLongPollTimeout(49);
-        final DistributedLogManager dlm = createNewDLM(confLocal, name);
-        ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
-        ScheduledFuture writerClosedFuture = null;
-        try {
-            final Thread currentThread = Thread.currentThread();
-            writerClosedFuture = executor.schedule(
-                    new Runnable() {
-                        @Override
-                        public void run() {
-                            try {
-                                writeRecordsForNonBlockingReads(confLocal, dlm, false);
-                            } catch (Exception exc) {
-                                currentThread.interrupt();
-                            }
-
-                        }
-                    }, 100, TimeUnit.MILLISECONDS);
-
-            readNonBlocking(dlm, false);
-            assertFalse(currentThread.isInterrupted());
-        } finally {
-            if (writerClosedFuture != null){
-                // ensure writer.closeAndComplete is done before we close dlm
-                writerClosedFuture.get();
-            }
-            executor.shutdown();
-            dlm.close();
-        }
-    }
-
-    @Test(timeout = 100000)
-    public void testNonBlockingReadRecovery() throws Exception {
-        String name = "distrlog-non-blocking-reader-recovery";
-        final DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setReadAheadBatchSize(10);
-        confLocal.setReadAheadMaxRecords(10);
-        final DistributedLogManager dlm = createNewDLM(confLocal, name);
-        ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
-        ScheduledFuture writerClosedFuture = null;
-        try {
-            final Thread currentThread = Thread.currentThread();
-            writerClosedFuture = executor.schedule(
-                    new Runnable() {
-                        @Override
-                        public void run() {
-                            try {
-                                writeRecordsForNonBlockingReads(confLocal, dlm, true);
-                            } catch (Exception exc) {
-                                currentThread.interrupt();
-                            }
-
-                        }
-                    }, 100, TimeUnit.MILLISECONDS);
-
-
-            readNonBlocking(dlm, false);
-            assertFalse(currentThread.isInterrupted());
-        } finally {
-            if (writerClosedFuture != null){
-                // ensure writer.closeAndComplete is done before we close dlm
-                writerClosedFuture.get();
-            }
-            executor.shutdown();
-            dlm.close();
-        }
-    }
-
-    @Test(timeout = 100000)
-    public void testNonBlockingReadIdleError() throws Exception {
-        String name = "distrlog-non-blocking-reader-error";
-        final DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setReadAheadBatchSize(1);
-        confLocal.setReadAheadMaxRecords(1);
-        confLocal.setReadLACLongPollTimeout(24);
-        confLocal.setReaderIdleWarnThresholdMillis(50);
-        confLocal.setReaderIdleErrorThresholdMillis(100);
-        final DistributedLogManager dlm = createNewDLM(confLocal, name);
-        ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
-        ScheduledFuture writerClosedFuture = null;
-        try {
-            final Thread currentThread = Thread.currentThread();
-            writerClosedFuture = executor.schedule(
-                    new Runnable() {
-                        @Override
-                        public void run() {
-                            try {
-                                writeRecordsForNonBlockingReads(confLocal, dlm, false);
-                            } catch (Exception exc) {
-                                currentThread.interrupt();
-                            }
-
-                        }
-                    }, 100, TimeUnit.MILLISECONDS);
-
-            boolean exceptionEncountered = false;
-            try {
-                readNonBlocking(dlm, false, DEFAULT_SEGMENT_SIZE, true);
-            } catch (IdleReaderException exc) {
-                exceptionEncountered = true;
-            }
-            assertTrue(exceptionEncountered);
-            assertFalse(currentThread.isInterrupted());
-        } finally {
-            if (writerClosedFuture != null){
-                // ensure writer.closeAndComplete is done before we close dlm
-                writerClosedFuture.get();
-            }
-            executor.shutdown();
-            dlm.close();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testNonBlockingReadAheadStall() throws Exception {
-        String name = "distrlog-non-blocking-reader-stall";
-        final DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setReadAheadBatchSize(1);
-        confLocal.setReadAheadMaxRecords(3);
-        confLocal.setReadLACLongPollTimeout(249);
-        confLocal.setReaderIdleWarnThresholdMillis(500);
-        confLocal.setReaderIdleErrorThresholdMillis(30000);
-        final DistributedLogManager dlm = createNewDLM(confLocal, name);
-        ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
-        ScheduledFuture writerClosedFuture = null;
-        try {
-            final Thread currentThread = Thread.currentThread();
-            writerClosedFuture = executor.schedule(
-                    new Runnable() {
-                        @Override
-                        public void run() {
-                            try {
-                                writeRecordsForNonBlockingReads(confLocal, dlm, false, 3);
-                            } catch (Exception exc) {
-                                currentThread.interrupt();
-                            }
-
-                        }
-                    }, 10, TimeUnit.MILLISECONDS);
-
-            boolean exceptionEncountered = false;
-            try {
-                readNonBlocking(dlm, false, 3, false);
-            } catch (IdleReaderException exc) {
-                LOG.info("Exception encountered", exc);
-                exceptionEncountered = true;
-            }
-            assertFalse(exceptionEncountered);
-            assertFalse(currentThread.isInterrupted());
-        } finally {
-            if (writerClosedFuture != null){
-                // ensure writer.closeAndComplete is done before we close dlm
-                writerClosedFuture.get();
-            }
-            executor.shutdown();
-            dlm.close();
-        }
-    }
-
-    private long createStreamWithInconsistentMetadata(String name) throws Exception {
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        ZooKeeperClient zkClient = TestZooKeeperClientBuilder.newBuilder()
-                .uri(createDLMURI("/"))
-                .build();
-        long txid = 1;
-
-        long numRecordsWritten = 0;
-        int segmentSize = 10;
-        for (long i = 0; i < 3; i++) {
-            BKAsyncLogWriter out = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-            for (long j = 1; j <= segmentSize; j++) {
-                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                FutureUtils.result(out.write(op));
-                numRecordsWritten++;
-            }
-            out.closeAndComplete();
-        }
-
-        BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
-        String completedZNode = blplm.completedLedgerZNode(txid - segmentSize, txid - 1, 3);
-        LogSegmentMetadata metadata = FutureUtils.result(LogSegmentMetadata.read(zkClient, completedZNode));
-        zkClient.get().delete(completedZNode, -1);
-        LogSegmentMetadata metadataToChange =
-                metadata.mutator()
-                        .setLastEntryId(metadata.getLastEntryId() + 100)
-                        .setLastTxId(metadata.getLastTxId() + 100)
-                        .build();
-        metadataToChange.write(zkClient);
-
-        txid += 100;
-
-
-        for (long i = 0; i < 3; i++) {
-            BKAsyncLogWriter out = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-            for (long j = 1; j <= segmentSize; j++) {
-                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
-                FutureUtils.result(out.write(op));
-                numRecordsWritten++;
-            }
-            out.closeAndComplete();
-        }
-        dlm.close();
-
-        return numRecordsWritten;
-    }
-
-    @Test(timeout = 60000)
-    public void testHandleInconsistentMetadata() throws Exception {
-        String name = "distrlog-inconsistent-metadata-blocking-read";
-        long numRecordsWritten = createStreamWithInconsistentMetadata(name);
-
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        try {
-            LogReader reader = dlm.getInputStream(45);
-            long numRecordsRead = 0;
-            LogRecord record = reader.readNext(false);
-            long lastTxId = -1;
-            while (numRecordsRead < numRecordsWritten / 2) {
-                if (null != record) {
-                    DLMTestUtil.verifyLogRecord(record);
-                    Assert.assertTrue(lastTxId < record.getTransactionId());
-                    lastTxId = record.getTransactionId();
-                    numRecordsRead++;
-                } else {
-                    Thread.sleep(1);
-                }
-                record = reader.readNext(false);
-            }
-            reader.close();
-            assertEquals(numRecordsWritten / 2, numRecordsRead);
-        } finally {
-            dlm.close();
-        }
-    }
-
-    @Test(timeout = 15000)
-    public void testHandleInconsistentMetadataNonBlocking() throws Exception {
-        String name = "distrlog-inconsistent-metadata-nonblocking-read";
-        long numRecordsWritten = createStreamWithInconsistentMetadata(name);
-
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        try {
-            LogReader reader = dlm.getInputStream(45);
-            long numRecordsRead = 0;
-            long lastTxId = -1;
-            while (numRecordsRead < (numRecordsWritten / 2)) {
-                LogRecord record = reader.readNext(false);
-                if (record != null) {
-                    DLMTestUtil.verifyLogRecord(record);
-                    Assert.assertTrue(lastTxId < record.getTransactionId());
-                    lastTxId = record.getTransactionId();
-                    numRecordsRead++;
-                } else {
-                    Thread.sleep(1);
-                }
-            }
-            reader.close();
-        } finally {
-            dlm.close();
-        }
-    }
-
-    @Test(timeout = 15000)
-    public void testHandleInconsistentMetadataDLSNNonBlocking() throws Exception {
-        String name = "distrlog-inconsistent-metadata-nonblocking-read-dlsn";
-        long numRecordsWritten = createStreamWithInconsistentMetadata(name);
-
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        try {
-            LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
-            long numRecordsRead = 0;
-            long lastTxId = -1;
-            while (numRecordsRead < numRecordsWritten) {
-                LogRecord record = reader.readNext(false);
-                if (record != null) {
-                    DLMTestUtil.verifyLogRecord(record);
-                    Assert.assertTrue(lastTxId < record.getTransactionId());
-                    lastTxId = record.getTransactionId();
-                    numRecordsRead++;
-                } else {
-                    Thread.sleep(1);
-                }
-            }
-            reader.close();
-        } finally {
-            dlm.close();
-        }
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReadsMultiReader.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReadsMultiReader.java
deleted file mode 100644
index bfa9156..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestNonBlockingReadsMultiReader.java
+++ /dev/null
@@ -1,168 +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 com.twitter.distributedlog;
-
-import com.google.common.util.concurrent.RateLimiter;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Utils;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.junit.Assert.*;
-
-public class TestNonBlockingReadsMultiReader extends TestDistributedLogBase {
-
-        static class ReaderThread extends Thread {
-
-        final LogReader reader;
-        final boolean nonBlockReading;
-        volatile boolean running = true;
-        final AtomicInteger readCount = new AtomicInteger(0);
-
-        ReaderThread(String name, LogReader reader, boolean nonBlockReading) {
-            super(name);
-            this.reader = reader;
-            this.nonBlockReading = nonBlockReading;
-        }
-
-        @Override
-        public void run() {
-            while (running) {
-                try {
-                    LogRecord r = reader.readNext(nonBlockReading);
-                    if (r != null) {
-                        readCount.incrementAndGet();
-                        if (readCount.get() % 1000 == 0) {
-                            LOG.info("{} reading {}", getName(), r.getTransactionId());
-                        }
-                    }
-                } catch (DLInterruptedException die) {
-                    Thread.currentThread().interrupt();
-                } catch (IOException e) {
-                    break;
-                }
-            }
-        }
-
-        void stopReading() {
-            LOG.info("Stopping reader.");
-            running = false;
-            interrupt();
-            try {
-                join();
-            } catch (InterruptedException e) {
-                LOG.error("Interrupted on waiting reader thread {} exiting : ", getName(), e);
-            }
-        }
-
-        int getReadCount() {
-            return readCount.get();
-        }
-
-    }
-
-    @Test(timeout = 60000)
-    public void testMultiReaders() throws Exception {
-        String name = "distrlog-multireaders";
-        final RateLimiter limiter = RateLimiter.create(1000);
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.setOutputBufferSize(0);
-        confLocal.setImmediateFlushEnabled(true);
-
-        DistributedLogManager dlmwrite = createNewDLM(confLocal, name);
-
-        final AsyncLogWriter writer = dlmwrite.startAsyncLogSegmentNonPartitioned();
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(0)));
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1)));
-        final AtomicInteger writeCount = new AtomicInteger(2);
-
-        DistributedLogManager dlmread = createNewDLM(conf, name);
-
-        BKSyncLogReader reader0 = (BKSyncLogReader) dlmread.getInputStream(0);
-
-        try {
-            ReaderThread[] readerThreads = new ReaderThread[1];
-            readerThreads[0] = new ReaderThread("reader0-non-blocking", reader0, false);
-            // readerThreads[1] = new ReaderThread("reader1-non-blocking", reader0, false);
-
-            final AtomicBoolean running = new AtomicBoolean(true);
-            Thread writerThread = new Thread("WriteThread") {
-                @Override
-                public void run() {
-                    try {
-                        long txid = 2;
-                        DLSN dlsn = DLSN.InvalidDLSN;
-                        while (running.get()) {
-                            limiter.acquire();
-                            long curTxId = txid++;
-                            dlsn = FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(curTxId)));
-                            writeCount.incrementAndGet();
-                            if (curTxId % 1000 == 0) {
-                                LOG.info("writer write {}", curTxId);
-                            }
-                        }
-                        LOG.info("Completed writing record at {}", dlsn);
-                        Utils.close(writer);
-                    } catch (DLInterruptedException die) {
-                        Thread.currentThread().interrupt();
-                    } catch (IOException e) {
-
-                    }
-                }
-            };
-
-            for (ReaderThread rt : readerThreads) {
-                rt.start();
-            }
-
-            writerThread.start();
-
-            TimeUnit.SECONDS.sleep(5);
-
-            LOG.info("Stopping writer");
-
-            running.set(false);
-            writerThread.join();
-
-            LOG.info("Writer stopped after writing {} records, waiting for reader to complete",
-                    writeCount.get());
-            while (writeCount.get() > (readerThreads[0].getReadCount())) {
-                LOG.info("Write Count = {}, Read Count = {}",
-                        new Object[] { writeCount.get(), readerThreads[0].getReadCount() });
-                TimeUnit.MILLISECONDS.sleep(100);
-            }
-            assertEquals(writeCount.get(),
-                    (readerThreads[0].getReadCount()));
-
-            for (ReaderThread readerThread : readerThreads) {
-                readerThread.stopReading();
-            }
-        } finally {
-            dlmwrite.close();
-            reader0.close();
-            dlmread.close();
-        }
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadAheadEntryReader.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadAheadEntryReader.java
deleted file mode 100644
index cf4fc4f..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadAheadEntryReader.java
+++ /dev/null
@@ -1,463 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Ticker;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.exceptions.AlreadyTruncatedTransactionException;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryStore;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.Promise;
-import org.apache.bookkeeper.stats.AlertStatsLogger;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.*;
-
-/**
- * Test Case {@link ReadAheadEntryReader}
- */
-public class TestReadAheadEntryReader extends TestDistributedLogBase {
-
-    private static final int MAX_CACHED_ENTRIES = 5;
-    private static final int NUM_PREFETCH_ENTRIES = 10;
-
-    @Rule
-    public TestName runtime = new TestName();
-    private DistributedLogConfiguration baseConf;
-    private OrderedScheduler scheduler;
-    private BookKeeperClient bkc;
-    private ZooKeeperClient zkc;
-
-    @Before
-    public void setup() throws Exception {
-        super.setup();
-        baseConf = new DistributedLogConfiguration();
-        baseConf.addConfiguration(conf);
-        baseConf.setOutputBufferSize(0);
-        baseConf.setPeriodicFlushFrequencyMilliSeconds(0);
-        baseConf.setImmediateFlushEnabled(false);
-        baseConf.setReadAheadMaxRecords(MAX_CACHED_ENTRIES);
-        baseConf.setNumPrefetchEntriesPerLogSegment(NUM_PREFETCH_ENTRIES);
-        baseConf.setMaxPrefetchEntriesPerLogSegment(NUM_PREFETCH_ENTRIES);
-        zkc = ZooKeeperClientBuilder.newBuilder()
-                .name("test-zk")
-                .zkServers(bkutil.getZkServers())
-                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                .zkAclId(conf.getZkAclId())
-                .build();
-        bkc = BookKeeperClientBuilder.newBuilder()
-                .name("test-bk")
-                .dlConfig(conf)
-                .ledgersPath("/ledgers")
-                .zkServers(bkutil.getZkServers())
-                .build();
-        scheduler = OrderedScheduler.newBuilder()
-                .name("test-read-ahead-entry-reader")
-                .corePoolSize(1)
-                .build();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        if (null != bkc) {
-            bkc.close();
-        }
-        if (null != scheduler) {
-            scheduler.shutdown();
-        }
-        if (null != zkc) {
-            zkc.close();
-        }
-        super.teardown();
-    }
-
-    private ReadAheadEntryReader createEntryReader(String streamName,
-                                                   DLSN fromDLSN,
-                                                   BKDistributedLogManager dlm,
-                                                   DistributedLogConfiguration conf)
-            throws Exception {
-        BKLogReadHandler readHandler = dlm.createReadHandler(
-                Optional.<String>absent(),
-                true);
-        LogSegmentEntryStore entryStore = new BKLogSegmentEntryStore(
-                conf,
-                ConfUtils.getConstDynConf(conf),
-                zkc,
-                bkc,
-                scheduler,
-                null,
-                NullStatsLogger.INSTANCE,
-                AsyncFailureInjector.NULL);
-        return new ReadAheadEntryReader(
-                streamName,
-                fromDLSN,
-                conf,
-                readHandler,
-                entryStore,
-                scheduler,
-                Ticker.systemTicker(),
-                new AlertStatsLogger(NullStatsLogger.INSTANCE, "test-alert"));
-    }
-
-    private void ensureOrderSchedulerEmpty(String streamName) throws Exception {
-        final Promise<Void> promise = new Promise<Void>();
-        scheduler.submit(streamName, new Runnable() {
-            @Override
-            public void run() {
-                FutureUtils.setValue(promise, null);
-            }
-        });
-        FutureUtils.result(promise);
-    }
-
-    void generateCompletedLogSegments(DistributedLogManager dlm,
-                                      long numCompletedSegments,
-                                      long segmentSize) throws Exception {
-        generateCompletedLogSegments(dlm, numCompletedSegments, segmentSize, 1L);
-    }
-
-    void generateCompletedLogSegments(DistributedLogManager dlm,
-                                      long numCompletedSegments,
-                                      long segmentSize,
-                                      long startTxId) throws Exception {
-
-        long txid = startTxId;
-        for (long i = 0; i < numCompletedSegments; i++) {
-            AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
-            for (long j = 1; j <= segmentSize; j++) {
-                FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
-                LogRecord ctrlRecord = DLMTestUtil.getLogRecordInstance(txid);
-                ctrlRecord.setControl();
-                FutureUtils.result(writer.write(ctrlRecord));
-            }
-            Utils.close(writer);
-        }
-    }
-
-    AsyncLogWriter createInprogressLogSegment(DistributedLogManager dlm,
-                                              DistributedLogConfiguration conf,
-                                              long segmentSize) throws Exception {
-        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
-        for (long i = 1L; i <= segmentSize; i++) {
-            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
-            LogRecord ctrlRecord = DLMTestUtil.getLogRecordInstance(i);
-            ctrlRecord.setControl();
-            FutureUtils.result(writer.write(ctrlRecord));
-        }
-        return writer;
-    }
-
-    void expectAlreadyTruncatedTransactionException(ReadAheadEntryReader reader,
-                                                    String errMsg)
-            throws Exception {
-        try {
-            reader.checkLastException();
-            fail(errMsg);
-        } catch (AlreadyTruncatedTransactionException atte) {
-            // expected
-        }
-    }
-
-    void expectIllegalStateException(ReadAheadEntryReader reader,
-                                     String errMsg)
-            throws Exception {
-        try {
-            reader.checkLastException();
-            fail(errMsg);
-        } catch (DLIllegalStateException le) {
-            // expected
-        }
-    }
-
-    void expectNoException(ReadAheadEntryReader reader) throws Exception {
-        reader.checkLastException();
-    }
-
-    //
-    // Test Positioning
-    //
-
-    @Test(timeout = 60000)
-    public void testStartWithEmptySegmentList() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager dlm = createNewDLM(baseConf, streamName);
-        ReadAheadEntryReader readAheadEntryReader =
-                createEntryReader(streamName, DLSN.InitialDLSN, dlm, baseConf);
-
-        readAheadEntryReader.start(Lists.<LogSegmentMetadata>newArrayList());
-
-        ensureOrderSchedulerEmpty(streamName);
-        assertFalse("ReadAhead should not be initialized with empty segment list",
-                readAheadEntryReader.isInitialized());
-        assertTrue("ReadAhead should be empty when it isn't initialized",
-                readAheadEntryReader.isCacheEmpty());
-        assertFalse("ReadAhead should not be marked as caught up when it isn't initialized",
-                readAheadEntryReader.isReadAheadCaughtUp());
-
-        // generate list of log segments
-        generateCompletedLogSegments(dlm, 1, MAX_CACHED_ENTRIES / 2 + 1);
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(segments.size() + " log segments found, expected to be only one",
-                1, segments.size());
-
-        // notify the readahead reader with new segment lsit
-        readAheadEntryReader.onSegmentsUpdated(segments);
-
-        // check the reader state after initialization
-        ensureOrderSchedulerEmpty(streamName);
-        assertTrue("ReadAhead should be initialized with non-empty segment list",
-                readAheadEntryReader.isInitialized());
-        assertNotNull("current segment reader should be initialized",
-                readAheadEntryReader.getCurrentSegmentReader());
-        assertEquals("current segment sequence number should be " + segments.get(0).getLogSegmentSequenceNumber(),
-                segments.get(0).getLogSegmentSequenceNumber(), readAheadEntryReader.getCurrentSegmentSequenceNumber());
-        assertNull("there should be no next segment reader",
-                readAheadEntryReader.getNextSegmentReader());
-        assertTrue("there should be no remaining segment readers",
-                readAheadEntryReader.getSegmentReaders().isEmpty());
-
-        Utils.close(readAheadEntryReader);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testInitializeMultipleClosedLogSegments0() throws Exception {
-        // 5 completed log segments, start from the begin
-        testInitializeMultipleClosedLogSegments(5, DLSN.InitialDLSN, 0);
-    }
-
-    @Test(timeout = 60000)
-    public void testInitializeMultipleClosedLogSegments1() throws Exception {
-        // 5 completed log segments, start from the 4th segment and it should skip first 3 log segments
-        testInitializeMultipleClosedLogSegments(5, new DLSN(4L, 0L, 0L), 3);
-    }
-
-    private void testInitializeMultipleClosedLogSegments(
-            int numLogSegments,
-            DLSN fromDLSN,
-            int expectedCurrentSegmentIdx
-    ) throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager dlm = createNewDLM(baseConf, streamName);
-
-        // generate list of log segments
-        generateCompletedLogSegments(dlm, 1, MAX_CACHED_ENTRIES / 2 + 1, 1L);
-        generateCompletedLogSegments(dlm, numLogSegments - 1, 1, MAX_CACHED_ENTRIES + 2);
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(segments.size() + " log segments found, expected to be " + numLogSegments,
-                numLogSegments, segments.size());
-
-        ReadAheadEntryReader readAheadEntryReader =
-                createEntryReader(streamName, fromDLSN, dlm, baseConf);
-        readAheadEntryReader.start(segments);
-
-        ensureOrderSchedulerEmpty(streamName);
-        assertTrue("ReadAhead should be initialized with non-empty segment list",
-                readAheadEntryReader.isInitialized());
-        assertNotNull("current segment reader should be initialized",
-                readAheadEntryReader.getCurrentSegmentReader());
-        assertTrue("current segment reader should be open and started",
-                readAheadEntryReader.getCurrentSegmentReader().isReaderOpen()
-                        && readAheadEntryReader.getCurrentSegmentReader().isReaderStarted());
-        assertEquals("current segment reader should read " + segments.get(expectedCurrentSegmentIdx),
-                segments.get(expectedCurrentSegmentIdx),
-                readAheadEntryReader.getCurrentSegmentReader().getSegment());
-        assertEquals("current segment sequence number should be "
-                + segments.get(expectedCurrentSegmentIdx).getLogSegmentSequenceNumber(),
-                segments.get(expectedCurrentSegmentIdx).getLogSegmentSequenceNumber(),
-                readAheadEntryReader.getCurrentSegmentSequenceNumber());
-        assertNull("next segment reader should not be initialized since it is a closed log segment",
-                readAheadEntryReader.getNextSegmentReader());
-        assertEquals("there should be " + (numLogSegments - (expectedCurrentSegmentIdx + 1))
-                + " remaining segment readers",
-                numLogSegments - (expectedCurrentSegmentIdx + 1),
-                readAheadEntryReader.getSegmentReaders().size());
-        int segmentIdx = expectedCurrentSegmentIdx + 1;
-        for (ReadAheadEntryReader.SegmentReader reader : readAheadEntryReader.getSegmentReaders()) {
-            LogSegmentMetadata expectedSegment = segments.get(segmentIdx);
-            assertEquals("Segment should " + expectedSegment,
-                    expectedSegment, reader.getSegment());
-            assertTrue("Segment reader for " + expectedSegment + " should be open",
-                    reader.isReaderOpen());
-            assertFalse("Segment reader for " + expectedSegment + " should not be started",
-                    reader.isReaderStarted());
-            ++segmentIdx;
-        }
-
-        Utils.close(readAheadEntryReader);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testPositioningAtInvalidLogSegment() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager dlm = createNewDLM(baseConf, streamName);
-
-        // generate list of log segments
-        generateCompletedLogSegments(dlm, 3, 3);
-        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
-        FutureUtils.result(writer.truncate(new DLSN(2L, 1L, 0L)));
-
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-
-        // positioning on a truncated log segment (segment 1)
-        ReadAheadEntryReader readAheadEntryReader =
-                createEntryReader(streamName, DLSN.InitialDLSN, dlm, baseConf);
-        readAheadEntryReader.start(segments);
-        // ensure initialization to complete
-        ensureOrderSchedulerEmpty(streamName);
-        expectNoException(readAheadEntryReader);
-        Entry.Reader entryReader =
-                readAheadEntryReader.getNextReadAheadEntry(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        assertEquals(2L, entryReader.getLSSN());
-        assertEquals(1L, entryReader.getEntryId());
-        Utils.close(readAheadEntryReader);
-
-        // positioning on a partially truncated log segment (segment 2) before min active dlsn
-        readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 0L, 0L), dlm, baseConf);
-        readAheadEntryReader.start(segments);
-        // ensure initialization to complete
-        ensureOrderSchedulerEmpty(streamName);
-        expectNoException(readAheadEntryReader);
-        entryReader =
-                readAheadEntryReader.getNextReadAheadEntry(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        assertEquals(2L, entryReader.getLSSN());
-        assertEquals(1L, entryReader.getEntryId());
-        Utils.close(readAheadEntryReader);
-
-        // positioning on a partially truncated log segment (segment 2) after min active dlsn
-        readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 2L, 0L), dlm, baseConf);
-        readAheadEntryReader.start(segments);
-        // ensure initialization to complete
-        ensureOrderSchedulerEmpty(streamName);
-        expectNoException(readAheadEntryReader);
-        entryReader =
-                readAheadEntryReader.getNextReadAheadEntry(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        assertEquals(2L, entryReader.getLSSN());
-        assertEquals(2L, entryReader.getEntryId());
-        Utils.close(readAheadEntryReader);
-
-        Utils.close(writer);
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testPositioningIgnoreTruncationStatus() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(baseConf);
-        confLocal.setIgnoreTruncationStatus(true);
-
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager dlm = createNewDLM(confLocal, streamName);
-
-        // generate list of log segments
-        generateCompletedLogSegments(dlm, 3, 2);
-        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
-        FutureUtils.result(writer.truncate(new DLSN(2L, 1L, 0L)));
-
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-
-        // positioning on a truncated log segment (segment 1)
-        ReadAheadEntryReader readAheadEntryReader =
-                createEntryReader(streamName, DLSN.InitialDLSN, dlm, confLocal);
-        readAheadEntryReader.start(segments);
-        // ensure initialization to complete
-        ensureOrderSchedulerEmpty(streamName);
-        expectNoException(readAheadEntryReader);
-        Utils.close(readAheadEntryReader);
-
-        // positioning on a partially truncated log segment (segment 2) before min active dlsn
-        readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 0L, 0L), dlm, confLocal);
-        readAheadEntryReader.start(segments);
-        // ensure initialization to complete
-        ensureOrderSchedulerEmpty(streamName);
-        expectNoException(readAheadEntryReader);
-        Utils.close(readAheadEntryReader);
-
-        // positioning on a partially truncated log segment (segment 2) after min active dlsn
-        readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 1L, 0L), dlm, confLocal);
-        readAheadEntryReader.start(segments);
-        // ensure initialization to complete
-        ensureOrderSchedulerEmpty(streamName);
-        expectNoException(readAheadEntryReader);
-        Utils.close(readAheadEntryReader);
-
-        Utils.close(writer);
-        dlm.close();
-    }
-
-    //
-    // Test Reinitialization
-    //
-
-    @Test(timeout = 60000)
-    public void testLogSegmentSequenceNumberGap() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager dlm = createNewDLM(baseConf, streamName);
-
-        // generate list of log segments
-        generateCompletedLogSegments(dlm, 3, 2);
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-
-        ReadAheadEntryReader readAheadEntryReader =
-                createEntryReader(streamName, DLSN.InitialDLSN, dlm, baseConf);
-        readAheadEntryReader.start(segments.subList(0, 1));
-        int expectedCurrentSegmentIdx = 0;
-        ensureOrderSchedulerEmpty(streamName);
-        assertTrue("ReadAhead should be initialized with non-empty segment list",
-                readAheadEntryReader.isInitialized());
-        assertNotNull("current segment reader should be initialized",
-                readAheadEntryReader.getCurrentSegmentReader());
-        assertTrue("current segment reader should be open and started",
-                readAheadEntryReader.getCurrentSegmentReader().isReaderOpen()
-                        && readAheadEntryReader.getCurrentSegmentReader().isReaderStarted());
-        assertEquals("current segment reader should read " + segments.get(expectedCurrentSegmentIdx),
-                segments.get(expectedCurrentSegmentIdx),
-                readAheadEntryReader.getCurrentSegmentReader().getSegment());
-        assertEquals("current segment sequence number should be "
-                + segments.get(expectedCurrentSegmentIdx).getLogSegmentSequenceNumber(),
-                segments.get(expectedCurrentSegmentIdx).getLogSegmentSequenceNumber(),
-                readAheadEntryReader.getCurrentSegmentSequenceNumber());
-        assertNull("next segment reader should not be initialized since it is a closed log segment",
-                readAheadEntryReader.getNextSegmentReader());
-
-        readAheadEntryReader.onSegmentsUpdated(segments.subList(2, 3));
-        ensureOrderSchedulerEmpty(streamName);
-        expectIllegalStateException(readAheadEntryReader,
-                "inconsistent log segment found");
-
-        Utils.close(readAheadEntryReader);
-        dlm.close();
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadUtils.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadUtils.java
deleted file mode 100644
index 4358a8e..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReadUtils.java
+++ /dev/null
@@ -1,344 +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 com.twitter.distributedlog;
-
-import java.util.List;
-import java.util.concurrent.Executors;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.logsegment.LogSegmentFilter;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.junit.Assert.*;
-
-/**
- * Test {@link ReadUtils}
- */
-public class TestReadUtils extends TestDistributedLogBase {
-
-    static final Logger LOG = LoggerFactory.getLogger(TestReadUtils.class);
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    private Future<Optional<LogRecordWithDLSN>> getLogRecordNotLessThanTxId(
-            BKDistributedLogManager bkdlm, int logsegmentIdx, long transactionId) throws Exception {
-        List<LogSegmentMetadata> logSegments = bkdlm.getLogSegments();
-        return ReadUtils.getLogRecordNotLessThanTxId(
-                bkdlm.getStreamName(),
-                logSegments.get(logsegmentIdx),
-                transactionId,
-                Executors.newSingleThreadExecutor(),
-                bkdlm.getReaderEntryStore(),
-                10
-        );
-    }
-
-    private Future<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),
-                bkdlm.getReaderEntryStore(), dlsn
-        );
-    }
-
-    private Future<LogRecordWithDLSN> getLastUserRecord(BKDistributedLogManager bkdlm, int ledgerNo) throws Exception {
-        BKLogReadHandler readHandler = bkdlm.createReadHandler();
-        List<LogSegmentMetadata> ledgerList = FutureUtils.result(
-                readHandler.readLogSegmentsFromStore(
-                        LogSegmentMetadata.COMPARATOR,
-                        LogSegmentFilter.DEFAULT_FILTER,
-                        null)
-        ).getValue();
-        return ReadUtils.asyncReadLastRecord(
-                bkdlm.getStreamName(), ledgerList.get(ledgerNo), false, false, false, 2, 16, new AtomicInteger(0), Executors.newFixedThreadPool(1),
-                bkdlm.getReaderEntryStore()
-        );
-    }
-
-    @Test(timeout = 60000)
-    public void testForwardScanFirstRecord() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
-        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);
-        assertEquals("should be an exact match", dlsn, logrec.getDlsn());
-        bkdlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testForwardScanNotFirstRecord() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
-        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);
-        assertEquals("should be an exact match", dlsn, logrec.getDlsn());
-        bkdlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testForwardScanValidButNonExistentRecord() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
-        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);
-        assertEquals(new DLSN(1,1,0), logrec.getDlsn());
-        bkdlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testForwardScanForRecordAfterLedger() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
-        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);
-        assertEquals(null, logrec);
-        bkdlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testForwardScanForRecordBeforeLedger() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
-        long txid = 1;
-        txid += DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5 /* user recs */ , txid);
-        txid += DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5 /* user recs */ , txid);
-        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);
-        assertEquals(new DLSN(2,0,0), logrec.getDlsn());
-        bkdlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testForwardScanControlRecord() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
-        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);
-        assertEquals(new DLSN(1,5,0), logrec.getDlsn());
-        bkdlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLastRecordUserRecord() throws Exception {
-        String streamName = runtime.getMethodName();
-        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);
-        assertEquals(new DLSN(1,9,0), logrec.getDlsn());
-        bkdlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLastRecordControlRecord() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
-
-        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.close(out);
-
-        Future<LogRecordWithDLSN> futureLogrec = getLastUserRecord(bkdlm, 0);
-        LogRecordWithDLSN logrec = Await.result(futureLogrec);
-        assertEquals(new DLSN(1,2,0), logrec.getDlsn());
-        bkdlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLastRecordAllControlRecords() throws Exception {
-        String streamName = runtime.getMethodName();
-        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);
-        assertEquals(null, logrec);
-        bkdlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testGetEntriesToSearch() throws Exception {
-        assertTrue(ReadUtils.getEntriesToSearch(2L, 1L, 10).isEmpty());
-        assertEquals(Lists.newArrayList(1L),
-                ReadUtils.getEntriesToSearch(1L, 1L, 10));
-        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L),
-                ReadUtils.getEntriesToSearch(1L, 10L, 10));
-        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L),
-                ReadUtils.getEntriesToSearch(1L, 9L, 10));
-        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L),
-                ReadUtils.getEntriesToSearch(1L, 8L, 10));
-        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 11L),
-                ReadUtils.getEntriesToSearch(1L, 11L, 10));
-        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 12L),
-                ReadUtils.getEntriesToSearch(1L, 12L, 10));
-    }
-
-    @Test(timeout = 60000)
-    public void testGetEntriesToSearchByTxnId() throws Exception {
-        LogRecordWithDLSN firstRecord =
-                DLMTestUtil.getLogRecordWithDLSNInstance(new DLSN(1L, 0L, 0L), 999L);
-        LogRecordWithDLSN secondRecord =
-                DLMTestUtil.getLogRecordWithDLSNInstance(new DLSN(1L, 10L, 0L), 99L);
-        LogRecordWithDLSN thirdRecord =
-                DLMTestUtil.getLogRecordWithDLSNInstance(new DLSN(1L, 100L, 0L), 1099L);
-        // out-of-order sequence
-        assertTrue(ReadUtils.getEntriesToSearch(888L, firstRecord, secondRecord, 10).isEmpty());
-        // same transaction id
-        assertTrue(ReadUtils.getEntriesToSearch(888L, firstRecord, firstRecord, 10).isEmpty());
-        // small nways (nways = 2)
-        assertEquals(2, ReadUtils.getEntriesToSearch(888L, firstRecord, thirdRecord, 2).size());
-        // small nways with equal transaction id
-        assertEquals(3, ReadUtils.getEntriesToSearch(1099L, firstRecord, thirdRecord, 2).size());
-        LogRecordWithDLSN record1 =
-                DLMTestUtil.getLogRecordWithDLSNInstance(new DLSN(1L, 0L, 0L), 88L);
-        LogRecordWithDLSN record2 =
-                DLMTestUtil.getLogRecordWithDLSNInstance(new DLSN(1L, 12L, 0L), 888L);
-        LogRecordWithDLSN record3 =
-                DLMTestUtil.getLogRecordWithDLSNInstance(new DLSN(1L, 12L, 0L), 999L);
-        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 10L, 11L),
-                ReadUtils.getEntriesToSearch(888L, record1, record2, 10));
-        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 11L),
-                ReadUtils.getEntriesToSearch(888L, record1, record3, 10));
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordNotLessThanTxIdWithGreaterTxId() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
-        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 1, 1 /* txid */);
-
-        Optional<LogRecordWithDLSN> result =
-                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 999L));
-        assertFalse(result.isPresent());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordNotLessThanTxIdWithLessTxId() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
-        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 1, 999L /* txid */);
-
-        Optional<LogRecordWithDLSN> result =
-                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 99L));
-        assertTrue(result.isPresent());
-        assertEquals(999L, result.get().getTransactionId());
-        assertEquals(0L, result.get().getDlsn().getEntryId());
-        assertEquals(0L, result.get().getDlsn().getSlotId());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordNotLessThanTxIdOnSmallSegment() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
-        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 5, 1L /* txid */);
-
-        Optional<LogRecordWithDLSN> result =
-                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 3L));
-        assertTrue(result.isPresent());
-        assertEquals(3L, result.get().getTransactionId());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordNotLessThanTxIdOnLargeSegment() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
-        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 100, 1L /* txid */);
-
-        Optional<LogRecordWithDLSN> result =
-                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 9L));
-        assertTrue(result.isPresent());
-        assertEquals(9L, result.get().getTransactionId());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordGreaterThanTxIdOnLargeSegment() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
-        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 100, 1L /* txid */, 3L);
-
-        Optional<LogRecordWithDLSN> result =
-                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 23L));
-        assertTrue(result.isPresent());
-        assertEquals(25L, result.get().getTransactionId());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogRecordGreaterThanTxIdOnSameTxId() throws Exception {
-        String streamName = runtime.getMethodName();
-        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
-        AsyncLogWriter out = bkdlm.startAsyncLogSegmentNonPartitioned();
-        long txid = 1L;
-        for (int i = 0; i < 10; ++i) {
-            LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
-            Await.result(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));
-        }
-        for (int i = 0; i < 10; ++i) {
-            LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
-            Await.result(out.write(record));
-            txid += 1;
-        }
-        Utils.close(out);
-        Optional<LogRecordWithDLSN> result =
-                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, txidToSearch));
-        assertTrue(result.isPresent());
-        assertEquals(10L, result.get().getDlsn().getEntryId());
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReader.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReader.java
deleted file mode 100644
index 6040549..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestReader.java
+++ /dev/null
@@ -1,206 +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 com.twitter.distributedlog;
-
-import com.twitter.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;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.junit.Assert.*;
-
-/**
- * A Reader wraps reading next logic for testing.
- */
-public class TestReader implements FutureEventListener<LogRecordWithDLSN> {
-
-    static final Logger LOG = LoggerFactory.getLogger(TestReader.class);
-
-    final String readerName;
-    final DistributedLogManager dlm;
-    AsyncLogReader reader;
-    final DLSN startDLSN;
-    DLSN nextDLSN;
-    final boolean simulateErrors;
-    int delayMs;
-    final ScheduledExecutorService executorService;
-
-    // Latches
-
-    // Latch on notifying reader is ready to read
-    final CountDownLatch readyLatch;
-    // Latch no notifying reads are completed or errors are encountered
-    final CountDownLatch completionLatch;
-    // Latch no notifying reads are done.
-    final CountDownLatch countLatch;
-
-    // States
-    final AtomicBoolean errorsFound;
-    final AtomicInteger readCount;
-    final AtomicInteger positionReaderCount;
-
-    public TestReader(String name,
-                      DistributedLogManager dlm,
-                      DLSN startDLSN,
-                      boolean simulateErrors,
-                      int delayMs,
-                      CountDownLatch readyLatch,
-                      CountDownLatch countLatch,
-                      CountDownLatch completionLatch) {
-        this.readerName = name;
-        this.dlm = dlm;
-        this.startDLSN = startDLSN;
-        this.simulateErrors = simulateErrors;
-        this.delayMs = delayMs;
-        this.readyLatch = readyLatch;
-        this.countLatch = countLatch;
-        this.completionLatch = completionLatch;
-        // States
-        this.errorsFound = new AtomicBoolean(false);
-        this.readCount = new AtomicInteger(0);
-        this.positionReaderCount = new AtomicInteger(0);
-        // Executors
-        this.executorService = Executors.newSingleThreadScheduledExecutor();
-    }
-
-    public AtomicInteger getNumReaderPositions() {
-        return this.positionReaderCount;
-    }
-
-    public AtomicInteger getNumReads() {
-        return this.readCount;
-    }
-
-    public boolean areErrorsFound() {
-        return errorsFound.get();
-    }
-
-    private int nextDelayMs() {
-        int newDelayMs = Math.min(delayMs * 2, 500);
-        if (0 == delayMs) {
-            newDelayMs = 10;
-        }
-        delayMs = newDelayMs;
-        return delayMs;
-    }
-
-    private void positionReader(final DLSN dlsn) {
-        positionReaderCount.incrementAndGet();
-        Runnable runnable = new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    AsyncLogReader reader = dlm.getAsyncLogReader(dlsn);
-                    if (simulateErrors) {
-                        ((BKAsyncLogReader) reader).simulateErrors();
-                    }
-                    nextDLSN = dlsn;
-                    LOG.info("Positioned reader {} at {}", readerName, dlsn);
-                    if (null != TestReader.this.reader) {
-                        Utils.close(TestReader.this.reader);
-                    }
-                    TestReader.this.reader = reader;
-                    readNext();
-                    readyLatch.countDown();
-                } catch (IOException exc) {
-                    int nextMs = nextDelayMs();
-                    LOG.info("Encountered exception {} on opening reader {} at {}, retrying in {} ms",
-                            new Object[] { exc, readerName, dlsn, nextMs });
-                    positionReader(dlsn);
-                }
-            }
-        };
-        executorService.schedule(runnable, delayMs, TimeUnit.MILLISECONDS);
-    }
-
-    private void readNext() {
-        Future<LogRecordWithDLSN> record = reader.readNext();
-        record.addEventListener(this);
-    }
-
-    @Override
-    public void onSuccess(LogRecordWithDLSN value) {
-        try {
-            assertTrue(value.getDlsn().compareTo(nextDLSN) >= 0);
-            LOG.info("Received record {} from log {} for reader {}",
-                    new Object[] { value.getDlsn(), dlm.getStreamName(), readerName });
-            assertFalse(value.isControl());
-            assertEquals(0, value.getDlsn().getSlotId());
-            DLMTestUtil.verifyLargeLogRecord(value);
-        } catch (Exception exc) {
-            LOG.error("Exception encountered when verifying received log record {} for reader {} :",
-                    new Object[] { value.getDlsn(), exc, readerName });
-            errorsFound.set(true);
-            completionLatch.countDown();
-            return;
-        }
-        readCount.incrementAndGet();
-        countLatch.countDown();
-        if (countLatch.getCount() <= 0) {
-            LOG.info("Reader {} is completed", readerName);
-            closeReader();
-            completionLatch.countDown();
-        } else {
-            LOG.info("Reader {} : read count becomes {}, latch = {}",
-                    new Object[] { readerName, readCount.get(), countLatch.getCount() });
-            nextDLSN = value.getDlsn().getNextDLSN();
-            readNext();
-        }
-    }
-
-    @Override
-    public void onFailure(Throwable cause) {
-        LOG.error("{} encountered exception on reading next record : ", readerName, cause);
-        closeReader();
-        nextDelayMs();
-        positionReader(nextDLSN);
-    }
-
-    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;
-                }
-            });
-        }
-    }
-
-    public void start() {
-        positionReader(startDLSN);
-    }
-
-    public void stop() {
-        closeReader();
-        executorService.shutdown();
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestRollLogSegments.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestRollLogSegments.java
deleted file mode 100644
index b183b84..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestRollLogSegments.java
+++ /dev/null
@@ -1,429 +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 com.twitter.distributedlog;
-
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.CountDownLatch;
-
-import com.twitter.distributedlog.feature.CoreFeatureKeys;
-import com.twitter.distributedlog.impl.logsegment.BKLogSegmentEntryReader;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.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;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.twitter.distributedlog.annotations.DistributedLogAnnotations.FlakyTest;
-import com.twitter.util.Await;
-import com.twitter.util.FutureEventListener;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
-
-public class TestRollLogSegments extends TestDistributedLogBase {
-    static final Logger logger = LoggerFactory.getLogger(TestRollLogSegments.class);
-
-    private static void ensureOnlyOneInprogressLogSegments(List<LogSegmentMetadata> segments) throws Exception {
-        int numInprogress = 0;
-        for (LogSegmentMetadata segment : segments) {
-            if (segment.isInProgress()) {
-                ++numInprogress;
-            }
-        }
-        assertEquals(1, numInprogress);
-    }
-
-    @Test(timeout = 60000)
-    public void testDisableRollingLogSegments() throws Exception {
-        String name = "distrlog-disable-rolling-log-segments";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setLogSegmentRollingIntervalMinutes(0);
-        confLocal.setMaxLogSegmentBytes(40);
-
-        int numEntries = 100;
-        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-
-        SettableFeature disableLogSegmentRolling =
-                (SettableFeature) dlm.getFeatureProvider()
-                        .getFeature(CoreFeatureKeys.DISABLE_LOGSEGMENT_ROLLING.name().toLowerCase());
-        disableLogSegmentRolling.set(true);
-
-        final CountDownLatch latch = new CountDownLatch(numEntries);
-
-        // send requests in parallel
-        for (int i = 1; i <= numEntries; i++) {
-            final int entryId = i;
-            writer.write(DLMTestUtil.getLogRecordInstance(entryId)).addEventListener(new FutureEventListener<DLSN>() {
-
-                @Override
-                public void onSuccess(DLSN value) {
-                    logger.info("Completed entry {} : {}.", entryId, value);
-                    latch.countDown();
-                }
-
-                @Override
-                public void onFailure(Throwable cause) {
-                    // nope
-                }
-            });
-        }
-        latch.await();
-
-        // make sure all ensure blocks were executed
-        writer.closeAndComplete();
-
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(1, segments.size());
-
-        dlm.close();
-    }
-
-    @Test(timeout = 600000)
-    public void testLastDLSNInRollingLogSegments() throws Exception {
-        final Map<Long, DLSN> lastDLSNs = new HashMap<Long, DLSN>();
-        String name = "distrlog-lastdlsn-in-rolling-log-segments";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setLogSegmentRollingIntervalMinutes(0);
-        confLocal.setMaxLogSegmentBytes(40);
-
-        int numEntries = 100;
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-
-        final CountDownLatch latch = new CountDownLatch(numEntries);
-
-        // 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>() {
-
-                @Override
-                public void onSuccess(DLSN value) {
-                    logger.info("Completed entry {} : {}.", entryId, value);
-                    synchronized (lastDLSNs) {
-                        DLSN lastDLSN = lastDLSNs.get(value.getLogSegmentSequenceNo());
-                        if (null == lastDLSN || lastDLSN.compareTo(value) < 0) {
-                            lastDLSNs.put(value.getLogSegmentSequenceNo(), value);
-                        }
-                    }
-                    latch.countDown();
-                }
-
-                @Override
-                public void onFailure(Throwable cause) {
-
-                }
-            });
-            if (i == 1) {
-                // wait for first log segment created
-                FutureUtils.result(writeFuture);
-            }
-        }
-        latch.await();
-
-        // make sure all ensure blocks were executed.
-        writer.closeAndComplete();
-
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        logger.info("lastDLSNs after writes {} {}", lastDLSNs.size(), lastDLSNs);
-        logger.info("segments after writes {} {}", segments.size(), segments);
-        assertTrue(segments.size() >= 2);
-        assertTrue(lastDLSNs.size() >= 2);
-        assertEquals(lastDLSNs.size(), segments.size());
-        for (LogSegmentMetadata segment : segments) {
-            DLSN dlsnInMetadata = segment.getLastDLSN();
-            DLSN dlsnSeen = lastDLSNs.get(segment.getLogSegmentSequenceNumber());
-            assertNotNull(dlsnInMetadata);
-            assertNotNull(dlsnSeen);
-            if (dlsnInMetadata.compareTo(dlsnSeen) != 0) {
-                logger.error("Last dlsn recorded in log segment {} is different from the one already seen {}.",
-                             dlsnInMetadata, dlsnSeen);
-            }
-            assertEquals(0, dlsnInMetadata.compareTo(dlsnSeen));
-        }
-
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testUnableToRollLogSegments() throws Exception {
-        String name = "distrlog-unable-to-roll-log-segments";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setLogSegmentRollingIntervalMinutes(0);
-        confLocal.setMaxLogSegmentBytes(1);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-
-        long txId = 1L;
-
-        // Create Log Segments
-        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId)));
-
-        FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_StartLogSegmentBeforeLedgerCreate,
-                FailpointUtils.FailPointActions.FailPointAction_Throw);
-
-        try {
-            // If we couldn't open new log segment, we should keep using the old one
-            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>() {
-                    @Override
-                    public void onSuccess(DLSN value) {
-                        logger.info("Completed entry : {}.", value);
-                        latch.countDown();
-                    }
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        logger.error("Failed to write entries : ", cause);
-                    }
-                });
-            }
-
-            latch.await();
-
-            writer.close();
-
-            List<LogSegmentMetadata> segments = dlm.getLogSegments();
-            logger.info("LogSegments: {}", segments);
-
-            assertEquals(1, segments.size());
-
-            long expectedTxID = 1L;
-            LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
-            LogRecordWithDLSN record = reader.readNext(false);
-            while (null != record) {
-                DLMTestUtil.verifyLogRecord(record);
-                assertEquals(expectedTxID++, record.getTransactionId());
-                assertEquals(record.getTransactionId() - 1, record.getSequenceId());
-
-                record = reader.readNext(false);
-            }
-
-            assertEquals(12L, expectedTxID);
-
-            reader.close();
-
-            dlm.close();
-        } finally {
-            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_StartLogSegmentBeforeLedgerCreate);
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testRollingLogSegments() throws Exception {
-        logger.info("start testRollingLogSegments");
-        String name = "distrlog-rolling-logsegments-hightraffic";
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setLogSegmentRollingIntervalMinutes(0);
-        confLocal.setMaxLogSegmentBytes(1);
-        confLocal.setLogSegmentRollingConcurrency(Integer.MAX_VALUE);
-
-        int numLogSegments = 10;
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
-
-        final CountDownLatch latch = new CountDownLatch(numLogSegments);
-        long startTime = System.currentTimeMillis();
-        // 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>() {
-                @Override
-                public void onSuccess(DLSN value) {
-                    logger.info("Completed entry {} : {}.", entryId, value);
-                    latch.countDown();
-                }
-                @Override
-                public void onFailure(Throwable cause) {
-                    logger.error("Failed to write entries : {}", cause);
-                }
-            });
-            if (i == 1) {
-                // wait for first log segment created
-                FutureUtils.result(writeFuture);
-            }
-        }
-        latch.await();
-
-        logger.info("Took {} ms to completed all requests.", System.currentTimeMillis() - startTime);
-
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        logger.info("LogSegments : {}", segments);
-
-        assertTrue(segments.size() >= 2);
-        ensureOnlyOneInprogressLogSegments(segments);
-
-        int numSegmentsAfterAsyncWrites = segments.size();
-
-        // 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)));
-            logger.info("Completed entry {} : {}", numLogSegments + i, newDLSN);
-        }
-
-        segments = dlm.getLogSegments();
-        logger.info("LogSegments : {}", segments);
-
-        assertEquals(numSegmentsAfterAsyncWrites + numLogSegments / 2, segments.size());
-        ensureOnlyOneInprogressLogSegments(segments);
-
-        writer.close();
-        dlm.close();
-    }
-
-    private void checkAndWaitWriterReaderPosition(BKLogSegmentWriter writer, long expectedWriterPosition,
-                                                  BKAsyncLogReader reader, long expectedReaderPosition,
-                                                  LedgerHandle inspector, long expectedLac) throws Exception {
-        while (getLedgerHandle(writer).getLastAddConfirmed() < expectedWriterPosition) {
-            Thread.sleep(1000);
-        }
-        assertEquals(expectedWriterPosition, getLedgerHandle(writer).getLastAddConfirmed());
-        assertEquals(expectedLac, inspector.readLastConfirmed());
-        EntryPosition readPosition = reader.getReadAheadReader().getNextEntryPosition();
-        logger.info("ReadAhead moved read position {} : ", readPosition);
-        while (readPosition.getEntryId() < expectedReaderPosition) {
-            Thread.sleep(1000);
-            readPosition = reader.getReadAheadReader().getNextEntryPosition();
-            logger.info("ReadAhead moved read position {} : ", readPosition);
-        }
-        assertEquals(expectedReaderPosition, readPosition.getEntryId());
-    }
-
-    @FlakyTest
-    @Test(timeout = 60000)
-    @SuppressWarnings("deprecation")
-    public void testCaughtUpReaderOnLogSegmentRolling() throws Exception {
-        String name = "distrlog-caughtup-reader-on-logsegment-rolling";
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setOutputBufferSize(4 * 1024 * 1024);
-        confLocal.setTraceReadAheadMetadataChanges(true);
-        confLocal.setEnsembleSize(1);
-        confLocal.setWriteQuorumSize(1);
-        confLocal.setAckQuorumSize(1);
-        confLocal.setReadLACLongPollTimeout(99999999);
-        confLocal.setReaderIdleWarnThresholdMillis(2 * 99999999 + 1);
-        confLocal.setBKClientReadTimeout(99999999 + 1);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        BKSyncLogWriter writer = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-
-        // 1) writer added 5 entries.
-        final int numEntries = 5;
-        for (int i = 1; i <= numEntries; i++) {
-            writer.write(DLMTestUtil.getLogRecordInstance(i));
-            writer.setReadyToFlush();
-            writer.flushAndSync();
-        }
-
-        BKDistributedLogManager readDLM = (BKDistributedLogManager) createNewDLM(confLocal, name);
-        final BKAsyncLogReader reader = (BKAsyncLogReader) readDLM.getAsyncLogReader(DLSN.InitialDLSN);
-
-        // 2) reader should be able to read 5 entries.
-        for (long i = 1; i <= numEntries; i++) {
-            LogRecordWithDLSN record = Await.result(reader.readNext());
-            DLMTestUtil.verifyLogRecord(record);
-            assertEquals(i, record.getTransactionId());
-            assertEquals(record.getTransactionId() - 1, record.getSequenceId());
-        }
-
-        BKLogSegmentWriter perStreamWriter = writer.segmentWriter;
-        BookKeeperClient bkc = DLMTestUtil.getBookKeeperClient(readDLM);
-        LedgerHandle readLh = bkc.get().openLedgerNoRecovery(getLedgerHandle(perStreamWriter).getId(),
-                BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
-
-        // Writer moved to lac = 9, while reader knows lac = 8 and moving to wait on 9
-        checkAndWaitWriterReaderPosition(perStreamWriter, 9, reader, 9, readLh, 8);
-
-        // write 6th record
-        writer.write(DLMTestUtil.getLogRecordInstance(numEntries + 1));
-        writer.setReadyToFlush();
-        // Writer moved to lac = 10, while reader knows lac = 9 and moving to wait on 10
-        checkAndWaitWriterReaderPosition(perStreamWriter, 10, reader, 10, readLh, 9);
-
-        // write records without commit to simulate similar failure cases
-        writer.write(DLMTestUtil.getLogRecordInstance(numEntries + 2));
-        writer.setReadyToFlush();
-        // Writer moved to lac = 11, while reader knows lac = 10 and moving to wait on 11
-        checkAndWaitWriterReaderPosition(perStreamWriter, 11, reader, 11, readLh, 10);
-
-        while (true) {
-            BKLogSegmentEntryReader entryReader =
-                    (BKLogSegmentEntryReader) reader.getReadAheadReader().getCurrentSegmentReader().getEntryReader();
-            if (null != entryReader && null != entryReader.getOutstandingLongPoll()) {
-                break;
-            }
-            Thread.sleep(1000);
-        }
-        logger.info("Waiting for long poll getting interrupted with metadata changed");
-
-        // simulate a recovery without closing ledger causing recording wrong last dlsn
-        BKLogWriteHandler writeHandler = writer.getCachedWriteHandler();
-        writeHandler.completeAndCloseLogSegment(
-                writeHandler.inprogressZNodeName(perStreamWriter.getLogSegmentId(), perStreamWriter.getStartTxId(), perStreamWriter.getLogSegmentSequenceNumber()),
-                perStreamWriter.getLogSegmentSequenceNumber(),
-                perStreamWriter.getLogSegmentId(),
-                perStreamWriter.getStartTxId(), perStreamWriter.getLastTxId(),
-                perStreamWriter.getPositionWithinLogSegment() - 1,
-                9,
-                0);
-
-        BKSyncLogWriter anotherWriter = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
-        anotherWriter.write(DLMTestUtil.getLogRecordInstance(numEntries + 3));
-        anotherWriter.setReadyToFlush();
-        anotherWriter.flushAndSync();
-        anotherWriter.closeAndComplete();
-
-        for (long i = numEntries + 1; i <= numEntries + 3; i++) {
-            LogRecordWithDLSN record = Await.result(reader.readNext());
-            DLMTestUtil.verifyLogRecord(record);
-            assertEquals(i, record.getTransactionId());
-        }
-
-        Utils.close(reader);
-        readDLM.close();
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestSequenceID.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestSequenceID.java
deleted file mode 100644
index d306221..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestSequenceID.java
+++ /dev/null
@@ -1,254 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion;
-import com.twitter.util.Await;
-import com.twitter.util.FutureEventListener;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.List;
-import java.util.concurrent.LinkedBlockingQueue;
-
-import static org.junit.Assert.*;
-
-/**
- * Test Cases related to sequence ids.
- */
-public class TestSequenceID extends TestDistributedLogBase {
-
-    static final Logger logger = LoggerFactory.getLogger(TestSequenceID.class);
-
-    @Test(timeout = 60000)
-    public void testCompleteV4LogSegmentAsV4() throws Exception {
-        completeSingleInprogressSegment(LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value,
-                                        LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value);
-    }
-
-    @Test(timeout = 60000)
-    public void testCompleteV4LogSegmentAsV5() throws Exception {
-        completeSingleInprogressSegment(LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value,
-                                        LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value);
-    }
-
-    @Test(timeout = 60000)
-    public void testCompleteV5LogSegmentAsV4() throws Exception {
-        completeSingleInprogressSegment(LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value,
-                                        LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value);
-    }
-
-    @Test(timeout = 60000)
-    public void testCompleteV5LogSegmentAsV5() throws Exception {
-        completeSingleInprogressSegment(LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value,
-                                        LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value);
-    }
-
-    private void completeSingleInprogressSegment(int writeVersion, int completeVersion) throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setDLLedgerMetadataLayoutVersion(writeVersion);
-
-        String name = "distrlog-complete-single-inprogress-segment-versions-write-"
-                + writeVersion + "-complete-" + completeVersion;
-
-        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocal, name);
-        BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(0L)));
-
-        dlm.close();
-
-        DistributedLogConfiguration confLocal2 = new DistributedLogConfiguration();
-        confLocal2.addConfiguration(confLocal);
-        confLocal2.setDLLedgerMetadataLayoutVersion(completeVersion);
-
-        BKDistributedLogManager dlm2 = (BKDistributedLogManager) createNewDLM(confLocal2, name);
-        dlm2.startAsyncLogSegmentNonPartitioned();
-
-        List<LogSegmentMetadata> segments = dlm2.getLogSegments();
-        assertEquals(1, segments.size());
-
-        if (LogSegmentMetadata.supportsSequenceId(writeVersion)) {
-            if (LogSegmentMetadata.supportsSequenceId(completeVersion)) {
-                // the inprogress log segment is written in v5 and complete log segment in v5,
-                // then it support monotonic sequence id
-                assertEquals(0L, segments.get(0).getStartSequenceId());
-            } else {
-                // the inprogress log segment is written in v5 and complete log segment in v4,
-                // then it doesn't support monotonic sequence id
-                assertTrue(segments.get(0).getStartSequenceId() < 0);
-            }
-        } else {
-            // if the inprogress log segment is created prior to v5, it won't support monotonic sequence id
-            assertTrue(segments.get(0).getStartSequenceId() < 0);
-        }
-
-        dlm2.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testSequenceID() throws Exception {
-        DistributedLogConfiguration confLocalv4 = new DistributedLogConfiguration();
-        confLocalv4.addConfiguration(conf);
-        confLocalv4.setImmediateFlushEnabled(true);
-        confLocalv4.setOutputBufferSize(0);
-        confLocalv4.setDLLedgerMetadataLayoutVersion(LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value);
-
-        String name = "distrlog-sequence-id";
-
-        BKDistributedLogManager readDLM = (BKDistributedLogManager) createNewDLM(conf, name);
-        AsyncLogReader reader = null;
-        final LinkedBlockingQueue<LogRecordWithDLSN> readRecords =
-                new LinkedBlockingQueue<LogRecordWithDLSN>();
-
-        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocalv4, name);
-
-        long txId = 0L;
-
-        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++)));
-
-                if (null == reader) {
-                    reader = readDLM.getAsyncLogReader(DLSN.InitialDLSN);
-                    final AsyncLogReader r = reader;
-                    reader.readNext().addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
-                        @Override
-                        public void onSuccess(LogRecordWithDLSN record) {
-                            readRecords.add(record);
-                            r.readNext().addEventListener(this);
-                        }
-
-                        @Override
-                        public void onFailure(Throwable cause) {
-                            logger.error("Encountered exception on reading next : ", cause);
-                        }
-                    });
-                }
-            }
-            writer.closeAndComplete();
-        }
-
-        BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
-
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(4, segments.size());
-        for (int i = 0; i < 3; i++) {
-            assertFalse(segments.get(i).isInProgress());
-            assertTrue(segments.get(i).getStartSequenceId() < 0);
-        }
-        assertTrue(segments.get(3).isInProgress());
-        assertTrue(segments.get(3).getStartSequenceId() < 0);
-
-        dlm.close();
-
-        // simulate upgrading from v4 -> v5
-
-        DistributedLogConfiguration confLocalv5 = new DistributedLogConfiguration();
-        confLocalv5.addConfiguration(conf);
-        confLocalv5.setImmediateFlushEnabled(true);
-        confLocalv5.setOutputBufferSize(0);
-        confLocalv5.setDLLedgerMetadataLayoutVersion(LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value);
-
-        BKDistributedLogManager dlmv5 = (BKDistributedLogManager) createNewDLM(confLocalv5, name);
-        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++)));
-            }
-            writerv5.closeAndComplete();
-        }
-        BKAsyncLogWriter writerv5 = dlmv5.startAsyncLogSegmentNonPartitioned();
-        Await.result(writerv5.write(DLMTestUtil.getLogRecordInstance(txId++)));
-
-        List<LogSegmentMetadata> segmentsv5 = dlmv5.getLogSegments();
-        assertEquals(8, segmentsv5.size());
-
-        assertFalse(segmentsv5.get(3).isInProgress());
-        assertTrue(segmentsv5.get(3).getStartSequenceId() < 0);
-
-        long startSequenceId = 0L;
-        for (int i = 4; i < 7; i++) {
-            assertFalse(segmentsv5.get(i).isInProgress());
-            assertEquals(startSequenceId, segmentsv5.get(i).getStartSequenceId());
-            startSequenceId += 2L;
-        }
-
-        assertTrue(segmentsv5.get(7).isInProgress());
-        assertEquals(startSequenceId, segmentsv5.get(7).getStartSequenceId());
-
-        dlmv5.close();
-
-        // rollback from v5 to v4
-
-        BKDistributedLogManager dlmv4 = (BKDistributedLogManager) createNewDLM(confLocalv4, name);
-        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++)));
-            }
-            writerv4.closeAndComplete();
-        }
-
-        List<LogSegmentMetadata> segmentsv4 = dlmv4.getLogSegments();
-        assertEquals(11, segmentsv4.size());
-
-        for(int i = 7; i < 11; i++) {
-            assertFalse(segmentsv4.get(i).isInProgress());
-            assertTrue(segmentsv4.get(i).getStartSequenceId() < 0);
-        }
-
-        dlmv4.close();
-
-        // wait until readers read all records
-        while (readRecords.size() < txId) {
-            Thread.sleep(100);
-        }
-
-        assertEquals(txId, readRecords.size());
-        long sequenceId = Long.MIN_VALUE;
-        for (LogRecordWithDLSN record : readRecords) {
-            if (record.getDlsn().getLogSegmentSequenceNo() <= 4) {
-                assertTrue(record.getSequenceId() < 0);
-                assertTrue(record.getSequenceId() > sequenceId);
-                sequenceId = record.getSequenceId();
-            } else if (record.getDlsn().getLogSegmentSequenceNo() <= 7) {
-                if (sequenceId < 0L) {
-                    sequenceId = 0L;
-                }
-                assertEquals(sequenceId, record.getSequenceId());
-                ++sequenceId;
-            } else if (record.getDlsn().getLogSegmentSequenceNo() >= 9) {
-                if (sequenceId > 0) {
-                    sequenceId = Long.MIN_VALUE;
-                }
-                assertTrue(record.getSequenceId() < 0);
-                assertTrue(record.getSequenceId() > sequenceId);
-                sequenceId = record.getSequenceId();
-            }
-        }
-
-        readDLM.close();
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestTruncate.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestTruncate.java
deleted file mode 100644
index 011fc70..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestTruncate.java
+++ /dev/null
@@ -1,341 +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 com.twitter.distributedlog;
-
-import java.net.URI;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Utils;
-import org.apache.commons.lang3.tuple.ImmutablePair;
-import org.apache.commons.lang3.tuple.Pair;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.twitter.distributedlog.LogSegmentMetadata.TruncationStatus;
-import com.twitter.util.Await;
-
-import static org.junit.Assert.*;
-
-public class TestTruncate extends TestDistributedLogBase {
-    static final Logger LOG = LoggerFactory.getLogger(TestTruncate.class);
-
-    protected static DistributedLogConfiguration conf =
-            new DistributedLogConfiguration()
-                    .setLockTimeout(10)
-                    .setOutputBufferSize(0)
-                    .setPeriodicFlushFrequencyMilliSeconds(10)
-                    .setSchedulerShutdownTimeoutMs(0)
-                    .setDLLedgerMetadataLayoutVersion(LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value);
-
-    static void updateCompletionTime(ZooKeeperClient zkc,
-                                     LogSegmentMetadata l, long completionTime) throws Exception {
-        LogSegmentMetadata newSegment = l.mutator().setCompletionTime(completionTime).build();
-        DLMTestUtil.updateSegmentMetadata(zkc, newSegment);
-    }
-
-    static void setTruncationStatus(ZooKeeperClient zkc,
-                                    LogSegmentMetadata l,
-                                    TruncationStatus status) throws Exception {
-        LogSegmentMetadata newSegment =
-                l.mutator().setTruncationStatus(status).build();
-        DLMTestUtil.updateSegmentMetadata(zkc, newSegment);
-    }
-
-    @Test(timeout = 60000)
-    public void testPurgeLogs() throws Exception {
-        String name = "distrlog-purge-logs";
-        URI uri = createDLMURI("/" + name);
-
-        populateData(new HashMap<Long, DLSN>(), conf, name, 10, 10, false);
-
-        DistributedLogManager distributedLogManager = createNewDLM(conf, name);
-
-        List<LogSegmentMetadata> segments = distributedLogManager.getLogSegments();
-        LOG.info("Segments before modifying completion time : {}", segments);
-
-        ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder(conf)
-                .uri(uri)
-                .build();
-
-        // Update completion time of first 5 segments
-        long newTimeMs = System.currentTimeMillis() - 60*60*1000*2;
-        for (int i = 0; i < 5; i++) {
-            LogSegmentMetadata segment = segments.get(i);
-            updateCompletionTime(zkc, segment, newTimeMs + i);
-        }
-        zkc.close();
-
-        segments = distributedLogManager.getLogSegments();
-        LOG.info("Segments after modifying completion time : {}", segments);
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setRetentionPeriodHours(1);
-        confLocal.setExplicitTruncationByApplication(false);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-        long txid = 1 + 10 * 10;
-        for (int j = 1; j <= 10; j++) {
-            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
-        }
-
-        // to make sure the truncation task is executed
-        DLSN lastDLSN = Await.result(dlm.getLastDLSNAsync());
-        LOG.info("Get last dlsn of stream {} : {}", name, lastDLSN);
-
-        assertEquals(6, distributedLogManager.getLogSegments().size());
-
-        Utils.close(writer);
-        dlm.close();
-
-        distributedLogManager.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testTruncation() throws Exception {
-        String name = "distrlog-truncation";
-
-        long txid = 1;
-        Map<Long, DLSN> txid2DLSN = new HashMap<Long, DLSN>();
-        Pair<DistributedLogManager, AsyncLogWriter> pair =
-                populateData(txid2DLSN, conf, name, 4, 10, true);
-
-        Thread.sleep(1000);
-
-        // delete invalid dlsn
-        assertFalse(Await.result(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)));
-            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)));
-        verifyEntries(name, 1, 41, 10);
-
-        Utils.close(pair.getRight());
-        pair.getLeft().close();
-    }
-
-    @Test(timeout = 60000)
-    public void testExplicitTruncation() throws Exception {
-        String name = "distrlog-truncation-explicit";
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setExplicitTruncationByApplication(true);
-
-        Map<Long, DLSN> txid2DLSN = new HashMap<Long, DLSN>();
-        Pair<DistributedLogManager, AsyncLogWriter> pair =
-                populateData(txid2DLSN, confLocal, name, 4, 10, true);
-
-        Thread.sleep(1000);
-
-        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)));
-            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)));
-        verifyEntries(name, 1, 41, 10);
-
-        Utils.close(pair.getRight());
-        pair.getLeft().close();
-
-        // Try force truncation
-        BKDistributedLogManager dlm = (BKDistributedLogManager)createNewDLM(confLocal, name);
-        BKLogWriteHandler handler = dlm.createWriteHandler(true);
-        FutureUtils.result(handler.purgeLogSegmentsOlderThanTxnId(Integer.MAX_VALUE));
-
-        verifyEntries(name, 1, 41, 10);
-    }
-
-    @Test(timeout = 60000)
-    public void testOnlyPurgeSegmentsBeforeNoneFullyTruncatedSegment() throws Exception {
-        String name = "distrlog-only-purge-segments-before-none-fully-truncated-segment";
-        URI uri = createDLMURI("/" + name);
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setExplicitTruncationByApplication(true);
-
-        // populate data
-        populateData(new HashMap<Long, DLSN>(), confLocal, name, 4, 10, false);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        LOG.info("Segments before modifying segment status : {}", segments);
-
-        ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder(conf)
-                .uri(uri)
-                .build();
-        setTruncationStatus(zkc, segments.get(0), TruncationStatus.PARTIALLY_TRUNCATED);
-        for (int i = 1; i < 4; i++) {
-            LogSegmentMetadata segment = segments.get(i);
-            setTruncationStatus(zkc, segment, TruncationStatus.TRUNCATED);
-        }
-        List<LogSegmentMetadata> segmentsAfterTruncated = dlm.getLogSegments();
-
-        dlm.purgeLogsOlderThan(999999);
-        List<LogSegmentMetadata> newSegments = dlm.getLogSegments();
-        LOG.info("Segments after purge segments older than 999999 : {}", newSegments);
-        assertArrayEquals(segmentsAfterTruncated.toArray(new LogSegmentMetadata[segmentsAfterTruncated.size()]),
-                          newSegments.toArray(new LogSegmentMetadata[newSegments.size()]));
-
-        dlm.close();
-
-        // Update completion time of all 4 segments
-        long newTimeMs = System.currentTimeMillis() - 60 * 60 * 1000 * 10;
-        for (int i = 0; i < 4; i++) {
-            LogSegmentMetadata segment = newSegments.get(i);
-            updateCompletionTime(zkc, segment, newTimeMs + i);
-        }
-
-        DistributedLogConfiguration newConf = new DistributedLogConfiguration();
-        newConf.addConfiguration(confLocal);
-        newConf.setRetentionPeriodHours(1);
-
-        DistributedLogManager newDLM = createNewDLM(newConf, name);
-        AsyncLogWriter newWriter = newDLM.startAsyncLogSegmentNonPartitioned();
-        long txid = 1 + 4 * 10;
-        for (int j = 1; j <= 10; j++) {
-            Await.result(newWriter.write(DLMTestUtil.getLogRecordInstance(txid++)));
-        }
-
-        // to make sure the truncation task is executed
-        DLSN lastDLSN = Await.result(newDLM.getLastDLSNAsync());
-        LOG.info("Get last dlsn of stream {} : {}", name, lastDLSN);
-
-        assertEquals(5, newDLM.getLogSegments().size());
-
-        Utils.close(newWriter);
-        newDLM.close();
-
-        zkc.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testPartiallyTruncateTruncatedSegments() throws Exception {
-        String name = "distrlog-partially-truncate-truncated-segments";
-        URI uri = createDLMURI("/" + name);
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setExplicitTruncationByApplication(true);
-
-        // populate
-        Map<Long, DLSN> dlsnMap = new HashMap<Long, DLSN>();
-        populateData(dlsnMap, confLocal, name, 4, 10, false);
-
-        DistributedLogManager dlm = createNewDLM(confLocal, name);
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        LOG.info("Segments before modifying segment status : {}", segments);
-
-        ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder(conf)
-                .uri(uri)
-                .build();
-        for (int i = 0; i < 4; i++) {
-            LogSegmentMetadata segment = segments.get(i);
-            setTruncationStatus(zkc, segment, TruncationStatus.TRUNCATED);
-        }
-
-        List<LogSegmentMetadata> newSegments = dlm.getLogSegments();
-        LOG.info("Segments after changing truncation status : {}", newSegments);
-
-        dlm.close();
-
-        DistributedLogManager newDLM = createNewDLM(confLocal, name);
-        AsyncLogWriter newWriter = newDLM.startAsyncLogSegmentNonPartitioned();
-        Await.result(newWriter.truncate(dlsnMap.get(15L)));
-
-        List<LogSegmentMetadata> newSegments2 = newDLM.getLogSegments();
-        assertArrayEquals(newSegments.toArray(new LogSegmentMetadata[4]),
-                          newSegments2.toArray(new LogSegmentMetadata[4]));
-
-        Utils.close(newWriter);
-        newDLM.close();
-        zkc.close();
-    }
-
-    private Pair<DistributedLogManager, AsyncLogWriter> populateData(
-            Map<Long, DLSN> txid2DLSN, DistributedLogConfiguration confLocal,
-            String name, int numLogSegments, int numEntriesPerLogSegment,
-            boolean createInprogressLogSegment) throws Exception {
-        long txid = 1;
-        for (long i = 1; i <= numLogSegments; i++) {
-            LOG.info("Writing Log Segment {}.", i);
-            DistributedLogManager dlm = createNewDLM(confLocal, name);
-            AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-            for (int j = 1; j <= numEntriesPerLogSegment; j++) {
-                long curTxId = txid++;
-                DLSN dlsn = Await.result(writer.write(DLMTestUtil.getLogRecordInstance(curTxId)));
-                txid2DLSN.put(curTxId, dlsn);
-            }
-            Utils.close(writer);
-            dlm.close();
-        }
-
-        if (createInprogressLogSegment) {
-            DistributedLogManager dlm = createNewDLM(confLocal, name);
-            AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
-            for (int j = 1; j <= 10; j++) {
-                long curTxId = txid++;
-                DLSN dlsn = Await.result(writer.write(DLMTestUtil.getLogRecordInstance(curTxId)));
-                txid2DLSN.put(curTxId, dlsn);
-            }
-            return new ImmutablePair<DistributedLogManager, AsyncLogWriter>(dlm, writer);
-        } else {
-            return null;
-        }
-    }
-
-    private void verifyEntries(String name, long readFromTxId, long startTxId, int numEntries) throws Exception {
-        DistributedLogManager dlm = createNewDLM(conf, name);
-        LogReader reader = dlm.getInputStream(readFromTxId);
-
-        long txid = startTxId;
-        int numRead = 0;
-        LogRecord r = reader.readNext(false);
-        while (null != r) {
-            DLMTestUtil.verifyLogRecord(r);
-            assertEquals(txid++, r.getTransactionId());
-            ++numRead;
-            r = reader.readNext(false);
-        }
-        assertEquals(numEntries, numRead);
-        reader.close();
-        dlm.close();
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestWriteLimiter.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestWriteLimiter.java
deleted file mode 100644
index 4956913..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestWriteLimiter.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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-import com.twitter.distributedlog.util.PermitLimiter;
-import com.twitter.distributedlog.util.SimplePermitLimiter;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-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);
-
-    SimplePermitLimiter createPermitLimiter(boolean darkmode, int permits) {
-        return createPermitLimiter(darkmode, permits, new SettableFeature("", 0));
-    }
-
-    SimplePermitLimiter createPermitLimiter(boolean darkmode, int permits, Feature feature) {
-        return new SimplePermitLimiter(darkmode, permits, new NullStatsLogger(), false, feature);
-    }
-
-    @Test(timeout = 60000)
-    public void testGlobalOnly() throws Exception {
-        SimplePermitLimiter streamLimiter = createPermitLimiter(false, Integer.MAX_VALUE);
-        SimplePermitLimiter globalLimiter = createPermitLimiter(false, 1);
-        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
-        limiter.acquire();
-        try {
-            limiter.acquire();
-            fail("should have thrown global limit exception");
-        } catch (OverCapacityException ex) {
-        }
-        assertPermits(streamLimiter, 1, globalLimiter, 1);
-        limiter.release();
-        assertPermits(streamLimiter, 0, globalLimiter, 0);
-    }
-
-    @Test(timeout = 60000)
-    public void testStreamOnly() throws Exception {
-        SimplePermitLimiter streamLimiter = createPermitLimiter(false, 1);
-        SimplePermitLimiter globalLimiter = createPermitLimiter(false, Integer.MAX_VALUE);
-        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
-        limiter.acquire();
-        try {
-            limiter.acquire();
-            fail("should have thrown stream limit exception");
-        } catch (OverCapacityException ex) {
-        }
-        assertPermits(streamLimiter, 1, globalLimiter, 1);
-    }
-
-    @Test(timeout = 60000)
-    public void testDarkmode() throws Exception {
-        SimplePermitLimiter streamLimiter = createPermitLimiter(true, Integer.MAX_VALUE);
-        SimplePermitLimiter globalLimiter = createPermitLimiter(true, 1);
-        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
-        limiter.acquire();
-        limiter.acquire();
-        assertPermits(streamLimiter, 2, globalLimiter, 2);
-    }
-
-    @Test(timeout = 60000)
-    public void testDarkmodeWithDisabledFeature() throws Exception {
-        SettableFeature feature = new SettableFeature("test", 10000);
-        SimplePermitLimiter streamLimiter = createPermitLimiter(true, 1, feature);
-        SimplePermitLimiter globalLimiter = createPermitLimiter(true, Integer.MAX_VALUE, feature);
-        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
-        limiter.acquire();
-        limiter.acquire();
-        assertPermits(streamLimiter, 2, globalLimiter, 2);
-        limiter.release();
-        limiter.release();
-        assertPermits(streamLimiter, 0, globalLimiter, 0);
-    }
-
-    @Test(timeout = 60000)
-    public void testDisabledFeature() throws Exception {
-        // Disable darkmode, but should still ignore limits because of the feature.
-        SettableFeature feature = new SettableFeature("test", 10000);
-        SimplePermitLimiter streamLimiter = createPermitLimiter(false, 1, feature);
-        SimplePermitLimiter globalLimiter = createPermitLimiter(false, Integer.MAX_VALUE, feature);
-        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
-        limiter.acquire();
-        limiter.acquire();
-        assertPermits(streamLimiter, 2, globalLimiter, 2);
-        limiter.release();
-        limiter.release();
-        assertPermits(streamLimiter, 0, globalLimiter, 0);
-    }
-
-    @Test(timeout = 60000)
-    public void testSetDisableFeatureAfterAcquireAndBeforeRelease() throws Exception {
-        SettableFeature feature = new SettableFeature("test", 0);
-        SimplePermitLimiter streamLimiter = createPermitLimiter(false, 2, feature);
-        SimplePermitLimiter globalLimiter = createPermitLimiter(false, Integer.MAX_VALUE, feature);
-        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
-        limiter.acquire();
-        limiter.acquire();
-        assertPermits(streamLimiter, 2, globalLimiter, 2);
-        feature.set(10000);
-        limiter.release();
-        limiter.release();
-        assertPermits(streamLimiter, 0, globalLimiter, 0);
-    }
-
-    @Test(timeout = 60000)
-    public void testUnsetDisableFeatureAfterPermitsExceeded() throws Exception {
-        SettableFeature feature = new SettableFeature("test", 10000);
-        SimplePermitLimiter streamLimiter = createPermitLimiter(false, 1, feature);
-        SimplePermitLimiter globalLimiter = createPermitLimiter(false, Integer.MAX_VALUE, feature);
-        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
-        limiter.acquire();
-        limiter.acquire();
-        limiter.acquire();
-        limiter.acquire();
-        assertPermits(streamLimiter, 4, globalLimiter, 4);
-        feature.set(0);
-        limiter.release();
-        assertPermits(streamLimiter, 3, globalLimiter, 3);
-        try {
-            limiter.acquire();
-            fail("should have thrown stream limit exception");
-        } catch (OverCapacityException ex) {
-        }
-        assertPermits(streamLimiter, 3, globalLimiter, 3);
-        limiter.release();
-        limiter.release();
-        limiter.release();
-        assertPermits(streamLimiter, 0, globalLimiter, 0);
-    }
-
-    @Test(timeout = 60000)
-    public void testUnsetDisableFeatureBeforePermitsExceeded() throws Exception {
-        SettableFeature feature = new SettableFeature("test", 0);
-        SimplePermitLimiter streamLimiter = createPermitLimiter(false, 1, feature);
-        SimplePermitLimiter globalLimiter = createPermitLimiter(false, Integer.MAX_VALUE, feature);
-        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
-        limiter.acquire();
-        try {
-            limiter.acquire();
-            fail("should have thrown stream limit exception");
-        } catch (OverCapacityException ex) {
-        }
-        assertPermits(streamLimiter, 1, globalLimiter, 1);
-        feature.set(10000);
-        limiter.acquire();
-        assertPermits(streamLimiter, 2, globalLimiter, 2);
-    }
-
-    @Test(timeout = 60000)
-    public void testDarkmodeGlobalUnderStreamOver() throws Exception {
-        SimplePermitLimiter streamLimiter = createPermitLimiter(true, 1);
-        SimplePermitLimiter globalLimiter = createPermitLimiter(true, 2);
-        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
-        limiter.acquire();
-        limiter.acquire();
-        assertPermits(streamLimiter, 2, globalLimiter, 2);
-        limiter.release();
-        limiter.release();
-        assertPermits(streamLimiter, 0, globalLimiter, 0);
-    }
-
-    @Test(timeout = 60000)
-    public void testDarkmodeGlobalOverStreamUnder() throws Exception {
-        SimplePermitLimiter streamLimiter = createPermitLimiter(true, 2);
-        SimplePermitLimiter globalLimiter = createPermitLimiter(true, 1);
-        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
-        limiter.acquire();
-        limiter.acquire();
-        assertPermits(streamLimiter, 2, globalLimiter, 2);
-        limiter.release();
-        assertPermits(streamLimiter, 1, globalLimiter, 1);
-        limiter.release();
-        assertPermits(streamLimiter, 0, globalLimiter, 0);
-    }
-
-    void assertPermits(SimplePermitLimiter streamLimiter, int streamPermits, SimplePermitLimiter globalLimiter, int globalPermits) {
-        assertEquals(streamPermits, streamLimiter.getPermits());
-        assertEquals(globalPermits, globalLimiter.getPermits());
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestZooKeeperClient.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestZooKeeperClient.java
deleted file mode 100644
index 33effbc..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestZooKeeperClient.java
+++ /dev/null
@@ -1,449 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.ZooKeeperClient.Credentials;
-import com.twitter.distributedlog.ZooKeeperClient.DigestCredentials;
-import com.twitter.distributedlog.annotations.DistributedLogAnnotations;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.Watcher.Event.EventType;
-import org.apache.zookeeper.Watcher.Event.KeeperState;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.Stat;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.lang.reflect.Field;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.*;
-
-/**
- * Test Cases for {@link com.twitter.distributedlog.ZooKeeperClient}
- */
-public class TestZooKeeperClient extends ZooKeeperClusterTestCase {
-    static final Logger LOG = LoggerFactory.getLogger(TestZooKeeperClient.class);
-
-    private final static int sessionTimeoutMs = 2000;
-
-    private ZooKeeperClient zkc;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = buildClient();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        zkc.close();
-    }
-
-    private ZooKeeperClientBuilder clientBuilder() throws Exception {
-        return clientBuilder(sessionTimeoutMs);
-    }
-
-    private ZooKeeperClientBuilder clientBuilder(int sessionTimeoutMs)
-            throws Exception {
-        return ZooKeeperClientBuilder.newBuilder()
-                .name("zkc")
-                .uri(DLMTestUtil.createDLMURI(zkPort, "/"))
-                .sessionTimeoutMs(sessionTimeoutMs)
-                .zkServers(zkServers)
-                .retryPolicy(new BoundExponentialBackoffRetryPolicy(100, 200, 2));
-    }
-
-    private ZooKeeperClient buildClient() throws Exception {
-        return clientBuilder().zkAclId(null).build();
-    }
-
-    private ZooKeeperClient buildAuthdClient(String id) throws Exception {
-        return clientBuilder().zkAclId(id).build();
-    }
-
-    private void rmAll(ZooKeeperClient client, String path) throws Exception {
-        List<String> nodes = client.get().getChildren(path, false);
-        for (String node : nodes) {
-            String childPath = path + "/" + node;
-            rmAll(client, childPath);
-        }
-        client.get().delete(path, 0);
-    }
-
-    @Test(timeout = 60000)
-    public void testAclCreatePerms() throws Exception {
-        ZooKeeperClient zkcAuth = buildAuthdClient("test");
-        zkcAuth.get().create("/test", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        zkcAuth.get().create("/test/key1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        zkcAuth.get().create("/test/key2", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
-
-        ZooKeeperClient zkcNoAuth = buildClient();
-        zkcNoAuth.get().create("/test/key1/key1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        try {
-            zkcNoAuth.get().create("/test/key2/key1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-            fail("create should fail on acl protected key");
-        } catch (KeeperException.NoAuthException ex) {
-            LOG.info("caught exception writing to protected key", ex);
-        }
-
-        rmAll(zkcAuth, "/test");
-    }
-
-    @Test(timeout = 60000)
-    public void testAclNullIdDisablesAuth() throws Exception {
-        ZooKeeperClient zkcAuth = buildAuthdClient(null);
-        zkcAuth.get().create("/test", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        zkcAuth.get().create("/test/key1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        try {
-            zkcAuth.get().create("/test/key2", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
-            fail("create should fail because we're not authenticated");
-        } catch (KeeperException.InvalidACLException ex) {
-            LOG.info("caught exception writing to protected key", ex);
-        }
-
-        rmAll(zkcAuth, "/test");
-    }
-
-    @Test(timeout = 60000)
-    public void testAclAllowsReadsForNoAuth() throws Exception {
-        ZooKeeperClient zkcAuth = buildAuthdClient("test");
-        zkcAuth.get().create("/test", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
-        zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
-        zkcAuth.get().create("/test/key1/key2", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
-
-        ZooKeeperClient zkcNoAuth = buildClient();
-        List<String> nodes = null;
-        String path = "/test";
-        nodes = zkcNoAuth.get().getChildren(path, false);
-        path = path + "/" + nodes.get(0);
-        nodes = zkcNoAuth.get().getChildren(path, false);
-        assertEquals("key2", nodes.get(0));
-
-        ZooKeeperClient zkcAuth2 = buildAuthdClient("test2");
-        path = "/test";
-        nodes = zkcNoAuth.get().getChildren(path, false);
-        path = path + "/" + nodes.get(0);
-        nodes = zkcNoAuth.get().getChildren(path, false);
-        assertEquals("key2", nodes.get(0));
-
-        rmAll(zkcAuth, "/test");
-    }
-
-    @Test(timeout = 60000)
-    public void testAclDigestCredentialsBasics() throws Exception {
-        ZooKeeperClient zkcAuth = buildClient();
-        zkcAuth.get().create("/test", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-
-        try {
-            zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
-            fail("should have failed");
-        } catch (Exception ex) {
-        }
-
-        Credentials credentials = new DigestCredentials("test", "test");
-        credentials.authenticate(zkcAuth.get());
-
-        // Should not throw now that we're authenticated.
-        zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
-
-        rmAll(zkcAuth, "/test");
-    }
-
-    @Test(timeout = 60000)
-    public void testAclNoopCredentialsDoesNothing() throws Exception {
-        Credentials.NONE.authenticate(null);
-    }
-
-    class FailingCredentials implements Credentials {
-        boolean shouldFail = true;
-        @Override
-        public void authenticate(ZooKeeper zooKeeper) {
-            if (shouldFail) {
-                throw new RuntimeException("authfailed");
-            }
-        }
-        public void setShouldFail(boolean shouldFail) {
-            this.shouldFail = shouldFail;
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testAclFailedAuthenticationCanBeRecovered() throws Exception {
-        FailingCredentials credentials = new FailingCredentials();
-        ZooKeeperClient zkc = new ZooKeeperClient("test", 2000, 2000, zkServers,
-                null, NullStatsLogger.INSTANCE, 1, 10000, credentials);
-
-        try {
-            zkc.get();
-            fail("should have failed on auth");
-        } catch (Exception ex) {
-            assertEquals("authfailed", ex.getMessage());
-        }
-
-        // Should recover fine
-        credentials.setShouldFail(false);
-        zkc.get().create("/test", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-
-        rmAll(zkc, "/test");
-    }
-
-    private void expireZooKeeperSession(ZooKeeper zk, int timeout)
-            throws IOException, InterruptedException, KeeperException {
-        final CountDownLatch latch = new CountDownLatch(1);
-
-        ZooKeeper newZk = new ZooKeeper(zkServers, timeout, new Watcher() {
-            @Override
-            public void process(WatchedEvent event) {
-                if (event.getType() == EventType.None && event.getState() == KeeperState.SyncConnected) {
-                    latch.countDown();
-                }
-            }},
-            zk.getSessionId(),
-            zk.getSessionPasswd());
-
-        if (!latch.await(timeout, TimeUnit.MILLISECONDS)) {
-            throw KeeperException.create(KeeperException.Code.CONNECTIONLOSS);
-        }
-
-        newZk.close();
-    }
-
-    private CountDownLatch awaitConnectionEvent(final KeeperState state, final ZooKeeperClient zkc) {
-        final CountDownLatch connected = new CountDownLatch(1);
-        Watcher watcher = new Watcher() {
-            @Override
-            public void process(WatchedEvent event) {
-                if (event.getType() == EventType.None && event.getState() == state) {
-                    connected.countDown();
-                }
-            }
-        };
-        zkc.register(watcher);
-        return connected;
-    }
-
-    /**
-     * {@link https://issues.apache.org/jira/browse/DL-34}
-     */
-    @DistributedLogAnnotations.FlakyTest
-    @Ignore
-    @Test(timeout = 60000)
-    public void testAclAuthSpansExpiration() throws Exception {
-        ZooKeeperClient zkcAuth = buildAuthdClient("test");
-        zkcAuth.get().create("/test", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
-
-        CountDownLatch expired = awaitConnectionEvent(KeeperState.Expired, zkcAuth);
-        CountDownLatch connected = awaitConnectionEvent(KeeperState.SyncConnected, zkcAuth);
-
-        expireZooKeeperSession(zkcAuth.get(), 2000);
-
-        expired.await(2, TimeUnit.SECONDS);
-        connected.await(2, TimeUnit.SECONDS);
-
-        zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
-
-        rmAll(zkcAuth, "/test");
-    }
-
-    /**
-     * {@link https://issues.apache.org/jira/browse/DL-34}
-     */
-    @DistributedLogAnnotations.FlakyTest
-    @Ignore
-    @Test(timeout = 60000)
-    public void testAclAuthSpansExpirationNonRetryableClient() throws Exception {
-        ZooKeeperClient zkcAuth = clientBuilder().retryPolicy(null).zkAclId("test").build();
-        zkcAuth.get().create("/test", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
-
-        CountDownLatch expired = awaitConnectionEvent(KeeperState.Expired, zkcAuth);
-        CountDownLatch connected = awaitConnectionEvent(KeeperState.SyncConnected, zkcAuth);
-
-        expireZooKeeperSession(zkcAuth.get(), 2000);
-
-        expired.await(2, TimeUnit.SECONDS);
-        connected.await(2, TimeUnit.SECONDS);
-
-        zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
-
-        rmAll(zkcAuth, "/test");
-    }
-
-    static class TestWatcher implements Watcher {
-
-        final List<WatchedEvent> receivedEvents = new ArrayList<WatchedEvent>();
-        CountDownLatch latch = new CountDownLatch(0);
-
-        public TestWatcher setLatch(CountDownLatch latch) {
-            this.latch = latch;
-            return this;
-        }
-
-        @Override
-        public void process(WatchedEvent event) {
-            if (event.getType() == Event.EventType.NodeDataChanged) {
-                synchronized (receivedEvents) {
-                    receivedEvents.add(event);
-                }
-                latch.countDown();
-            }
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testRegisterUnregisterWatchers() throws Exception {
-        TestWatcher w1 = new TestWatcher();
-        TestWatcher w2 = new TestWatcher();
-
-        final CountDownLatch latch = new CountDownLatch(2);
-        w1.setLatch(latch);
-        w2.setLatch(latch);
-
-        zkc.register(w1);
-        zkc.register(w2);
-
-        assertEquals(2, zkc.watchers.size());
-
-        final String zkPath = "/test-register-unregister-watchers";
-
-        zkc.get().create(zkPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        zkc.get().getData(zkPath, true, new Stat());
-
-        zkc.get().setData(zkPath, "first-set".getBytes(), -1);
-        latch.await();
-        assertEquals(1, w1.receivedEvents.size());
-        assertEquals(zkPath, w1.receivedEvents.get(0).getPath());
-        assertEquals(Watcher.Event.EventType.NodeDataChanged, w1.receivedEvents.get(0).getType());
-        assertEquals(1, w2.receivedEvents.size());
-        assertEquals(zkPath, w2.receivedEvents.get(0).getPath());
-        assertEquals(Watcher.Event.EventType.NodeDataChanged, w2.receivedEvents.get(0).getType());
-
-        final CountDownLatch latch1 = new CountDownLatch(1);
-        final CountDownLatch latch2 = new CountDownLatch(1);
-        w1.setLatch(latch1);
-        w2.setLatch(latch2);
-
-        zkc.unregister(w2);
-
-        assertEquals(1, zkc.watchers.size());
-        zkc.get().getData(zkPath, true, new Stat());
-        zkc.get().setData(zkPath, "second-set".getBytes(), -1);
-        latch1.await();
-        assertEquals(2, w1.receivedEvents.size());
-        assertEquals(zkPath, w1.receivedEvents.get(1).getPath());
-        assertEquals(Watcher.Event.EventType.NodeDataChanged, w1.receivedEvents.get(1).getType());
-        assertFalse(latch2.await(2, TimeUnit.SECONDS));
-        assertEquals(1, w2.receivedEvents.size());
-    }
-
-    @Test(timeout = 60000)
-    public void testExceptionOnWatchers() throws Exception {
-        TestWatcher w1 = new TestWatcher();
-        TestWatcher w2 = new TestWatcher();
-
-        final CountDownLatch latch = new CountDownLatch(2);
-        w1.setLatch(latch);
-        w2.setLatch(latch);
-
-        zkc.register(w1);
-        zkc.register(w2);
-        // register bad watcher
-        zkc.register(new Watcher() {
-            @Override
-            public void process(WatchedEvent event) {
-                throw new NullPointerException("bad watcher returning null");
-            }
-        });
-
-        assertEquals(3, zkc.watchers.size());
-
-        final String zkPath = "/test-exception-on-watchers";
-
-        zkc.get().create(zkPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        zkc.get().getData(zkPath, true, new Stat());
-
-        zkc.get().setData(zkPath, "first-set".getBytes(), -1);
-        latch.await();
-        assertEquals(1, w1.receivedEvents.size());
-        assertEquals(zkPath, w1.receivedEvents.get(0).getPath());
-        assertEquals(Watcher.Event.EventType.NodeDataChanged, w1.receivedEvents.get(0).getType());
-        assertEquals(1, w2.receivedEvents.size());
-        assertEquals(zkPath, w2.receivedEvents.get(0).getPath());
-        assertEquals(Watcher.Event.EventType.NodeDataChanged, w2.receivedEvents.get(0).getType());
-    }
-
-    @Test(timeout = 60000)
-    public void testZooKeeperReconnection() throws Exception {
-        int sessionTimeoutMs = 100;
-        ZooKeeperClient zkc = clientBuilder(sessionTimeoutMs).zkAclId(null).build();
-        ZooKeeper zk = zkc.get();
-        long sessionId = zk.getSessionId();
-        ZooKeeperClientUtils.expireSession(zkc, zkServers, 2 * sessionTimeoutMs);
-        ZooKeeper newZk = zkc.get();
-        while (!ZooKeeper.States.CONNECTED.equals(newZk.getState())) {
-            TimeUnit.MILLISECONDS.sleep(sessionTimeoutMs / 2);
-        }
-        long newSessionId = newZk.getSessionId();
-        assertTrue(newZk == zk);
-        assertFalse(sessionId == newSessionId);
-    }
-
-    @Test(timeout = 60000)
-    public void testZooKeeperReconnectionBlockingRetryThread() throws Exception {
-        int sessionTimeoutMs = 100;
-        ZooKeeperClient zkc = clientBuilder(sessionTimeoutMs).zkAclId(null).build();
-        ZooKeeper zk = zkc.get();
-        assertTrue(zk instanceof org.apache.bookkeeper.zookeeper.ZooKeeperClient);
-        org.apache.bookkeeper.zookeeper.ZooKeeperClient bkZkc =
-                (org.apache.bookkeeper.zookeeper.ZooKeeperClient) zk;
-        // get the connect executor
-        Field connectExecutorField = bkZkc.getClass().getDeclaredField("connectExecutor");
-        connectExecutorField.setAccessible(true);
-        ExecutorService connectExecutor = (ExecutorService) connectExecutorField.get(bkZkc);
-        final CountDownLatch latch = new CountDownLatch(1);
-        // block retry thread in the zookeeper client
-        connectExecutor.submit(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    latch.await();
-                } catch (InterruptedException e) {
-                }
-            }
-        });
-        ZooKeeperClientUtils.expireSession(zkc, zkServers, 2 * sessionTimeoutMs);
-        ZooKeeper newZk;
-        while ((newZk = zkc.get()) == zk) {
-            TimeUnit.MILLISECONDS.sleep(sessionTimeoutMs / 2);
-        }
-        assertEquals(ZooKeeper.States.CONNECTED, newZk.getState());
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestZooKeeperClientBuilder.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/TestZooKeeperClientBuilder.java
deleted file mode 100644
index 93984f3..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/TestZooKeeperClientBuilder.java
+++ /dev/null
@@ -1,57 +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 com.twitter.distributedlog;
-
-import com.twitter.distributedlog.util.RetryPolicyUtils;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-
-/**
- * The zookeeper client builder used for testing.
- */
-public class TestZooKeeperClientBuilder {
-
-    /**
-     * Return a zookeeper client builder for testing.
-     *
-     * @return a zookeeper client builder
-     */
-    public static ZooKeeperClientBuilder newBuilder() {
-        return ZooKeeperClientBuilder.newBuilder()
-                .retryPolicy(RetryPolicyUtils.DEFAULT_INFINITE_RETRY_POLICY)
-                .connectionTimeoutMs(10000)
-                .sessionTimeoutMs(60000)
-                .zkAclId(null)
-                .statsLogger(NullStatsLogger.INSTANCE);
-    }
-
-    /**
-     * Create a zookeeper client builder with provided <i>conf</i> for testing.
-     *
-     * @param conf distributedlog configuration
-     * @return zookeeper client builder
-     */
-    public static ZooKeeperClientBuilder newBuilder(DistributedLogConfiguration conf) {
-        return ZooKeeperClientBuilder.newBuilder()
-                .retryPolicy(RetryPolicyUtils.DEFAULT_INFINITE_RETRY_POLICY)
-                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                .zkAclId(conf.getZkAclId())
-                .retryThreadCount(conf.getZKClientNumberRetryThreads())
-                .requestRateLimit(conf.getZKRequestRateLimit())
-                .statsLogger(NullStatsLogger.INSTANCE);
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/ZooKeeperClientUtils.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/ZooKeeperClientUtils.java
deleted file mode 100644
index df0d306..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/ZooKeeperClientUtils.java
+++ /dev/null
@@ -1,95 +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 com.twitter.distributedlog;
-
-import com.google.common.base.Stopwatch;
-
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.*;
-
-/**
- * Utilities of {@link com.twitter.distributedlog.ZooKeeperClient}
- */
-public class ZooKeeperClientUtils {
-
-    static final Logger logger = LoggerFactory.getLogger(ZooKeeperClientUtils.class);
-
-    /**
-     * Expire given zookeeper client's session.
-     *
-     * @param zkc
-     *          zookeeper client
-     * @param zkServers
-     *          zookeeper servers
-     * @param timeout
-     *          timeout
-     * @throws Exception
-     */
-    public static void expireSession(ZooKeeperClient zkc, String zkServers, int timeout)
-            throws Exception {
-        final CountDownLatch expireLatch = new CountDownLatch(1);
-        final CountDownLatch latch = new CountDownLatch(1);
-        ZooKeeper oldZk = zkc.get();
-        oldZk.exists("/", new Watcher() {
-            @Override
-            public void process(WatchedEvent event) {
-                logger.debug("Receive event : {}", event);
-                if (event.getType() == Event.EventType.None &&
-                        event.getState() == Event.KeeperState.Expired) {
-                    expireLatch.countDown();
-                }
-            }
-        });
-        ZooKeeper newZk = new ZooKeeper(zkServers, timeout, new Watcher() {
-            @Override
-            public void process(WatchedEvent event) {
-                if (Event.EventType.None == event.getType() &&
-                        Event.KeeperState.SyncConnected == event.getState()) {
-                    latch.countDown();
-                }
-            }
-        }, oldZk.getSessionId(), oldZk.getSessionPasswd());
-        if (!latch.await(timeout, TimeUnit.MILLISECONDS)) {
-            throw KeeperException.create(KeeperException.Code.CONNECTIONLOSS);
-        }
-        newZk.close();
-
-        boolean done = false;
-        Stopwatch expireWait = Stopwatch.createStarted();
-        while (!done && expireWait.elapsed(TimeUnit.MILLISECONDS) < timeout*2) {
-            try {
-                zkc.get().exists("/", false);
-                done = true;
-            } catch (KeeperException ke) {
-                done = (ke.code() == KeeperException.Code.SESSIONEXPIRED);
-            }
-        }
-
-        assertTrue("Client should receive session expired event.",
-                   expireLatch.await(timeout, TimeUnit.MILLISECONDS));
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/ZooKeeperClusterTestCase.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/ZooKeeperClusterTestCase.java
deleted file mode 100644
index 6747ef8..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/ZooKeeperClusterTestCase.java
+++ /dev/null
@@ -1,53 +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 com.twitter.distributedlog;
-
-import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.bookkeeper.util.LocalBookKeeper;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-
-import java.io.File;
-
-public class ZooKeeperClusterTestCase {
-
-    protected static File zkDir;
-    protected static ZooKeeperServerShim zks;
-    protected static String zkServers;
-    protected static int zkPort;
-
-    @BeforeClass
-    public static void setupZooKeeper() throws Exception {
-        zkDir = IOUtils.createTempDir("zookeeper", ZooKeeperClusterTestCase.class.getName());
-        Pair<ZooKeeperServerShim, Integer> serverAndPort = LocalDLMEmulator.runZookeeperOnAnyPort(zkDir);
-        zks = serverAndPort.getLeft();
-        zkPort = serverAndPort.getRight();
-        zkServers = "127.0.0.1:" + zkPort;
-    }
-
-    @AfterClass
-    public static void shutdownZooKeeper() throws Exception {
-        zks.stop();
-        if (null != zkDir) {
-            FileUtils.deleteDirectory(zkDir);
-        }
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControl.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControl.java
deleted file mode 100644
index ff924f8..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControl.java
+++ /dev/null
@@ -1,154 +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 com.twitter.distributedlog.acl;
-
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClusterTestCase;
-import com.twitter.distributedlog.impl.acl.ZKAccessControl;
-import com.twitter.distributedlog.thrift.AccessControlEntry;
-import com.twitter.util.Await;
-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.*;
-
-public class TestZKAccessControl extends ZooKeeperClusterTestCase {
-
-    private ZooKeeperClient zkc;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .uri(createURI("/"))
-                .build();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        zkc.close();
-    }
-
-    private URI createURI(String path) {
-        return URI.create("distributedlog://127.0.0.1:" + zkPort + path);
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateZKAccessControl() throws Exception {
-        AccessControlEntry ace = new AccessControlEntry();
-        ace.setDenyWrite(true);
-        String zkPath = "/create-zk-access-control";
-        ZKAccessControl zkac = new ZKAccessControl(ace, zkPath);
-        Await.result(zkac.create(zkc));
-
-        ZKAccessControl readZKAC = Await.result(ZKAccessControl.read(zkc, zkPath, null));
-        assertEquals(zkac, readZKAC);
-
-        ZKAccessControl another = new ZKAccessControl(ace, zkPath);
-        try {
-            Await.result(another.create(zkc));
-        } catch (KeeperException.NodeExistsException ke) {
-            // expected
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testDeleteZKAccessControl() throws Exception {
-        String zkPath = "/delete-zk-access-control";
-
-        AccessControlEntry ace = new AccessControlEntry();
-        ace.setDenyDelete(true);
-
-        ZKAccessControl zkac = new ZKAccessControl(ace, zkPath);
-        Await.result(zkac.create(zkc));
-
-        ZKAccessControl readZKAC = Await.result(ZKAccessControl.read(zkc, zkPath, null));
-        assertEquals(zkac, readZKAC);
-
-        Await.result(ZKAccessControl.delete(zkc, zkPath));
-
-        try {
-            Await.result(ZKAccessControl.read(zkc, zkPath, null));
-        } catch (KeeperException.NoNodeException nne) {
-            // expected.
-        }
-        Await.result(ZKAccessControl.delete(zkc, zkPath));
-    }
-
-    @Test(timeout = 60000)
-    public void testEmptyZKAccessControl() throws Exception {
-        String zkPath = "/empty-access-control";
-
-        zkc.get().create(zkPath, new byte[0], zkc.getDefaultACL(), CreateMode.PERSISTENT);
-
-        ZKAccessControl readZKAC = Await.result(ZKAccessControl.read(zkc, zkPath, null));
-
-        assertEquals(zkPath, readZKAC.getZKPath());
-        assertEquals(ZKAccessControl.DEFAULT_ACCESS_CONTROL_ENTRY, readZKAC.getAccessControlEntry());
-        assertTrue(ZKAccessControl.DEFAULT_ACCESS_CONTROL_ENTRY == readZKAC.getAccessControlEntry());
-    }
-
-    @Test(timeout = 60000)
-    public void testCorruptedZKAccessControl() throws Exception {
-        String zkPath = "/corrupted-zk-access-control";
-
-        zkc.get().create(zkPath, "corrupted-data".getBytes(UTF_8), zkc.getDefaultACL(), CreateMode.PERSISTENT);
-
-        try {
-            Await.result(ZKAccessControl.read(zkc, zkPath, null));
-        } catch (ZKAccessControl.CorruptedAccessControlException cace) {
-            // expected
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testUpdateZKAccessControl() throws Exception {
-        String zkPath = "/update-zk-access-control";
-
-        AccessControlEntry ace = new AccessControlEntry();
-        ace.setDenyDelete(true);
-
-        ZKAccessControl zkac = new ZKAccessControl(ace, zkPath);
-        Await.result(zkac.create(zkc));
-
-        ZKAccessControl readZKAC = Await.result(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));
-        assertEquals(newZKAC, readZKAC2);
-
-        try {
-            Await.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));
-        assertEquals(readZKAC2, readZKAC3);
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControlManager.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControlManager.java
deleted file mode 100644
index 5625306..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/acl/TestZKAccessControlManager.java
+++ /dev/null
@@ -1,178 +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 com.twitter.distributedlog.acl;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientUtils;
-import com.twitter.distributedlog.ZooKeeperClusterTestCase;
-import com.twitter.distributedlog.impl.acl.ZKAccessControl;
-import com.twitter.distributedlog.impl.acl.ZKAccessControlManager;
-import com.twitter.distributedlog.thrift.AccessControlEntry;
-import com.twitter.util.Await;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-
-import static org.junit.Assert.*;
-
-public class TestZKAccessControlManager extends ZooKeeperClusterTestCase {
-
-    static final Logger logger = LoggerFactory.getLogger(TestZKAccessControlManager.class);
-
-    private DistributedLogConfiguration conf;
-    private ZooKeeperClient zkc;
-    private ScheduledExecutorService executorService;
-
-    private URI createURI(String path) {
-        return URI.create("distributedlog://127.0.0.1:" + zkPort + path);
-    }
-
-    @Before
-    public void setup() throws Exception {
-        executorService = Executors.newSingleThreadScheduledExecutor();
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .uri(createURI("/"))
-                .build();
-        conf = new DistributedLogConfiguration();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        zkc.close();
-        executorService.shutdown();
-    }
-
-    void setACL(ZKAccessControl accessControl) throws Exception {
-        String zkPath = accessControl.getZKPath();
-        if (null == zkc.get().exists(zkPath, false)) {
-            accessControl.create(zkc);
-        } else {
-            accessControl.update(zkc);
-        }
-    }
-
-    static void verifyStreamPermissions(ZKAccessControlManager zkcm,
-                                        String stream,
-                                        boolean allowWrite,
-                                        boolean allowTruncate,
-                                        boolean allowRelease,
-                                        boolean allowDelete,
-                                        boolean allowAcquire) throws Exception {
-        assertEquals(allowWrite, zkcm.allowWrite(stream));
-        assertEquals(allowTruncate, zkcm.allowTruncate(stream));
-        assertEquals(allowRelease, zkcm.allowRelease(stream));
-        assertEquals(allowDelete, zkcm.allowDelete(stream));
-        assertEquals(allowAcquire, zkcm.allowAcquire(stream));
-    }
-
-    @Test(timeout = 60000)
-    public void testZKAccessControlManager() throws Exception {
-        String zkRootPath = "/test-zk-access-control-manager";
-        String stream1 = "test-acm-1";
-        String stream2 = "test-acm-2";
-        logger.info("Creating ACL Manager for {}", zkRootPath);
-        ZKAccessControlManager zkcm = new ZKAccessControlManager(conf, zkc, zkRootPath, executorService);
-        logger.info("Created ACL Manager for {}", zkRootPath);
-        try {
-            verifyStreamPermissions(zkcm, stream1, true, true, true, true, true);
-
-            // create stream1 (denyDelete = true)
-            String zkPath1 = zkRootPath + "/" + stream1;
-            AccessControlEntry ace1 = new AccessControlEntry();
-            ace1.setDenyDelete(true);
-            ZKAccessControl accessControl1 = new ZKAccessControl(ace1, zkPath1);
-            setACL(accessControl1);
-            logger.info("Create ACL for stream {} : {}", stream1, accessControl1);
-            while (zkcm.allowDelete(stream1)) {
-                Thread.sleep(100);
-            }
-            verifyStreamPermissions(zkcm, stream1, true, true, true, false, true);
-
-            // update stream1 (denyDelete = false, denyWrite = true)
-            ace1 = new AccessControlEntry();
-            ace1.setDenyWrite(true);
-            accessControl1 = new ZKAccessControl(ace1, zkPath1);
-            setACL(accessControl1);
-            logger.info("Update ACL for stream {} : {}", stream1, accessControl1);
-
-            // create stream2 (denyTruncate = true)
-            String zkPath2 = zkRootPath + "/" + stream2;
-            AccessControlEntry ace2 = new AccessControlEntry();
-            ace2.setDenyTruncate(true);
-            ZKAccessControl accessControl2 = new ZKAccessControl(ace2, zkPath2);
-            setACL(accessControl2);
-            logger.info("Create ACL for stream {} : {}", stream2, accessControl2);
-            while (zkcm.allowWrite(stream1)) {
-                Thread.sleep(100);
-            }
-            while (zkcm.allowTruncate(stream2)) {
-                Thread.sleep(100);
-            }
-
-            verifyStreamPermissions(zkcm, stream1, false, true, true, true, true);
-            verifyStreamPermissions(zkcm, stream2, true, false, true, true, true);
-
-            // delete stream2
-            Await.result(ZKAccessControl.delete(zkc, zkPath2));
-            logger.info("Delete ACL for stream {}", stream2);
-            while (!zkcm.allowTruncate(stream2)) {
-                Thread.sleep(100);
-            }
-
-            verifyStreamPermissions(zkcm, stream1, false, true, true, true, true);
-            verifyStreamPermissions(zkcm, stream2, true, true, true, true, true);
-
-            // expire session
-            ZooKeeperClientUtils.expireSession(zkc, zkServers, 1000);
-
-            // update stream1 (denyDelete = false, denyWrite = true)
-            ace1 = new AccessControlEntry();
-            ace1.setDenyRelease(true);
-            accessControl1 = new ZKAccessControl(ace1, zkPath1);
-            setACL(accessControl1);
-            logger.info("Update ACL for stream {} : {}", stream1, accessControl1);
-
-            // create stream2 (denyTruncate = true)
-            ace2 = new AccessControlEntry();
-            ace2.setDenyAcquire(true);
-            accessControl2 = new ZKAccessControl(ace2, zkPath2);
-            setACL(accessControl2);
-            logger.info("Created ACL for stream {} again : {}", stream2, accessControl2);
-
-            while (zkcm.allowRelease(stream1)) {
-                Thread.sleep(100);
-            }
-            while (zkcm.allowAcquire(stream2)) {
-                Thread.sleep(100);
-            }
-
-            verifyStreamPermissions(zkcm, stream1, true, true, false, true, true);
-            verifyStreamPermissions(zkcm, stream2, true, true, true, true, false);
-        } finally {
-            zkcm.close();
-        }
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDLCK.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDLCK.java
deleted file mode 100644
index 60bc420..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDLCK.java
+++ /dev/null
@@ -1,164 +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 com.twitter.distributedlog.admin;
-
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
-import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.ZooDefs;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-public class TestDLCK extends TestDistributedLogBase {
-
-    static final Logger LOG = LoggerFactory.getLogger(TestDLCK.class);
-
-    protected static DistributedLogConfiguration conf =
-            new DistributedLogConfiguration().setLockTimeout(10)
-                .setEnableLedgerAllocatorPool(true).setLedgerAllocatorPoolName("test");
-
-    private ZooKeeperClient zkc;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = TestZooKeeperClientBuilder
-            .newBuilder()
-            .uri(createDLMURI("/"))
-            .build();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        zkc.close();
-    }
-
-    static Map<Long, LogSegmentMetadata> getLogSegments(DistributedLogManager dlm) throws Exception {
-        Map<Long, LogSegmentMetadata> logSegmentMap =
-                new HashMap<Long, LogSegmentMetadata>();
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        for (LogSegmentMetadata segment : segments) {
-            logSegmentMap.put(segment.getLogSegmentSequenceNumber(), segment);
-        }
-        return logSegmentMap;
-    }
-
-    static void verifyLogSegment(Map<Long, LogSegmentMetadata> segments,
-                                 DLSN lastDLSN, long logSegmentSequenceNumber,
-                                 int recordCount, long lastTxId) {
-        LogSegmentMetadata segment = segments.get(logSegmentSequenceNumber);
-        assertNotNull(segment);
-        assertEquals(lastDLSN, segment.getLastDLSN());
-        assertEquals(recordCount, segment.getRecordCount());
-        assertEquals(lastTxId, segment.getLastTxId());
-    }
-
-    @Test(timeout = 60000)
-    @SuppressWarnings("deprecation")
-    public void testCheckAndRepairDLNamespace() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.loadConf(conf);
-        confLocal.setImmediateFlushEnabled(true);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setLogSegmentSequenceNumberValidationEnabled(false);
-        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()
-                .conf(confLocal)
-                .uri(uri)
-                .build();
-        OrderedScheduler scheduler = OrderedScheduler.newBuilder()
-                .name("dlck-tool")
-                .corePoolSize(1)
-                .build();
-        ExecutorService executorService = Executors.newCachedThreadPool();
-
-        String streamName = "check-and-repair-dl-namespace";
-
-        // Create completed log segments
-        DistributedLogManager dlm = namespace.openLog(streamName);
-        DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 1L, 1L, 10, false);
-        DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 2L, 11L, 10, true);
-        DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 3L, 21L, 10, false);
-        DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 4L, 31L, 10, true);
-
-        // dryrun
-        DistributedLogAdmin.checkAndRepairDLNamespace(
-                uri,
-                namespace,
-                new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(namespace)),
-                scheduler,
-                false,
-                false);
-
-        Map<Long, LogSegmentMetadata> segments = getLogSegments(dlm);
-        LOG.info("segments after drynrun {}", segments);
-        verifyLogSegment(segments, new DLSN(1L, 18L, 0L), 1L, 10, 10L);
-        verifyLogSegment(segments, new DLSN(2L, 16L, 0L), 2L, 9, 19L);
-        verifyLogSegment(segments, new DLSN(3L, 18L, 0L), 3L, 10, 30L);
-        verifyLogSegment(segments, new DLSN(4L, 16L, 0L), 4L, 9, 39L);
-
-        // check and repair
-        DistributedLogAdmin.checkAndRepairDLNamespace(
-                uri,
-                namespace,
-                LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(namespace)),
-                scheduler,
-                false,
-                false);
-
-        segments = getLogSegments(dlm);
-        LOG.info("segments after repair {}", segments);
-        verifyLogSegment(segments, new DLSN(1L, 18L, 0L), 1L, 10, 10L);
-        verifyLogSegment(segments, new DLSN(2L, 18L, 0L), 2L, 10, 20L);
-        verifyLogSegment(segments, new DLSN(3L, 18L, 0L), 3L, 10, 30L);
-        verifyLogSegment(segments, new DLSN(4L, 18L, 0L), 4L, 10, 40L);
-
-        dlm.close();
-        SchedulerUtils.shutdownScheduler(executorService, 5, TimeUnit.MINUTES);
-        namespace.close();
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDistributedLogAdmin.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDistributedLogAdmin.java
deleted file mode 100644
index 1e39e49..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/admin/TestDistributedLogAdmin.java
+++ /dev/null
@@ -1,194 +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 com.twitter.distributedlog.admin;
-
-import java.net.URI;
-import java.util.concurrent.TimeUnit;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.annotations.DistributedLogAnnotations;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.util.Utils;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.ZooDefs;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import com.twitter.distributedlog.AsyncLogReader;
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.metadata.DryrunLogSegmentMetadataStoreUpdater;
-import com.twitter.distributedlog.metadata.LogSegmentMetadataStoreUpdater;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-
-import static org.junit.Assert.*;
-
-public class TestDistributedLogAdmin extends TestDistributedLogBase {
-
-    static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogAdmin.class);
-
-    private ZooKeeperClient zooKeeperClient;
-
-    @Before
-    public void setup() throws Exception {
-        zooKeeperClient = TestZooKeeperClientBuilder
-            .newBuilder()
-            .uri(createDLMURI("/"))
-            .build();
-        conf.setTraceReadAheadMetadataChanges(true);
-    }
-
-    @After
-    public void teardown() throws Exception {
-        zooKeeperClient.close();
-    }
-
-    /**
-     * {@link https://issues.apache.org/jira/browse/DL-44}
-     */
-    @DistributedLogAnnotations.FlakyTest
-    @Ignore
-    @Test(timeout = 60000)
-    @SuppressWarnings("deprecation")
-    public void testChangeSequenceNumber() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setLogSegmentSequenceNumberValidationEnabled(false);
-        confLocal.setLogSegmentCacheEnabled(false);
-
-        DistributedLogConfiguration readConf = new DistributedLogConfiguration();
-        readConf.addConfiguration(conf);
-        readConf.setLogSegmentCacheEnabled(false);
-        readConf.setLogSegmentSequenceNumberValidationEnabled(true);
-
-        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()
-                .conf(confLocal)
-                .uri(uri)
-                .build();
-        DistributedLogNamespace readNamespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(readConf)
-                .uri(uri)
-                .build();
-
-        String streamName = "change-sequence-number";
-
-        // create completed log segments
-        DistributedLogManager dlm = namespace.openLog(streamName);
-        DLMTestUtil.generateCompletedLogSegments(dlm, confLocal, 4, 10);
-        DLMTestUtil.injectLogSegmentWithGivenLogSegmentSeqNo(dlm, confLocal, 5, 41, false, 10, true);
-        dlm.close();
-
-        // create a reader
-        DistributedLogManager readDLM = readNamespace.openLog(streamName);
-        AsyncLogReader reader = readDLM.getAsyncLogReader(DLSN.InitialDLSN);
-
-        // read the records
-        long expectedTxId = 1L;
-        DLSN lastDLSN = DLSN.InitialDLSN;
-        for (int i = 0; i < 4 * 10; i++) {
-            LogRecordWithDLSN record = Await.result(reader.readNext());
-            assertNotNull(record);
-            DLMTestUtil.verifyLogRecord(record);
-            assertEquals(expectedTxId, record.getTransactionId());
-            expectedTxId++;
-            lastDLSN = record.getDlsn();
-        }
-
-        LOG.info("Injecting bad log segment '3'");
-
-        dlm = namespace.openLog(streamName);
-        DLMTestUtil.injectLogSegmentWithGivenLogSegmentSeqNo(dlm, confLocal, 3L, 5 * 10 + 1, true, 10, false);
-
-        LOG.info("Injected bad log segment '3'");
-
-        // there isn't records should be read
-        Future<LogRecordWithDLSN> readFuture = reader.readNext();
-        try {
-            LogRecordWithDLSN record = Await.result(readFuture);
-            fail("Should fail reading next record "
-                    + record
-                    + " when there is a corrupted log segment");
-        } catch (UnexpectedException ue) {
-            // expected
-        }
-
-        LOG.info("Dryrun fix inprogress segment that has lower sequence number");
-
-        // Dryrun
-        DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(namespace,
-                new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(namespace)), streamName, false, false);
-
-        try {
-            reader = readDLM.getAsyncLogReader(lastDLSN);
-            Await.result(reader.readNext());
-            fail("Should fail reading next when there is a corrupted log segment");
-        } catch (UnexpectedException ue) {
-            // expected
-        }
-
-        LOG.info("Actual run fix inprogress segment that has lower sequence number");
-
-        // Actual run
-        DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(namespace,
-                LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(namespace)), streamName, false, false);
-
-        // be able to read more after fix
-        reader = readDLM.getAsyncLogReader(lastDLSN);
-        // skip the first record
-        Await.result(reader.readNext());
-        readFuture = reader.readNext();
-
-        expectedTxId = 51L;
-        LogRecord record = Await.result(readFuture);
-        assertNotNull(record);
-        DLMTestUtil.verifyLogRecord(record);
-        assertEquals(expectedTxId, record.getTransactionId());
-        expectedTxId++;
-
-        for (int i = 1; i < 10; i++) {
-            record = Await.result(reader.readNext());
-            assertNotNull(record);
-            DLMTestUtil.verifyLogRecord(record);
-            assertEquals(expectedTxId, record.getTransactionId());
-            expectedTxId++;
-        }
-
-        Utils.close(reader);
-        readDLM.close();
-
-        dlm.close();
-        namespace.close();
-        readNamespace.close();
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/bk/TestLedgerAllocator.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/bk/TestLedgerAllocator.java
deleted file mode 100644
index 66b97be..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/bk/TestLedgerAllocator.java
+++ /dev/null
@@ -1,389 +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 com.twitter.distributedlog.bk;
-
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.BookKeeperClientBuilder;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.annotations.DistributedLogAnnotations;
-import com.twitter.distributedlog.bk.SimpleLedgerAllocator.AllocationException;
-import com.twitter.distributedlog.bk.SimpleLedgerAllocator.Phase;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.distributedlog.zk.DefaultZKOp;
-import com.twitter.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;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.data.Stat;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Enumeration;
-import java.util.HashSet;
-import java.util.Set;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class TestLedgerAllocator extends TestDistributedLogBase {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestLedgerAllocator.class);
-
-    private static final String ledgersPath = "/ledgers";
-    private static final OpListener<LedgerHandle> NULL_LISTENER = new OpListener<LedgerHandle>() {
-        @Override
-        public void onCommit(LedgerHandle r) {
-            // no-op
-        }
-
-        @Override
-        public void onAbort(Throwable t) {
-            // no-op
-        }
-    };
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    private ZooKeeperClient zkc;
-    private BookKeeperClient bkc;
-    private DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
-
-    private URI createURI(String path) {
-        return URI.create("distributedlog://" + zkServers + path);
-    }
-
-    @Before
-    public void setup() throws Exception {
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .uri(createURI("/"))
-                .zkServers(zkServers)
-                .build();
-        bkc = BookKeeperClientBuilder.newBuilder().name("bkc")
-                .dlConfig(dlConf).ledgersPath(ledgersPath).zkc(zkc).build();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        bkc.close();
-        zkc.close();
-    }
-
-    private QuorumConfigProvider newQuorumConfigProvider(DistributedLogConfiguration conf) {
-        return new ImmutableQuorumConfigProvider(conf.getQuorumConfig());
-    }
-
-    private ZKTransaction newTxn() {
-        return new ZKTransaction(zkc);
-    }
-
-    private SimpleLedgerAllocator createAllocator(String allocationPath) throws IOException {
-        return createAllocator(allocationPath, dlConf);
-    }
-
-    private SimpleLedgerAllocator createAllocator(String allocationPath,
-                                                  DistributedLogConfiguration conf) throws IOException {
-        return FutureUtils.result(SimpleLedgerAllocator.of(allocationPath, null, newQuorumConfigProvider(conf), zkc, bkc));
-    }
-
-    /**
-     * {@link https://issues.apache.org/jira/browse/DL-43}
-     */
-    @DistributedLogAnnotations.FlakyTest
-    @Ignore
-    @Test(timeout = 60000)
-    public void testAllocation() throws Exception {
-        String allocationPath = "/allocation1";
-        SimpleLedgerAllocator allocator = createAllocator(allocationPath);
-        allocator.allocate();
-        ZKTransaction txn = newTxn();
-        LedgerHandle lh = FutureUtils.result(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());
-            fail("Should fail the transaction when setting unexisted path");
-        } catch (ZKException ke) {
-            // expected
-            logger.info("Should fail on executing transaction when setting unexisted path", ke);
-        }
-        data = zkc.get().getData(allocationPath, false, null);
-        assertEquals((Long) lh.getId(), Long.valueOf(new String(data, UTF_8)));
-
-        // 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));
-        assertEquals(lh.getId(), newLh.getId());
-        FutureUtils.result(txn.execute());
-        data = zkc.get().getData(allocationPath, false, null);
-        assertEquals(0, data.length);
-        Utils.close(allocator);
-    }
-
-    @Test(timeout = 60000)
-    public void testBadVersionOnTwoAllocators() throws Exception {
-        String allocationPath = "/allocation-bad-version";
-        zkc.get().create(allocationPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        Stat stat = new Stat();
-        byte[] data = zkc.get().getData(allocationPath, false, stat);
-        Versioned<byte[]> allocationData = new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
-
-        SimpleLedgerAllocator allocator1 =
-                new SimpleLedgerAllocator(allocationPath, allocationData, newQuorumConfigProvider(dlConf), zkc, bkc);
-        SimpleLedgerAllocator allocator2 =
-                new SimpleLedgerAllocator(allocationPath, allocationData, newQuorumConfigProvider(dlConf), zkc, bkc);
-        allocator1.allocate();
-        // wait until allocated
-        ZKTransaction txn1 = newTxn();
-        LedgerHandle lh = FutureUtils.result(allocator1.tryObtain(txn1, NULL_LISTENER));
-        allocator2.allocate();
-        ZKTransaction txn2 = newTxn();
-        try {
-            FutureUtils.result(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());
-        }
-        FutureUtils.result(txn1.execute());
-        Utils.close(allocator1);
-        Utils.close(allocator2);
-
-        long eid = lh.addEntry("hello world".getBytes());
-        lh.close();
-        LedgerHandle readLh = bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes());
-        Enumeration<LedgerEntry> entries = readLh.readEntries(eid, eid);
-        int i = 0;
-        while (entries.hasMoreElements()) {
-            LedgerEntry entry = entries.nextElement();
-            assertEquals("hello world", new String(entry.getEntry(), UTF_8));
-            ++i;
-        }
-        assertEquals(1, i);
-    }
-
-    @Test(timeout = 60000)
-    public void testAllocatorWithoutEnoughBookies() throws Exception {
-        String allocationPath = "/allocator-without-enough-bookies";
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setEnsembleSize(numBookies * 2);
-        confLocal.setWriteQuorumSize(numBookies * 2);
-
-        SimpleLedgerAllocator allocator1 = createAllocator(allocationPath, confLocal);
-        allocator1.allocate();
-        ZKTransaction txn1 = newTxn();
-
-        try {
-            FutureUtils.result(allocator1.tryObtain(txn1, NULL_LISTENER));
-            fail("Should fail allocating ledger if there aren't enough bookies");
-        } catch (AllocationException ioe) {
-            // expected
-            assertEquals(Phase.ERROR, ioe.getPhase());
-        }
-        byte[] data = zkc.get().getData(allocationPath, false, null);
-        assertEquals(0, data.length);
-    }
-
-    @Test(timeout = 60000)
-    public void testSuccessAllocatorShouldDeleteUnusedledger() throws Exception {
-        String allocationPath = "/allocation-delete-unused-ledger";
-        zkc.get().create(allocationPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        Stat stat = new Stat();
-        byte[] data = zkc.get().getData(allocationPath, false, stat);
-
-        Versioned<byte[]> allocationData = new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
-
-        SimpleLedgerAllocator allocator1 =
-                new SimpleLedgerAllocator(allocationPath, allocationData, newQuorumConfigProvider(dlConf), zkc, bkc);
-        allocator1.allocate();
-        // wait until allocated
-        ZKTransaction txn1 = newTxn();
-        LedgerHandle lh1 = FutureUtils.result(allocator1.tryObtain(txn1, NULL_LISTENER));
-
-        // Second allocator kicks in
-        stat = new Stat();
-        data = zkc.get().getData(allocationPath, false, stat);
-        allocationData = new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
-        SimpleLedgerAllocator allocator2 =
-                new SimpleLedgerAllocator(allocationPath, allocationData, newQuorumConfigProvider(dlConf), zkc, bkc);
-        allocator2.allocate();
-        // wait until allocated
-        ZKTransaction txn2 = newTxn();
-        LedgerHandle lh2 = FutureUtils.result(allocator2.tryObtain(txn2, NULL_LISTENER));
-
-        // should fail to commit txn1 as version is changed by second allocator
-        try {
-            FutureUtils.result(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.close(allocator1);
-        Utils.close(allocator2);
-
-        // ledger handle should be deleted
-        try {
-            lh1.close();
-            fail("LedgerHandle allocated by allocator1 should be deleted.");
-        } catch (BKException bke) {
-            // as expected
-        }
-        try {
-            bkc.get().openLedger(lh1.getId(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes());
-            fail("LedgerHandle allocated by allocator1 should be deleted.");
-        } catch (BKException.BKNoSuchLedgerExistsException nslee) {
-            // as expected
-        }
-        long eid = lh2.addEntry("hello world".getBytes());
-        lh2.close();
-        LedgerHandle readLh = bkc.get().openLedger(lh2.getId(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes());
-        Enumeration<LedgerEntry> entries = readLh.readEntries(eid, eid);
-        int i = 0;
-        while (entries.hasMoreElements()) {
-            LedgerEntry entry = entries.nextElement();
-            assertEquals("hello world", new String(entry.getEntry(), UTF_8));
-            ++i;
-        }
-        assertEquals(1, i);
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseAllocatorDuringObtaining() throws Exception {
-        String allocationPath = "/allocation2";
-        SimpleLedgerAllocator allocator = createAllocator(allocationPath);
-        allocator.allocate();
-        ZKTransaction txn = newTxn();
-        // close during obtaining ledger.
-        LedgerHandle lh = FutureUtils.result(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)));
-        // the ledger is not deleted
-        bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32,
-                dlConf.getBKDigestPW().getBytes(UTF_8));
-    }
-
-    /**
-     * {@link https://issues.apache.org/jira/browse/DL-26}
-     */
-    @DistributedLogAnnotations.FlakyTest
-    @Ignore
-    @Test(timeout = 60000)
-    public void testCloseAllocatorAfterConfirm() throws Exception {
-        String allocationPath = "/allocation2";
-        SimpleLedgerAllocator allocator = createAllocator(allocationPath);
-        allocator.allocate();
-        ZKTransaction txn = newTxn();
-        // close during obtaining ledger.
-        LedgerHandle lh = FutureUtils.result(allocator.tryObtain(txn, NULL_LISTENER));
-        FutureUtils.result(txn.execute());
-        Utils.close(allocator);
-        byte[] data = zkc.get().getData(allocationPath, false, null);
-        assertEquals(0, data.length);
-        // the ledger is not deleted.
-        bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32,
-                dlConf.getBKDigestPW().getBytes(UTF_8));
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseAllocatorAfterAbort() throws Exception {
-        String allocationPath = "/allocation3";
-        SimpleLedgerAllocator allocator = createAllocator(allocationPath);
-        allocator.allocate();
-        ZKTransaction txn = newTxn();
-        // close during obtaining ledger.
-        LedgerHandle lh = FutureUtils.result(allocator.tryObtain(txn, NULL_LISTENER));
-        txn.addOp(DefaultZKOp.of(Op.setData("/unexistedpath", "data".getBytes(UTF_8), -1), null));
-        try {
-            FutureUtils.result(txn.execute());
-            fail("Should fail the transaction when setting unexisted path");
-        } catch (ZKException ke) {
-            // expected
-        }
-        Utils.close(allocator);
-        byte[] data = zkc.get().getData(allocationPath, false, null);
-        assertEquals((Long) lh.getId(), Long.valueOf(new String(data, UTF_8)));
-        // the ledger is not deleted.
-        bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32,
-                dlConf.getBKDigestPW().getBytes(UTF_8));
-    }
-
-    @Test(timeout = 60000)
-    public void testConcurrentAllocation() throws Exception {
-        String allcationPath = "/" + runtime.getMethodName();
-        SimpleLedgerAllocator allocator = createAllocator(allcationPath);
-        allocator.allocate();
-        ZKTransaction txn1 = newTxn();
-        Future<LedgerHandle> obtainFuture1 = allocator.tryObtain(txn1, NULL_LISTENER);
-        ZKTransaction txn2 = newTxn();
-        Future<LedgerHandle> obtainFuture2 = allocator.tryObtain(txn2, NULL_LISTENER);
-        assertTrue(obtainFuture2.isDefined());
-        assertTrue(obtainFuture2.isThrow());
-        try {
-            FutureUtils.result(obtainFuture2);
-            fail("Should fail the concurrent obtain since there is already a transaction obtaining the ledger handle");
-        } catch (SimpleLedgerAllocator.ConcurrentObtainException cbe) {
-            // expected
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testObtainMultipleLedgers() throws Exception {
-        String allocationPath = "/" + runtime.getMethodName();
-        SimpleLedgerAllocator allocator = createAllocator(allocationPath);
-        int numLedgers = 10;
-        Set<LedgerHandle> allocatedLedgers = new HashSet<LedgerHandle>();
-        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());
-            allocatedLedgers.add(lh);
-        }
-        assertEquals(numLedgers, allocatedLedgers.size());
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/bk/TestLedgerAllocatorPool.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/bk/TestLedgerAllocatorPool.java
deleted file mode 100644
index 269625f..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/bk/TestLedgerAllocatorPool.java
+++ /dev/null
@@ -1,311 +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 com.twitter.distributedlog.bk;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.BookKeeperClientBuilder;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Transaction.OpListener;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.distributedlog.zk.ZKTransaction;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.zookeeper.data.Stat;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.junit.Assert.*;
-
-public class TestLedgerAllocatorPool extends TestDistributedLogBase {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestLedgerAllocatorPool.class);
-
-    private static final String ledgersPath = "/ledgers";
-    private static final OpListener<LedgerHandle> NULL_LISTENER = new OpListener<LedgerHandle>() {
-        @Override
-        public void onCommit(LedgerHandle r) {
-            // no-op
-        }
-
-        @Override
-        public void onAbort(Throwable t) {
-            // no-op
-        }
-    };
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    private ZooKeeperClient zkc;
-    private BookKeeperClient bkc;
-    private DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
-    private ScheduledExecutorService allocationExecutor;
-
-    private URI createURI(String path) {
-        return URI.create("distributedlog://" + zkServers + path);
-    }
-
-    @Before
-    public void setup() throws Exception {
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .uri(createURI("/"))
-                .build();
-        bkc = BookKeeperClientBuilder.newBuilder().name("bkc")
-                .dlConfig(dlConf).ledgersPath(ledgersPath).zkc(zkc).build();
-        allocationExecutor = Executors.newSingleThreadScheduledExecutor();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        bkc.close();
-        zkc.close();
-        allocationExecutor.shutdown();
-    }
-
-    private ZKTransaction newTxn() {
-        return new ZKTransaction(zkc);
-    }
-
-    private void validatePoolSize(LedgerAllocatorPool pool,
-                                  int pendingSize,
-                                  int allocatingSize,
-                                  int obtainingSize,
-                                  int rescueSize) {
-        assertEquals(pendingSize, pool.pendingListSize());
-        assertEquals(allocatingSize, pool.allocatingListSize());
-        assertEquals(obtainingSize, pool.obtainMapSize());
-        assertEquals(rescueSize, pool.rescueSize());
-    }
-
-    @Test(timeout = 60000)
-    public void testNonAvailableAllocator() throws Exception {
-        String allocationPath = "/nonAvailableAllocator";
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(dlConf);
-        confLocal.setEnsembleSize(2 * numBookies);
-        confLocal.setWriteQuorumSize(2 * numBookies);
-
-        int numAllocators = 3;
-        LedgerAllocatorPool pool =
-                new LedgerAllocatorPool(allocationPath, numAllocators, confLocal, zkc, bkc, allocationExecutor);
-        for (int i = 0; i < numAllocators; i++) {
-            try {
-                pool.allocate();
-                FutureUtils.result(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());
-            }
-        }
-        for (int i = 0; i < numAllocators; i++) {
-            try {
-                pool.allocate();
-                FutureUtils.result(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());
-            } catch (IOException ioe) {
-                // expected
-            }
-        }
-        Utils.close(pool);
-    }
-
-    @Test(timeout = 60000)
-    public void testRescueAllocators() throws Exception {
-        String allocationPath = "/rescueAllocators";
-
-        int numAllocators = 3;
-        LedgerAllocatorPool pool =
-                new LedgerAllocatorPool(allocationPath, numAllocators, dlConf, zkc, bkc, allocationExecutor);
-        List<ZKTransaction> pendingTxns = Lists.newArrayListWithExpectedSize(numAllocators);
-        List<String> allocatePaths = Lists.newArrayListWithExpectedSize(numAllocators);
-        for (int i = 0; i < numAllocators; i++) {
-            ZKTransaction txn = newTxn();
-            pool.allocate();
-            LedgerHandle lh = FutureUtils.result(pool.tryObtain(txn, NULL_LISTENER));
-
-            // get the corresponding ledger allocator
-            SimpleLedgerAllocator sla = pool.getLedgerAllocator(lh);
-            String slaPath = sla.allocatePath;
-
-            logger.info("Allocated ledger {} from path {}", lh.getId(), slaPath);
-
-            pendingTxns.add(txn);
-            allocatePaths.add(slaPath);
-        }
-
-        for (int i = 0; i < numAllocators; i++) {
-            ZKTransaction txn = pendingTxns.get(i);
-            String slaPath = allocatePaths.get(i);
-
-            // execute the transaction to confirm/abort obtain
-            FutureUtils.result(txn.execute());
-
-            // introduce error to individual ledger allocator
-            byte[] data = zkc.get().getData(slaPath, false, new Stat());
-            zkc.get().setData(slaPath, data, -1);
-        }
-        int numSuccess = 0;
-        Set<String> allocatedPathSet = new HashSet<String>();
-        while (numSuccess < 2 * numAllocators) {
-            try {
-                pool.allocate();
-                ZKTransaction txn = newTxn();
-                LedgerHandle lh = FutureUtils.result(pool.tryObtain(txn, NULL_LISTENER));
-
-                // get the corresponding ledger allocator
-                SimpleLedgerAllocator sla = pool.getLedgerAllocator(lh);
-                String slaPath = sla.allocatePath;
-
-                logger.info("Allocated ledger {} from path {}", lh.getId(), slaPath);
-                allocatedPathSet.add(slaPath);
-
-                FutureUtils.result(txn.execute());
-                ++numSuccess;
-            } catch (IOException ioe) {
-                // continue
-            }
-        }
-        assertEquals(2 * numAllocators, numSuccess);
-        assertEquals(numAllocators, allocatedPathSet.size());
-        Utils.close(pool);
-    }
-
-    @Test(timeout = 60000)
-    public void testAllocateWhenNoAllocator() throws Exception {
-        String allocationPath = "/allocateWhenNoAllocator";
-        LedgerAllocatorPool pool = new LedgerAllocatorPool(allocationPath, 0, dlConf, zkc, bkc, allocationExecutor);
-        try {
-            pool.allocate();
-            fail("Should fail to allocate ledger if there isn't allocator.");
-        } catch (SimpleLedgerAllocator.AllocationException ae) {
-            fail("Should fail to allocate ledger if there isn't allocator.");
-        } catch (IOException ioe) {
-            // expected
-        }
-        Utils.close(pool);
-    }
-
-    @Test(timeout = 60000)
-    public void testObtainWhenNoAllocator() throws Exception {
-        String allocationPath = "/obtainWhenNoAllocator";
-        LedgerAllocatorPool pool = new LedgerAllocatorPool(allocationPath, 0, dlConf, zkc, bkc, allocationExecutor);
-        ZKTransaction txn = newTxn();
-        try {
-            FutureUtils.result(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.");
-        } catch (IOException ioe) {
-            // expected.
-        }
-
-        Utils.close(pool);
-    }
-
-    @Test(timeout = 60000)
-    public void testAllocateMultipleLedgers() throws Exception {
-        String allocationPath = "/" + runtime.getMethodName();
-        int numAllocators = 5;
-        final LedgerAllocatorPool pool =
-                new LedgerAllocatorPool(allocationPath, numAllocators, dlConf, zkc, bkc, allocationExecutor);
-        int numLedgers = 20;
-        Set<LedgerHandle> allocatedLedgers = new HashSet<LedgerHandle>();
-        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());
-            allocatedLedgers.add(lh);
-        }
-        assertEquals(numLedgers, allocatedLedgers.size());
-    }
-
-    @Test(timeout = 60000)
-    public void testConcurrentAllocation() throws Exception {
-        final int numAllocators = 5;
-        String allocationPath = "/concurrentAllocation";
-        final LedgerAllocatorPool pool =
-                new LedgerAllocatorPool(allocationPath, numAllocators, dlConf, zkc, bkc, allocationExecutor);
-        final ConcurrentMap<Long, LedgerHandle> allocatedLedgers =
-                new ConcurrentHashMap<Long, LedgerHandle>();
-        final AtomicInteger numFailures = new AtomicInteger(0);
-        Thread[] allocationThreads = new Thread[numAllocators];
-        for (int i = 0; i < numAllocators; i++) {
-            final int tid = i;
-            allocationThreads[i] = new Thread() {
-
-                int numLedgers = 50;
-
-                @Override
-                public void run() {
-                    try {
-                        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());
-                            lh.close();
-                            allocatedLedgers.putIfAbsent(lh.getId(), lh);
-                            logger.info("[thread {}] allocate {}th ledger {}",
-                                    new Object[] { tid, i, lh.getId() });
-                        }
-                    } catch (Exception ioe) {
-                        numFailures.incrementAndGet();
-                    }
-                }
-            };
-        }
-
-        for (Thread t : allocationThreads) {
-            t.start();
-        }
-
-        for (Thread t : allocationThreads) {
-            t.join();
-        }
-
-        assertEquals(0, numFailures.get());
-        assertEquals(50 * numAllocators, allocatedLedgers.size());
-
-        Utils.close(pool);
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/config/PropertiesWriter.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/config/PropertiesWriter.java
deleted file mode 100644
index 6626b3b..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/config/PropertiesWriter.java
+++ /dev/null
@@ -1,67 +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 com.twitter.distributedlog.config;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.util.Properties;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-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;
-    }
-}
\ No newline at end of file
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/config/TestConcurrentBaseConfiguration.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/config/TestConcurrentBaseConfiguration.java
deleted file mode 100644
index 33f10e4..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/config/TestConcurrentBaseConfiguration.java
+++ /dev/null
@@ -1,46 +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 com.twitter.distributedlog.config;
-
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.junit.Assert.*;
-
-public class TestConcurrentBaseConfiguration {
-    static final Logger LOG = LoggerFactory.getLogger(TestConcurrentBaseConfiguration.class);
-
-    @Test(timeout = 20000)
-    public void testBasicOperations() throws Exception {
-        ConcurrentBaseConfiguration conf = new ConcurrentBaseConfiguration();
-        conf.setProperty("prop1", "1");
-        assertEquals(1, conf.getInt("prop1"));
-        conf.setProperty("prop1", "2");
-        assertEquals(2, conf.getInt("prop1"));
-        conf.clearProperty("prop1");
-        assertEquals(null, conf.getInteger("prop1", null));
-        conf.setProperty("prop1", "1");
-        conf.setProperty("prop2", "2");
-        assertEquals(1, conf.getInt("prop1"));
-        assertEquals(2, conf.getInt("prop2"));
-        conf.clearProperty("prop1");
-        assertEquals(null, conf.getInteger("prop1", null));
-        assertEquals(2, conf.getInt("prop2"));
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/config/TestConfigurationSubscription.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/config/TestConfigurationSubscription.java
deleted file mode 100644
index 24733a4..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/config/TestConfigurationSubscription.java
+++ /dev/null
@@ -1,171 +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 com.twitter.distributedlog.config;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import org.apache.commons.configuration.event.ConfigurationEvent;
-import org.apache.commons.configuration.event.ConfigurationListener;
-import org.jmock.lib.concurrent.DeterministicScheduler;
-import org.junit.Test;
-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
- * get picked up.
- */
-public class TestConfigurationSubscription {
-    static final Logger LOG = LoggerFactory.getLogger(TestConfigurationSubscription.class);
-
-    /**
-     * Give FileChangedReloadingStrategy some time to start reloading
-     * Make sure now!=lastChecked
-     * {@link org.apache.commons.configuration.reloading.FileChangedReloadingStrategy#reloadingRequired()}
-     */
-    private void ensureConfigReloaded() throws InterruptedException {
-        // sleep 1 ms so that System.currentTimeMillis() !=
-        // lastChecked (the time we construct FileChangedReloadingStrategy
-        Thread.sleep(1);
-    }
-
-    @Test(timeout = 60000)
-    public void testReloadConfiguration() throws Exception {
-        PropertiesWriter writer = new PropertiesWriter();
-        FileConfigurationBuilder builder = new PropertiesConfigurationBuilder(writer.getFile().toURI().toURL());
-        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
-        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 com.twitter.distributedlog.config.ConfigurationListener() {
-            @Override
-            public void onReload(ConcurrentBaseConfiguration conf) {
-                confHolder.set(conf);
-            }
-        });
-        assertEquals(null, conf.getProperty("prop1"));
-
-        // add
-        writer.setProperty("prop1", "1");
-        writer.save();
-        // ensure the file change reloading event can be triggered
-        ensureConfigReloaded();
-        // reload the config
-        confSub.reload();
-        assertNotNull(confHolder.get());
-        assertTrue(conf == confHolder.get());
-        assertEquals("1", conf.getProperty("prop1"));
-    }
-
-    @Test(timeout = 60000)
-    public void testAddReloadBasicsConfig() throws Exception {
-        PropertiesWriter writer = new PropertiesWriter();
-        DeterministicScheduler mockScheduler = new DeterministicScheduler();
-        FileConfigurationBuilder builder = new PropertiesConfigurationBuilder(writer.getFile().toURI().toURL());
-        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
-        List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(builder);
-        ConfigurationSubscription confSub =
-                new ConfigurationSubscription(conf, fileConfigBuilders, mockScheduler, 100, TimeUnit.MILLISECONDS);
-        assertEquals(null, conf.getProperty("prop1"));
-
-        // add
-        writer.setProperty("prop1", "1");
-        writer.save();
-        // ensure the file change reloading event can be triggered
-        ensureConfigReloaded();
-        mockScheduler.tick(100, TimeUnit.MILLISECONDS);
-        assertEquals("1", conf.getProperty("prop1"));
-
-    }
-
-    @Test(timeout = 60000)
-    public void testInitialConfigLoad() throws Exception {
-        PropertiesWriter writer = new PropertiesWriter();
-        writer.setProperty("prop1", "1");
-        writer.setProperty("prop2", "abc");
-        writer.setProperty("prop3", "123.0");
-        writer.setProperty("prop4", "11132");
-        writer.setProperty("prop5", "true");
-        writer.save();
-
-        ScheduledExecutorService mockScheduler = new DeterministicScheduler();
-        FileConfigurationBuilder builder = new PropertiesConfigurationBuilder(writer.getFile().toURI().toURL());
-        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
-        List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(builder);
-        ConfigurationSubscription confSub =
-                new ConfigurationSubscription(conf, fileConfigBuilders, mockScheduler, 100, TimeUnit.MILLISECONDS);
-        assertEquals(1, conf.getInt("prop1"));
-        assertEquals("abc", conf.getString("prop2"));
-        assertEquals(123.0, conf.getFloat("prop3"), 0);
-        assertEquals(11132, conf.getInt("prop4"));
-        assertEquals(true, conf.getBoolean("prop5"));
-    }
-
-    @Test(timeout = 60000)
-    public void testExceptionInConfigLoad() throws Exception {
-        PropertiesWriter writer = new PropertiesWriter();
-        writer.setProperty("prop1", "1");
-        writer.save();
-
-        DeterministicScheduler mockScheduler = new DeterministicScheduler();
-        FileConfigurationBuilder builder = new PropertiesConfigurationBuilder(writer.getFile().toURI().toURL());
-        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
-        List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(builder);
-        ConfigurationSubscription confSub =
-                new ConfigurationSubscription(conf, fileConfigBuilders, mockScheduler, 100, TimeUnit.MILLISECONDS);
-
-        final AtomicInteger count = new AtomicInteger(1);
-        conf.addConfigurationListener(new ConfigurationListener() {
-            @Override
-            public void configurationChanged(ConfigurationEvent event) {
-                LOG.info("config changed {}", event);
-                // Throw after so we actually see the update anyway.
-                if (!event.isBeforeUpdate()) {
-                    count.getAndIncrement();
-                    throw new RuntimeException("config listener threw and exception");
-                }
-            }
-        });
-
-        int i = 0;
-        int initial = 0;
-        while (count.get() == initial) {
-            writer.setProperty("prop1", Integer.toString(i++));
-            writer.save();
-            mockScheduler.tick(100, TimeUnit.MILLISECONDS);
-        }
-
-        initial = count.get();
-        while (count.get() == initial) {
-            writer.setProperty("prop1", Integer.toString(i++));
-            writer.save();
-            mockScheduler.tick(100, TimeUnit.MILLISECONDS);
-        }
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/config/TestDynamicConfigurationFactory.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/config/TestDynamicConfigurationFactory.java
deleted file mode 100644
index 7ac8ae2..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/config/TestDynamicConfigurationFactory.java
+++ /dev/null
@@ -1,101 +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 com.twitter.distributedlog.config;
-
-import com.google.common.base.Objects;
-import com.google.common.base.Optional;
-
-import com.twitter.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.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.junit.Assert.*;
-
-public class TestDynamicConfigurationFactory {
-    static final Logger LOG = LoggerFactory.getLogger(TestDynamicConfigurationFactory.class);
-
-    private void waitForConfig(DynamicDistributedLogConfiguration conf, int value) throws Exception {
-        while (!Objects.equal(conf.getRetentionPeriodHours(), value)) {
-            Thread.sleep(100);
-        }
-    }
-
-    private DynamicConfigurationFactory getConfigFactory(File configFile) {
-        String streamConfigPath = configFile.getParent();
-        ScheduledExecutorService executorService = new ScheduledThreadPoolExecutor(1);
-        ConcurrentBaseConfiguration defaultConf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
-        return new DynamicConfigurationFactory(executorService, 100, TimeUnit.MILLISECONDS);
-    }
-
-    private String getNamePart(File configFile) {
-        String propsFilename = configFile.getName();
-        return propsFilename.substring(0, propsFilename.indexOf(".conf"));
-    }
-
-    @Test(timeout = 60000)
-    public void testGetDynamicConfigBasics() throws Exception {
-        PropertiesWriter writer = new PropertiesWriter();
-        DynamicConfigurationFactory factory = getConfigFactory(writer.getFile());
-        Optional<DynamicDistributedLogConfiguration> conf = factory.getDynamicConfiguration(writer.getFile().getPath());
-        assertEquals(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT, conf.get().getRetentionPeriodHours());
-        writer.setProperty(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS, "1");
-        writer.save();
-        waitForConfig(conf.get(), 1);
-        assertEquals(1, conf.get().getRetentionPeriodHours());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetDynamicConfigIsSingleton() throws Exception {
-        PropertiesWriter writer = new PropertiesWriter();
-        DynamicConfigurationFactory factory = getConfigFactory(writer.getFile());
-        String configPath = writer.getFile().getPath();
-        Optional<DynamicDistributedLogConfiguration> conf1 = factory.getDynamicConfiguration(configPath);
-        Optional<DynamicDistributedLogConfiguration> conf2 = factory.getDynamicConfiguration(configPath);
-        assertEquals(conf1, conf2);
-    }
-
-    /**
-     * If the file is missing, get-config should not fail, and the file should be picked up if its added.
-     * If the file is removed externally same should apply.
-     */
-    @Test(timeout = 60000)
-    public void testMissingConfig() throws Exception {
-        PropertiesWriter writer = new PropertiesWriter();
-        DynamicConfigurationFactory factory = getConfigFactory(writer.getFile());
-        Optional<DynamicDistributedLogConfiguration> conf = factory.getDynamicConfiguration(writer.getFile().getPath());
-        writer.setProperty(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS, "1");
-        writer.save();
-        waitForConfig(conf.get(), 1);
-        File configFile = writer.getFile();
-        configFile.delete();
-        Thread.sleep(1000);
-        PropertiesWriter writer2 = new PropertiesWriter(writer.getFile());
-        writer2.setProperty(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS, "2");
-        writer2.save();
-        waitForConfig(conf.get(), 2);
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/config/TestDynamicDistributedLogConfiguration.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/config/TestDynamicDistributedLogConfiguration.java
deleted file mode 100644
index 2f78ec6..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/config/TestDynamicDistributedLogConfiguration.java
+++ /dev/null
@@ -1,261 +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 com.twitter.distributedlog.config;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-
-import com.twitter.distributedlog.bk.QuorumConfig;
-import org.junit.Test;
-
-import static com.twitter.distributedlog.DistributedLogConfiguration.*;
-import static org.junit.Assert.*;
-
-public class TestDynamicDistributedLogConfiguration {
-
-    @Test(timeout = 20000)
-    public void testDefaults() throws Exception {
-        // Default config defines retention period plus two other params, but eaves ack quorum unspecified
-        DistributedLogConfiguration underlyingConfig = new DistributedLogConfiguration();
-        underlyingConfig.setRetentionPeriodHours(99);
-        underlyingConfig.setProperty("rpsHardWriteLimit", 99);
-
-        ConcurrentConstConfiguration defaultConfig = new ConcurrentConstConfiguration(underlyingConfig);
-        DynamicDistributedLogConfiguration config = new DynamicDistributedLogConfiguration(defaultConfig);
-        assertEquals(99, config.getRetentionPeriodHours());
-        assertEquals(99, config.getRpsHardWriteLimit());
-        config.setProperty(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS, 5);
-
-        // Config checks primary then secondary then const defaults
-        assertEquals(5, config.getRetentionPeriodHours());
-        assertEquals(99, config.getRpsHardWriteLimit());
-    }
-
-    @Test(timeout = 20000)
-    public void testGetRetentionPeriodHours() {
-        ConcurrentBaseConfiguration defaultConfig = new ConcurrentBaseConfiguration();
-        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(defaultConfig);
-        // get default value
-        assertEquals(BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT, dynConf.getRetentionPeriodHours());
-        // get value from old key of default config
-        defaultConfig.setProperty(BKDL_RETENTION_PERIOD_IN_HOURS_OLD, BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 1);
-        assertEquals(BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 1, dynConf.getRetentionPeriodHours());
-        // get value from new key of default config
-        defaultConfig.setProperty(BKDL_RETENTION_PERIOD_IN_HOURS, BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT + 2);
-        assertEquals(BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 2, dynConf.getRetentionPeriodHours());
-        // get value from old key of dynamic config
-        dynConf.setProperty(BKDL_RETENTION_PERIOD_IN_HOURS_OLD, BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 3);
-        assertEquals(BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 3, dynConf.getRetentionPeriodHours());
-        // get value from new key of default config
-        dynConf.setProperty(BKDL_RETENTION_PERIOD_IN_HOURS, BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 4);
-        assertEquals(BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 4, dynConf.getRetentionPeriodHours());
-    }
-
-    @Test(timeout = 20000)
-    public void testGetOutputBufferSize() {
-        ConcurrentBaseConfiguration defaultConfig = new ConcurrentBaseConfiguration();
-        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(defaultConfig);
-        // get default value
-        assertEquals(BKDL_OUTPUT_BUFFER_SIZE_DEFAULT, dynConf.getOutputBufferSize());
-        // get value from old key of default config
-        defaultConfig.setProperty(BKDL_OUTPUT_BUFFER_SIZE_OLD, BKDL_OUTPUT_BUFFER_SIZE_DEFAULT + 1);
-        assertEquals(BKDL_OUTPUT_BUFFER_SIZE_DEFAULT  + 1, dynConf.getOutputBufferSize());
-        // get value from new key of default config
-        defaultConfig.setProperty(BKDL_OUTPUT_BUFFER_SIZE, BKDL_OUTPUT_BUFFER_SIZE_DEFAULT + 2);
-        assertEquals(BKDL_OUTPUT_BUFFER_SIZE_DEFAULT  + 2, dynConf.getOutputBufferSize());
-        // get value from old key of dynamic config
-        dynConf.setProperty(BKDL_OUTPUT_BUFFER_SIZE_OLD, BKDL_OUTPUT_BUFFER_SIZE_DEFAULT  + 3);
-        assertEquals(BKDL_OUTPUT_BUFFER_SIZE_DEFAULT  + 3, dynConf.getOutputBufferSize());
-        // get value from new key of default config
-        dynConf.setProperty(BKDL_OUTPUT_BUFFER_SIZE, BKDL_OUTPUT_BUFFER_SIZE_DEFAULT  + 4);
-        assertEquals(BKDL_OUTPUT_BUFFER_SIZE_DEFAULT  + 4, dynConf.getOutputBufferSize());
-    }
-
-    @Test(timeout = 20000)
-    public void testGetReadAheadBatchSize() {
-        ConcurrentBaseConfiguration defaultConfig = new ConcurrentBaseConfiguration();
-        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(defaultConfig);
-        // get default value
-        assertEquals(BKDL_READAHEAD_BATCHSIZE_DEFAULT, dynConf.getReadAheadBatchSize());
-        // get value from old key of default config
-        defaultConfig.setProperty(BKDL_READAHEAD_BATCHSIZE_OLD, BKDL_READAHEAD_BATCHSIZE_DEFAULT + 1);
-        assertEquals(BKDL_READAHEAD_BATCHSIZE_DEFAULT  + 1, dynConf.getReadAheadBatchSize());
-        // get value from new key of default config
-        defaultConfig.setProperty(BKDL_READAHEAD_BATCHSIZE, BKDL_READAHEAD_BATCHSIZE_DEFAULT + 2);
-        assertEquals(BKDL_READAHEAD_BATCHSIZE_DEFAULT  + 2, dynConf.getReadAheadBatchSize());
-        // get value from old key of dynamic config
-        dynConf.setProperty(BKDL_READAHEAD_BATCHSIZE_OLD, BKDL_READAHEAD_BATCHSIZE_DEFAULT  + 3);
-        assertEquals(BKDL_READAHEAD_BATCHSIZE_DEFAULT  + 3, dynConf.getReadAheadBatchSize());
-        // get value from new key of default config
-        dynConf.setProperty(BKDL_READAHEAD_BATCHSIZE, BKDL_READAHEAD_BATCHSIZE_DEFAULT  + 4);
-        assertEquals(BKDL_READAHEAD_BATCHSIZE_DEFAULT  + 4, dynConf.getReadAheadBatchSize());
-    }
-
-    @Test(timeout = 20000)
-    public void testGetReadAheadMaxRecords() {
-        ConcurrentBaseConfiguration defaultConfig = new ConcurrentBaseConfiguration();
-        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(defaultConfig);
-        // get default value
-        assertEquals(BKDL_READAHEAD_MAX_RECORDS_DEFAULT, dynConf.getReadAheadMaxRecords());
-        // get value from old key of default config
-        defaultConfig.setProperty(BKDL_READAHEAD_MAX_RECORDS_OLD, BKDL_READAHEAD_MAX_RECORDS_DEFAULT + 1);
-        assertEquals(BKDL_READAHEAD_MAX_RECORDS_DEFAULT  + 1, dynConf.getReadAheadMaxRecords());
-        // get value from new key of default config
-        defaultConfig.setProperty(BKDL_READAHEAD_MAX_RECORDS, BKDL_READAHEAD_MAX_RECORDS_DEFAULT + 2);
-        assertEquals(BKDL_READAHEAD_MAX_RECORDS_DEFAULT  + 2, dynConf.getReadAheadMaxRecords());
-        // get value from old key of dynamic config
-        dynConf.setProperty(BKDL_READAHEAD_MAX_RECORDS_OLD, BKDL_READAHEAD_MAX_RECORDS_DEFAULT  + 3);
-        assertEquals(BKDL_READAHEAD_MAX_RECORDS_DEFAULT  + 3, dynConf.getReadAheadMaxRecords());
-        // get value from new key of default config
-        dynConf.setProperty(BKDL_READAHEAD_MAX_RECORDS, BKDL_READAHEAD_MAX_RECORDS_DEFAULT  + 4);
-        assertEquals(BKDL_READAHEAD_MAX_RECORDS_DEFAULT  + 4, dynConf.getReadAheadMaxRecords());
-    }
-
-    void assertQuorumConfig(QuorumConfig config,
-                            int expectedEnsembleSize,
-                            int expectedWriteQuorumSize,
-                            int expectedAckQuorumSize) {
-        assertEquals(expectedEnsembleSize, config.getEnsembleSize());
-        assertEquals(expectedWriteQuorumSize, config.getWriteQuorumSize());
-        assertEquals(expectedAckQuorumSize, config.getAckQuorumSize());
-    }
-
-    @Test(timeout = 20000)
-    public void testGetQuorumConfig() {
-        ConcurrentBaseConfiguration defaultConfig = new ConcurrentBaseConfiguration();
-        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(defaultConfig);
-        // get default value
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
-
-        // Test Ensemble Size
-
-        // get value from old key of default config
-        defaultConfig.setProperty(BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD, BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 1);
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 1,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
-        // get value from new key of default config
-        defaultConfig.setProperty(BKDL_BOOKKEEPER_ENSEMBLE_SIZE, BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 2);
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 2,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
-        // get value from old key of dynamic config
-        dynConf.setProperty(BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD, BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 3);
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 3,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
-        // get value from new key of dynamic config
-        dynConf.setProperty(BKDL_BOOKKEEPER_ENSEMBLE_SIZE, BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4);
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
-
-        // Test Write Quorum Size
-
-        // get value from old key of default config
-        defaultConfig.setProperty(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 1);
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 1,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
-        // get value from new key of default config
-        defaultConfig.setProperty(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 2);
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 2,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
-        // get value from old key of dynamic config
-        dynConf.setProperty(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 3);
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 3,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
-        // get value from new key of dynamic config
-        dynConf.setProperty(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 4);
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
-
-        // Test Ack Quorum Size
-
-        // get value from old key of default config
-        defaultConfig.setProperty(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 1);
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 1);
-        // get value from new key of default config
-        defaultConfig.setProperty(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 2);
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 2);
-        // get value from old key of dynamic config
-        dynConf.setProperty(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 3);
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 3);
-        // get value from new key of dynamic config
-        dynConf.setProperty(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 4);
-        assertQuorumConfig(
-                dynConf.getQuorumConfig(),
-                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 4,
-                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 4);
-    }
-
-    @Test(timeout = 20000)
-    public void testIsDurableWriteEnabled() {
-        ConcurrentBaseConfiguration defaultConfig = new ConcurrentBaseConfiguration();
-        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(defaultConfig);
-
-        assertTrue(dynConf.isDurableWriteEnabled());
-        defaultConfig.setProperty(BKDL_IS_DURABLE_WRITE_ENABLED, false);
-        assertFalse(dynConf.isDurableWriteEnabled());
-        dynConf.setProperty(BKDL_IS_DURABLE_WRITE_ENABLED, true);
-        assertTrue(dynConf.isDurableWriteEnabled());
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/feature/TestConfigurationFeatureProvider.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/feature/TestConfigurationFeatureProvider.java
deleted file mode 100644
index 7fa57ec..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/feature/TestConfigurationFeatureProvider.java
+++ /dev/null
@@ -1,65 +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 com.twitter.distributedlog.feature;
-
-import com.twitter.distributedlog.config.ConcurrentBaseConfiguration;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.junit.Test;
-
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-import static org.junit.Assert.*;
-
-/**
- * Test case for configuration based feature provider
- */
-public class TestConfigurationFeatureProvider {
-
-    @Test(timeout = 60000)
-    public void testConfigurationFeatureProvider() throws Exception {
-        String rootScope = "dl";
-        ConcurrentBaseConfiguration featureConf =
-                new ConcurrentBaseConfiguration();
-        ConcurrentMap<String, SettableFeature> features =
-                new ConcurrentHashMap<String, SettableFeature>();
-        ConfigurationFeatureProvider featureProvider =
-                new ConfigurationFeatureProvider(rootScope, featureConf, features);
-
-        String featureName1 = "feature1";
-        String fullFeatureName1 = rootScope + "." + featureName1;
-        int availability1 = 1234;
-        featureConf.setProperty(fullFeatureName1, availability1);
-        Feature feature1 = featureProvider.getFeature(featureName1);
-        assertEquals(availability1, feature1.availability());
-        assertTrue(features.containsKey(fullFeatureName1));
-        assertTrue(feature1 == features.get(fullFeatureName1));
-
-        String subScope = "subscope";
-        String featureName2 = "feature2";
-        String fullFeatureName2 = rootScope + "." + subScope + "." + featureName2;
-        int availability2 = 4321;
-        featureConf.setProperty(fullFeatureName2, availability2);
-        Feature feature2 = featureProvider.scope(subScope).getFeature(featureName2);
-        assertEquals(availability2, feature2.availability());
-        assertTrue(features.containsKey(fullFeatureName2));
-        assertTrue(feature2 == features.get(fullFeatureName2));
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java
deleted file mode 100644
index fc82b6e..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java
+++ /dev/null
@@ -1,183 +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 com.twitter.distributedlog.feature;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.annotations.DistributedLogAnnotations;
-import com.twitter.distributedlog.config.PropertiesWriter;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.junit.Ignore;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-/**
- * Test case for dynamic configuration based feature provider
- */
-public class TestDynamicConfigurationFeatureProvider {
-
-    /**
-     * Make sure config is reloaded
-     *
-     * Give FileChangedReloadingStrategy some time to allow reloading
-     * Make sure now!=lastChecked
-     * {@link org.apache.commons.configuration.reloading.FileChangedReloadingStrategy#reloadingRequired()}
-     */
-    private void ensureConfigReloaded() throws InterruptedException {
-        // sleep 1 ms so that System.currentTimeMillis() !=
-        // lastChecked (the time we construct FileChangedReloadingStrategy
-        Thread.sleep(1);
-    }
-
-    @Test(timeout = 60000)
-    public void testLoadFeaturesFromBase() throws Exception {
-        PropertiesWriter writer = new PropertiesWriter();
-        writer.setProperty("feature_1", "10000");
-        writer.setProperty("feature_2", "5000");
-        writer.save();
-
-        DistributedLogConfiguration conf = new DistributedLogConfiguration()
-                .setDynamicConfigReloadIntervalSec(Integer.MAX_VALUE)
-                .setFileFeatureProviderBaseConfigPath(writer.getFile().toURI().toURL().getPath());
-        DynamicConfigurationFeatureProvider provider =
-                new DynamicConfigurationFeatureProvider("", conf, NullStatsLogger.INSTANCE);
-        provider.start();
-        ensureConfigReloaded();
-
-        Feature feature1 = provider.getFeature("feature_1");
-        assertTrue(feature1.isAvailable());
-        assertEquals(10000, feature1.availability());
-        Feature feature2 = provider.getFeature("feature_2");
-        assertTrue(feature2.isAvailable());
-        assertEquals(5000, feature2.availability());
-        Feature feature3 = provider.getFeature("feature_3");
-        assertFalse(feature3.isAvailable());
-        assertEquals(0, feature3.availability());
-        Feature feature4 = provider.getFeature("unknown_feature");
-        assertFalse(feature4.isAvailable());
-        assertEquals(0, feature4.availability());
-
-        provider.stop();
-    }
-
-    /**
-     * {@link https://issues.apache.org/jira/browse/DL-40}
-     */
-    @DistributedLogAnnotations.FlakyTest
-    @Ignore
-    @Test(timeout = 60000)
-    public void testLoadFeaturesFromOverlay() throws Exception {
-        PropertiesWriter writer = new PropertiesWriter();
-        writer.setProperty("feature_1", "10000");
-        writer.setProperty("feature_2", "5000");
-        writer.save();
-
-        PropertiesWriter overlayWriter = new PropertiesWriter();
-        overlayWriter.setProperty("feature_2", "6000");
-        overlayWriter.setProperty("feature_4", "6000");
-        overlayWriter.save();
-
-        DistributedLogConfiguration conf = new DistributedLogConfiguration()
-                .setDynamicConfigReloadIntervalSec(Integer.MAX_VALUE)
-                .setFileFeatureProviderBaseConfigPath(writer.getFile().toURI().toURL().getPath())
-                .setFileFeatureProviderOverlayConfigPath(overlayWriter.getFile().toURI().toURL().getPath());
-        DynamicConfigurationFeatureProvider provider =
-                new DynamicConfigurationFeatureProvider("", conf, NullStatsLogger.INSTANCE);
-        provider.start();
-        ensureConfigReloaded();
-
-        Feature feature1 = provider.getFeature("feature_1");
-        assertTrue(feature1.isAvailable());
-        assertEquals(10000, feature1.availability());
-        Feature feature2 = provider.getFeature("feature_2");
-        assertTrue(feature2.isAvailable());
-        assertEquals(6000, feature2.availability());
-        Feature feature3 = provider.getFeature("feature_3");
-        assertFalse(feature3.isAvailable());
-        assertEquals(0, feature3.availability());
-        Feature feature4 = provider.getFeature("feature_4");
-        assertTrue(feature4.isAvailable());
-        assertEquals(6000, feature4.availability());
-        Feature feature5 = provider.getFeature("unknown_feature");
-        assertFalse(feature5.isAvailable());
-        assertEquals(0, feature5.availability());
-
-        provider.stop();
-    }
-
-    @Test(timeout = 60000)
-    public void testReloadFeaturesFromOverlay() throws Exception {
-        PropertiesWriter writer = new PropertiesWriter();
-        writer.setProperty("feature_1", "10000");
-        writer.setProperty("feature_2", "5000");
-        writer.save();
-
-        PropertiesWriter overlayWriter = new PropertiesWriter();
-        overlayWriter.setProperty("feature_2", "6000");
-        overlayWriter.setProperty("feature_4", "6000");
-        overlayWriter.save();
-
-        DistributedLogConfiguration conf = new DistributedLogConfiguration()
-                .setDynamicConfigReloadIntervalSec(Integer.MAX_VALUE)
-                .setFileFeatureProviderBaseConfigPath(writer.getFile().toURI().toURL().getPath())
-                .setFileFeatureProviderOverlayConfigPath(overlayWriter.getFile().toURI().toURL().getPath());
-        DynamicConfigurationFeatureProvider provider =
-                new DynamicConfigurationFeatureProvider("", conf, NullStatsLogger.INSTANCE);
-        provider.start();
-        ensureConfigReloaded();
-
-        Feature feature1 = provider.getFeature("feature_1");
-        assertTrue(feature1.isAvailable());
-        assertEquals(10000, feature1.availability());
-        Feature feature2 = provider.getFeature("feature_2");
-        assertTrue(feature2.isAvailable());
-        assertEquals(6000, feature2.availability());
-        Feature feature3 = provider.getFeature("feature_3");
-        assertFalse(feature3.isAvailable());
-        assertEquals(0, feature3.availability());
-        Feature feature4 = provider.getFeature("feature_4");
-        assertTrue(feature4.isAvailable());
-        assertEquals(6000, feature4.availability());
-        Feature feature5 = provider.getFeature("unknown_feature");
-        assertFalse(feature5.isAvailable());
-        assertEquals(0, feature5.availability());
-
-        // dynamic load config
-        provider.getFeatureConf().setProperty("feature_1", 3000);
-        provider.getFeatureConf().setProperty("feature_2", 7000);
-        provider.getFeatureConf().setProperty("feature_3", 8000);
-        provider.getFeatureConf().setProperty("feature_4", 9000);
-        provider.onReload(provider.getFeatureConf());
-        feature1 = provider.getFeature("feature_1");
-        assertTrue(feature1.isAvailable());
-        assertEquals(3000, feature1.availability());
-        feature2 = provider.getFeature("feature_2");
-        assertTrue(feature2.isAvailable());
-        assertEquals(7000, feature2.availability());
-        feature3 = provider.getFeature("feature_3");
-        assertTrue(feature3.isAvailable());
-        assertEquals(8000, feature3.availability());
-        feature4 = provider.getFeature("feature_4");
-        assertTrue(feature4.isAvailable());
-        assertEquals(9000, feature4.availability());
-
-        provider.stop();
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogMetadataStore.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogMetadataStore.java
deleted file mode 100644
index e09de3e..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogMetadataStore.java
+++ /dev/null
@@ -1,114 +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 com.twitter.distributedlog.impl;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Utils;
-import org.apache.zookeeper.CreateMode;
-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 java.net.URI;
-import java.util.Set;
-
-import static org.junit.Assert.*;
-
-/**
- * Test ZK based metadata store.
- */
-public class TestZKLogMetadataStore extends TestDistributedLogBase {
-
-    private final static int zkSessionTimeoutMs = 2000;
-
-    @Rule
-    public TestName runtime = new TestName();
-    protected final DistributedLogConfiguration baseConf =
-            new DistributedLogConfiguration();
-    protected ZooKeeperClient zkc;
-    protected ZKLogMetadataStore metadataStore;
-    protected OrderedScheduler scheduler;
-    protected URI uri;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .uri(createDLMURI("/"))
-                .sessionTimeoutMs(zkSessionTimeoutMs)
-                .build();
-        scheduler = OrderedScheduler.newBuilder()
-                .name("test-zk-logmetadata-store")
-                .corePoolSize(1)
-                .build();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.addConfiguration(baseConf);
-        this.uri = createDLMURI("/" + runtime.getMethodName());
-        metadataStore = new ZKLogMetadataStore(conf, uri, zkc, scheduler);
-    }
-
-    @After
-    public void teardown() throws Exception {
-        if (null != zkc) {
-            zkc.close();
-        }
-        if (null != scheduler) {
-            scheduler.shutdown();
-        }
-    }
-
-    private void createLogInNamespace(URI uri, String logName) throws Exception {
-        String logPath = uri.getPath() + "/" + logName;
-        Utils.zkCreateFullPathOptimistic(zkc, logPath, new byte[0],
-                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLog() throws Exception {
-        assertEquals(uri, FutureUtils.result(metadataStore.createLog("test")));
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogLocation() throws Exception {
-        Optional<URI> uriOptional = FutureUtils.result(metadataStore.getLogLocation("test"));
-        assertTrue(uriOptional.isPresent());
-        assertEquals(uri, uriOptional.get());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogs() throws Exception {
-        Set<String> logs = Sets.newHashSet();
-        for (int i = 0; i < 10; i++) {
-            String logName = "test-" + i;
-            logs.add(logName);
-            createLogInNamespace(uri, logName);
-        }
-        Set<String> result = Sets.newHashSet(FutureUtils.result(metadataStore.getLogs()));
-        assertEquals(10, result.size());
-        assertTrue(Sets.difference(logs, result).isEmpty());
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentFilters.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentFilters.java
deleted file mode 100644
index 52b1485..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentFilters.java
+++ /dev/null
@@ -1,74 +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 com.twitter.distributedlog.impl;
-
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.DistributedLogConstants;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import static com.twitter.distributedlog.impl.ZKLogSegmentFilters.*;
-import static org.junit.Assert.*;
-
-public class TestZKLogSegmentFilters {
-
-    static final Logger LOG = LoggerFactory.getLogger(TestZKLogSegmentFilters.class);
-
-    @Test(timeout = 60000)
-    public void testWriteFilter() {
-        Set<String> expectedFilteredSegments = new HashSet<String>();
-        List<String> segments = new ArrayList<String>();
-        for (int i = 1; i <= 5; i++) {
-            segments.add(DLMTestUtil.completedLedgerZNodeNameWithVersion(i, (i - 1) * 100, i * 100 - 1, i));
-        }
-        for (int i = 6; i <= 10; i++) {
-            String segmentName = DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i);
-            segments.add(segmentName);
-            if (i == 10) {
-                expectedFilteredSegments.add(segmentName);
-            }
-        }
-        for (int i = 11; i <= 15; i++) {
-            String segmentName = DLMTestUtil.completedLedgerZNodeNameWithTxID((i - 1) * 100, i * 100 - 1);
-            segments.add(segmentName);
-            expectedFilteredSegments.add(segmentName);
-        }
-        segments.add("");
-        segments.add("unknown");
-        segments.add(DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX + "_1234_5678_9");
-        expectedFilteredSegments.add(DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX + "_1234_5678_9");
-        segments.add(DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX + "_1_2_3_4_5_6_7_8_9");
-        expectedFilteredSegments.add(DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX + "_1_2_3_4_5_6_7_8_9");
-
-        Collection<String> filteredCollection = WRITE_HANDLE_FILTER.filter(segments);
-        LOG.info("Filter log segments {} to {}.", segments, filteredCollection);
-        assertEquals(expectedFilteredSegments.size(), filteredCollection.size());
-
-        Set<String> filteredSegments = Sets.newHashSet(filteredCollection);
-        Sets.SetView<String> diff = Sets.difference(filteredSegments, expectedFilteredSegments);
-        assertTrue(diff.isEmpty());
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentMetadataStore.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentMetadataStore.java
deleted file mode 100644
index de7016a..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKLogSegmentMetadataStore.java
+++ /dev/null
@@ -1,831 +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 com.twitter.distributedlog.impl;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientUtils;
-import com.twitter.distributedlog.callback.LogSegmentNamesListener;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.metadata.LogMetadata;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.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.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.data.Stat;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-/**
- * Test ZK based log segment metadata store.
- */
-public class TestZKLogSegmentMetadataStore extends TestDistributedLogBase {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestZKLogSegmentMetadataStore.class);
-
-    private final static int zkSessionTimeoutMs = 2000;
-
-    private LogSegmentMetadata createLogSegment(
-            long logSegmentSequenceNumber) {
-        return createLogSegment(logSegmentSequenceNumber, 99L);
-    }
-
-    private LogSegmentMetadata createLogSegment(
-        long logSegmentSequenceNumber,
-        long lastEntryId) {
-        return DLMTestUtil.completedLogSegment(
-                "/" + runtime.getMethodName(),
-                logSegmentSequenceNumber,
-                logSegmentSequenceNumber,
-                1L,
-                100,
-                logSegmentSequenceNumber,
-                lastEntryId,
-                0L,
-                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
-    }
-
-    @Rule
-    public TestName runtime = new TestName();
-    protected final DistributedLogConfiguration baseConf =
-            new DistributedLogConfiguration();
-    protected ZooKeeperClient zkc;
-    protected ZKLogSegmentMetadataStore lsmStore;
-    protected OrderedScheduler scheduler;
-    protected URI uri;
-    protected String rootZkPath;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .uri(createDLMURI("/"))
-                .sessionTimeoutMs(zkSessionTimeoutMs)
-                .build();
-        scheduler = OrderedScheduler.newBuilder()
-                .name("test-zk-logsegment-metadata-store")
-                .corePoolSize(1)
-                .build();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.addConfiguration(baseConf);
-        this.uri = createDLMURI("/" + runtime.getMethodName());
-        lsmStore = new ZKLogSegmentMetadataStore(conf, zkc, scheduler);
-        zkc.get().create(
-                "/" + runtime.getMethodName(),
-                new byte[0],
-                ZooDefs.Ids.OPEN_ACL_UNSAFE,
-                CreateMode.PERSISTENT);
-        this.rootZkPath = "/" + runtime.getMethodName();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        if (null != zkc) {
-            zkc.close();
-        }
-        if (null != scheduler) {
-            scheduler.shutdown();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogSegment() throws Exception {
-        LogSegmentMetadata segment = createLogSegment(1L);
-        Transaction<Object> createTxn = lsmStore.transaction();
-        lsmStore.createLogSegment(createTxn, segment, null);
-        FutureUtils.result(createTxn.execute());
-        // the log segment should be created
-        assertNotNull("LogSegment " + segment + " should be created",
-                zkc.get().exists(segment.getZkPath(), false));
-        LogSegmentMetadata segment2 = createLogSegment(1L);
-        Transaction<Object> createTxn2 = lsmStore.transaction();
-        lsmStore.createLogSegment(createTxn2, segment2, null);
-        try {
-            FutureUtils.result(createTxn2.execute());
-            fail("Should fail if log segment exists");
-        } catch (Throwable t) {
-            // expected
-            assertTrue("Should throw NodeExistsException if log segment exists",
-                    t instanceof ZKException);
-            ZKException zke = (ZKException) t;
-            assertEquals("Should throw NodeExistsException if log segment exists",
-                    KeeperException.Code.NODEEXISTS, zke.getKeeperExceptionCode());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testDeleteLogSegment() throws Exception {
-        LogSegmentMetadata segment = createLogSegment(1L);
-        Transaction<Object> createTxn = lsmStore.transaction();
-        lsmStore.createLogSegment(createTxn, segment, null);
-        FutureUtils.result(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());
-        assertNull("LogSegment " + segment + " should be deleted",
-                zkc.get().exists(segment.getZkPath(), false));
-    }
-
-    @Test(timeout = 60000)
-    public void testDeleteNonExistentLogSegment() throws Exception {
-        LogSegmentMetadata segment = createLogSegment(1L);
-        Transaction<Object> deleteTxn = lsmStore.transaction();
-        lsmStore.deleteLogSegment(deleteTxn, segment, null);
-        try {
-            FutureUtils.result(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",
-                    t instanceof ZKException);
-            ZKException zke = (ZKException) t;
-            assertEquals("Should throw NoNodeException if log segment doesn't exist",
-                    KeeperException.Code.NONODE, zke.getKeeperExceptionCode());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testUpdateNonExistentLogSegment() throws Exception {
-        LogSegmentMetadata segment = createLogSegment(1L);
-        Transaction<Object> updateTxn = lsmStore.transaction();
-        lsmStore.updateLogSegment(updateTxn, segment);
-        try {
-            FutureUtils.result(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",
-                    t instanceof ZKException);
-            ZKException zke = (ZKException) t;
-            assertEquals("Should throw NoNodeException if log segment doesn't exist",
-                    KeeperException.Code.NONODE, zke.getKeeperExceptionCode());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testUpdateLogSegment() throws Exception {
-        LogSegmentMetadata segment = createLogSegment(1L, 99L);
-        Transaction<Object> createTxn = lsmStore.transaction();
-        lsmStore.createLogSegment(createTxn, segment, null);
-        FutureUtils.result(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());
-        // the log segment should be updated
-        LogSegmentMetadata readSegment =
-                FutureUtils.result(LogSegmentMetadata.read(zkc, segment.getZkPath(), true));
-        assertEquals("Last entry id should be changed from 99L to 999L",
-                999L, readSegment.getLastEntryId());
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateDeleteLogSegmentSuccess() throws Exception {
-        LogSegmentMetadata segment1 = createLogSegment(1L);
-        LogSegmentMetadata segment2 = createLogSegment(2L);
-        // create log segment 1
-        Transaction<Object> createTxn = lsmStore.transaction();
-        lsmStore.createLogSegment(createTxn, segment1, null);
-        FutureUtils.result(createTxn.execute());
-        // the log segment should be created
-        assertNotNull("LogSegment " + segment1 + " should be created",
-                zkc.get().exists(segment1.getZkPath(), false));
-        // delete log segment 1 and create log segment 2
-        Transaction<Object> createDeleteTxn = lsmStore.transaction();
-        lsmStore.createLogSegment(createDeleteTxn, segment2, null);
-        lsmStore.deleteLogSegment(createDeleteTxn, segment1, null);
-        FutureUtils.result(createDeleteTxn.execute());
-        // segment 1 should be deleted, segment 2 should be created
-        assertNull("LogSegment " + segment1 + " should be deleted",
-                zkc.get().exists(segment1.getZkPath(), false));
-        assertNotNull("LogSegment " + segment2 + " should be created",
-                zkc.get().exists(segment2.getZkPath(), false));
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateDeleteLogSegmentFailure() throws Exception {
-        LogSegmentMetadata segment1 = createLogSegment(1L);
-        LogSegmentMetadata segment2 = createLogSegment(2L);
-        LogSegmentMetadata segment3 = createLogSegment(3L);
-        // create log segment 1
-        Transaction<Object> createTxn = lsmStore.transaction();
-        lsmStore.createLogSegment(createTxn, segment1, null);
-        FutureUtils.result(createTxn.execute());
-        // the log segment should be created
-        assertNotNull("LogSegment " + segment1 + " should be created",
-                zkc.get().exists(segment1.getZkPath(), false));
-        // delete log segment 1 and delete log segment 2
-        Transaction<Object> createDeleteTxn = lsmStore.transaction();
-        lsmStore.deleteLogSegment(createDeleteTxn, segment1, null);
-        lsmStore.deleteLogSegment(createDeleteTxn, segment2, null);
-        lsmStore.createLogSegment(createDeleteTxn, segment3, null);
-        try {
-            FutureUtils.result(createDeleteTxn.execute());
-            fail("Should fail transaction if one operation failed");
-        } catch (Throwable t) {
-            assertTrue("Transaction is aborted",
-                    t instanceof ZKException);
-            ZKException zke = (ZKException) t;
-            assertEquals("Transaction is aborted",
-                    KeeperException.Code.NONODE, zke.getKeeperExceptionCode());
-        }
-        // segment 1 should not be deleted
-        assertNotNull("LogSegment " + segment1 + " should not be deleted",
-                zkc.get().exists(segment1.getZkPath(), false));
-        // segment 3 should not be created
-        assertNull("LogSegment " + segment3 + " should be created",
-                zkc.get().exists(segment3.getZkPath(), false));
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogSegment() throws Exception {
-        LogSegmentMetadata segment = createLogSegment(1L, 99L);
-        Transaction<Object> createTxn = lsmStore.transaction();
-        lsmStore.createLogSegment(createTxn, segment, null);
-        FutureUtils.result(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()));
-        assertEquals("Log segment should match",
-                segment, readSegment);
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogSegmentNames() throws Exception {
-        Transaction<Object> createTxn = lsmStore.transaction();
-        List<LogSegmentMetadata> createdSegments = Lists.newArrayListWithExpectedSize(10);
-        for (int i = 0; i < 10; i++) {
-            LogSegmentMetadata segment = createLogSegment(i);
-            createdSegments.add(segment);
-            lsmStore.createLogSegment(createTxn, segment, null);
-        }
-        FutureUtils.result(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();
-        Collections.sort(logSegmentNames);
-        assertEquals("Should find 10 log segments",
-                10, logSegmentNames.size());
-        assertEquals(children, logSegmentNames);
-        List<Future<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));
-        for (int i = 0; i < 10; i++) {
-            assertEquals(createdSegments.get(i), segments.get(i));
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testRegisterListenerAfterLSMStoreClosed() throws Exception {
-        lsmStore.close();
-        LogSegmentMetadata segment = createLogSegment(1L);
-        lsmStore.getLogSegmentNames(segment.getZkPath(), new LogSegmentNamesListener() {
-            @Override
-            public void onSegmentsUpdated(Versioned<List<String>> segments) {
-                // no-op;
-            }
-            @Override
-            public void onLogStreamDeleted() {
-                // no-op;
-            }
-        });
-        assertTrue("No listener is registered",
-                lsmStore.listeners.isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testLogSegmentNamesListener() throws Exception {
-        int numSegments = 3;
-        Transaction<Object> createTxn = lsmStore.transaction();
-        for (int i = 0; i < numSegments; i++) {
-            LogSegmentMetadata segment = createLogSegment(i);
-            lsmStore.createLogSegment(createTxn, segment, null);
-        }
-        FutureUtils.result(createTxn.execute());
-        String rootPath = "/" + runtime.getMethodName();
-        List<String> children = zkc.get().getChildren(rootPath, false);
-        Collections.sort(children);
-
-        final AtomicInteger numNotifications = new AtomicInteger(0);
-        final List<List<String>> segmentLists = Lists.newArrayListWithExpectedSize(2);
-        LogSegmentNamesListener listener = new LogSegmentNamesListener() {
-            @Override
-            public void onSegmentsUpdated(Versioned<List<String>> segments) {
-                logger.info("Received segments : {}", segments);
-                segmentLists.add(segments.getValue());
-                numNotifications.incrementAndGet();
-            }
-            @Override
-            public void onLogStreamDeleted() {
-                // no-op;
-            }
-        };
-        lsmStore.getLogSegmentNames(rootPath, listener);
-        assertEquals(1, lsmStore.listeners.size());
-        assertTrue("Should contain listener", lsmStore.listeners.containsKey(rootPath));
-        assertTrue("Should contain listener", lsmStore.listeners.get(rootPath).containsKey(listener));
-        while (numNotifications.get() < 1) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals("Should receive one segment list update",
-                1, numNotifications.get());
-        List<String> firstSegmentList = segmentLists.get(0);
-        Collections.sort(firstSegmentList);
-        assertEquals("List of segments should be same",
-                children, firstSegmentList);
-
-        logger.info("Create another {} segments.", numSegments);
-
-        // create another log segment, it should trigger segment list updated
-        Transaction<Object> anotherCreateTxn = lsmStore.transaction();
-        for (int i = numSegments; i < 2 * numSegments; i++) {
-            LogSegmentMetadata segment = createLogSegment(i);
-            lsmStore.createLogSegment(anotherCreateTxn, segment, null);
-        }
-        FutureUtils.result(anotherCreateTxn.execute());
-        List<String> newChildren = zkc.get().getChildren(rootPath, false);
-        Collections.sort(newChildren);
-        logger.info("All log segments become {}", newChildren);
-        while (numNotifications.get() < 2) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals("Should receive second segment list update",
-                2, numNotifications.get());
-        List<String> secondSegmentList = segmentLists.get(1);
-        Collections.sort(secondSegmentList);
-        assertEquals("List of segments should be updated",
-                2 * numSegments, secondSegmentList.size());
-        assertEquals("List of segments should be updated",
-                newChildren, secondSegmentList);
-    }
-
-    @Test(timeout = 60000)
-    public void testLogSegmentNamesListenerOnDeletion() throws Exception {
-        int numSegments = 3;
-        Transaction<Object> createTxn = lsmStore.transaction();
-        for (int i = 0; i < numSegments; i++) {
-            LogSegmentMetadata segment = createLogSegment(i);
-            lsmStore.createLogSegment(createTxn, segment, null);
-        }
-        FutureUtils.result(createTxn.execute());
-        String rootPath = "/" + runtime.getMethodName();
-        List<String> children = zkc.get().getChildren(rootPath, false);
-        Collections.sort(children);
-
-        final AtomicInteger numNotifications = new AtomicInteger(0);
-        final List<List<String>> segmentLists = Lists.newArrayListWithExpectedSize(2);
-        LogSegmentNamesListener listener = new LogSegmentNamesListener() {
-            @Override
-            public void onSegmentsUpdated(Versioned<List<String>> segments) {
-                logger.info("Received segments : {}", segments);
-                segmentLists.add(segments.getValue());
-                numNotifications.incrementAndGet();
-            }
-
-            @Override
-            public void onLogStreamDeleted() {
-                // no-op;
-            }
-        };
-        lsmStore.getLogSegmentNames(rootPath, listener);
-        assertEquals(1, lsmStore.listeners.size());
-        assertTrue("Should contain listener", lsmStore.listeners.containsKey(rootPath));
-        assertTrue("Should contain listener", lsmStore.listeners.get(rootPath).containsKey(listener));
-        while (numNotifications.get() < 1) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals("Should receive one segment list update",
-                1, numNotifications.get());
-        List<String> firstSegmentList = segmentLists.get(0);
-        Collections.sort(firstSegmentList);
-        assertEquals("List of segments should be same",
-                children, firstSegmentList);
-
-        // delete all log segments, it should trigger segment list updated
-        Transaction<Object> deleteTxn = lsmStore.transaction();
-        for (int i = 0; i < numSegments; i++) {
-            LogSegmentMetadata segment = createLogSegment(i);
-            lsmStore.deleteLogSegment(deleteTxn, segment, null);
-        }
-        FutureUtils.result(deleteTxn.execute());
-        List<String> newChildren = zkc.get().getChildren(rootPath, false);
-        Collections.sort(newChildren);
-        while (numNotifications.get() < 2) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals("Should receive second segment list update",
-                2, numNotifications.get());
-        List<String> secondSegmentList = segmentLists.get(1);
-        Collections.sort(secondSegmentList);
-        assertEquals("List of segments should be updated",
-                0, secondSegmentList.size());
-        assertEquals("List of segments should be updated",
-                newChildren, secondSegmentList);
-
-        // delete the root path
-        zkc.get().delete(rootPath, -1);
-        while (!lsmStore.listeners.isEmpty()) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertTrue("listener should be removed after root path is deleted",
-                lsmStore.listeners.isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testLogSegmentNamesListenerOnSessionExpired() throws Exception {
-        int numSegments = 3;
-        Transaction<Object> createTxn = lsmStore.transaction();
-        for (int i = 0; i < numSegments; i++) {
-            LogSegmentMetadata segment = createLogSegment(i);
-            lsmStore.createLogSegment(createTxn, segment, null);
-        }
-        FutureUtils.result(createTxn.execute());
-        String rootPath = "/" + runtime.getMethodName();
-        List<String> children = zkc.get().getChildren(rootPath, false);
-        Collections.sort(children);
-
-        final AtomicInteger numNotifications = new AtomicInteger(0);
-        final List<List<String>> segmentLists = Lists.newArrayListWithExpectedSize(2);
-        LogSegmentNamesListener listener = new LogSegmentNamesListener() {
-            @Override
-            public void onSegmentsUpdated(Versioned<List<String>> segments) {
-                logger.info("Received segments : {}", segments);
-                segmentLists.add(segments.getValue());
-                numNotifications.incrementAndGet();
-            }
-
-            @Override
-            public void onLogStreamDeleted() {
-                // no-op;
-            }
-        };
-        lsmStore.getLogSegmentNames(rootPath, listener);
-        assertEquals(1, lsmStore.listeners.size());
-        assertTrue("Should contain listener", lsmStore.listeners.containsKey(rootPath));
-        assertTrue("Should contain listener", lsmStore.listeners.get(rootPath).containsKey(listener));
-        while (numNotifications.get() < 1) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals("Should receive one segment list update",
-                1, numNotifications.get());
-        List<String> firstSegmentList = segmentLists.get(0);
-        Collections.sort(firstSegmentList);
-        assertEquals("List of segments should be same",
-                children, firstSegmentList);
-
-        ZooKeeperClientUtils.expireSession(zkc,
-                BKNamespaceDriver.getZKServersFromDLUri(uri), conf.getZKSessionTimeoutMilliseconds());
-
-        logger.info("Create another {} segments.", numSegments);
-
-        // create another log segment, it should trigger segment list updated
-        Transaction<Object> anotherCreateTxn = lsmStore.transaction();
-        for (int i = numSegments; i < 2 * numSegments; i++) {
-            LogSegmentMetadata segment = createLogSegment(i);
-            lsmStore.createLogSegment(anotherCreateTxn, segment, null);
-        }
-        FutureUtils.result(anotherCreateTxn.execute());
-        List<String> newChildren = zkc.get().getChildren(rootPath, false);
-        Collections.sort(newChildren);
-        logger.info("All log segments become {}", newChildren);
-        while (numNotifications.get() < 2) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals("Should receive third segment list update",
-                2, numNotifications.get());
-        List<String> thirdSegmentList = segmentLists.get(1);
-        Collections.sort(thirdSegmentList);
-        assertEquals("List of segments should be updated",
-                2 * numSegments, thirdSegmentList.size());
-        assertEquals("List of segments should be updated",
-                newChildren, thirdSegmentList);
-    }
-
-    @Test(timeout = 60000)
-    public void testLogSegmentNamesListenerOnDeletingLogStream() throws Exception {
-        int numSegments = 3;
-        Transaction<Object> createTxn = lsmStore.transaction();
-        for (int i = 0; i < numSegments; i++) {
-            LogSegmentMetadata segment = createLogSegment(i);
-            lsmStore.createLogSegment(createTxn, segment, null);
-        }
-        FutureUtils.result(createTxn.execute());
-        String rootPath = "/" + runtime.getMethodName();
-        List<String> children = zkc.get().getChildren(rootPath, false);
-        Collections.sort(children);
-
-        final AtomicInteger numNotifications = new AtomicInteger(0);
-        final List<List<String>> segmentLists = Lists.newArrayListWithExpectedSize(2);
-        final CountDownLatch deleteLatch = new CountDownLatch(1);
-        LogSegmentNamesListener listener = new LogSegmentNamesListener() {
-            @Override
-            public void onSegmentsUpdated(Versioned<List<String>> segments) {
-                logger.info("Received segments : {}", segments);
-                segmentLists.add(segments.getValue());
-                numNotifications.incrementAndGet();
-            }
-
-            @Override
-            public void onLogStreamDeleted() {
-                deleteLatch.countDown();
-            }
-        };
-        lsmStore.getLogSegmentNames(rootPath, listener);
-        assertEquals(1, lsmStore.listeners.size());
-        assertTrue("Should contain listener", lsmStore.listeners.containsKey(rootPath));
-        assertTrue("Should contain listener", lsmStore.listeners.get(rootPath).containsKey(listener));
-        while (numNotifications.get() < 1) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals("Should receive one segment list update",
-                1, numNotifications.get());
-        List<String> firstSegmentList = segmentLists.get(0);
-        Collections.sort(firstSegmentList);
-        assertEquals("List of segments should be same",
-                children, firstSegmentList);
-
-        // delete all log segments, it should trigger segment list updated
-        Transaction<Object> deleteTxn = lsmStore.transaction();
-        for (int i = 0; i < numSegments; i++) {
-            LogSegmentMetadata segment = createLogSegment(i);
-            lsmStore.deleteLogSegment(deleteTxn, segment, null);
-        }
-        FutureUtils.result(deleteTxn.execute());
-        List<String> newChildren = zkc.get().getChildren(rootPath, false);
-        Collections.sort(newChildren);
-        while (numNotifications.get() < 2) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals("Should receive second segment list update",
-                2, numNotifications.get());
-        List<String> secondSegmentList = segmentLists.get(1);
-        Collections.sort(secondSegmentList);
-        assertEquals("List of segments should be updated",
-                0, secondSegmentList.size());
-        assertEquals("List of segments should be updated",
-                newChildren, secondSegmentList);
-
-        // delete the root path
-        zkc.get().delete(rootPath, -1);
-        while (!lsmStore.listeners.isEmpty()) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertTrue("listener should be removed after root path is deleted",
-                lsmStore.listeners.isEmpty());
-        deleteLatch.await();
-    }
-
-    @Test(timeout = 60000)
-    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>();
-        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);
-            }
-
-            @Override
-            public void onAbort(Throwable t) {
-                result.setException(t);
-            }
-        });
-        FutureUtils.result(updateTxn.execute());
-        assertEquals(1, ((ZkVersion) FutureUtils.result(result)).getZnodeVersion());
-        Stat stat = new Stat();
-        byte[] data = zkc.get().getData(rootZkPath, false, stat);
-        assertEquals(999L, DLUtils.deserializeLogSegmentSequenceNumber(data));
-        assertEquals(1, stat.getVersion());
-    }
-
-    @Test(timeout = 60000)
-    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>();
-        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);
-                    }
-
-                    @Override
-                    public void onAbort(Throwable t) {
-                        result.setException(t);
-                    }
-                });
-        try {
-            FutureUtils.result(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);
-            fail("Should fail on storing log segment sequence number if providing bad version");
-        } catch (KeeperException ke) {
-            assertEquals(KeeperException.Code.BADVERSION, ke.code());
-        }
-        Stat stat = new Stat();
-        byte[] data = zkc.get().getData(rootZkPath, false, stat);
-        assertEquals(0, stat.getVersion());
-        assertEquals(0, data.length);
-    }
-
-    @Test(timeout = 60000)
-    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>();
-        String nonExistentPath = rootZkPath + "/non-existent";
-        LogMetadata metadata = mock(LogMetadata.class);
-        when(metadata.getLogSegmentsPath()).thenReturn(nonExistentPath);
-        lsmStore.storeMaxLogSegmentSequenceNumber(updateTxn, metadata, value,
-                new Transaction.OpListener<Version>() {
-                    @Override
-                    public void onCommit(Version r) {
-                        result.setValue(r);
-                    }
-
-                    @Override
-                    public void onAbort(Throwable t) {
-                        result.setException(t);
-                    }
-                });
-        try {
-            FutureUtils.result(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);
-            fail("Should fail on storing log segment sequence number if path doesn't exist");
-        } catch (KeeperException ke) {
-            assertEquals(KeeperException.Code.NONODE, ke.code());
-        }
-    }
-
-    @Test(timeout = 60000)
-    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>();
-        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);
-            }
-
-            @Override
-            public void onAbort(Throwable t) {
-                result.setException(t);
-            }
-        });
-        FutureUtils.result(updateTxn.execute());
-        assertEquals(1, ((ZkVersion) FutureUtils.result(result)).getZnodeVersion());
-        Stat stat = new Stat();
-        byte[] data = zkc.get().getData(rootZkPath, false, stat);
-        assertEquals(999L, DLUtils.deserializeTransactionId(data));
-        assertEquals(1, stat.getVersion());
-    }
-
-    @Test(timeout = 60000)
-    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>();
-        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);
-                    }
-
-                    @Override
-                    public void onAbort(Throwable t) {
-                        result.setException(t);
-                    }
-                });
-        try {
-            FutureUtils.result(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);
-            fail("Should fail on storing log record transaction id if providing bad version");
-        } catch (KeeperException ke) {
-            assertEquals(KeeperException.Code.BADVERSION, ke.code());
-        }
-        Stat stat = new Stat();
-        byte[] data = zkc.get().getData(rootZkPath, false, stat);
-        assertEquals(0, stat.getVersion());
-        assertEquals(0, data.length);
-    }
-
-    @Test(timeout = 60000)
-    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>();
-        String nonExistentPath = rootZkPath + "/non-existent";
-        LogMetadataForWriter metadata = mock(LogMetadataForWriter.class);
-        when(metadata.getMaxTxIdPath()).thenReturn(nonExistentPath);
-        lsmStore.storeMaxTxnId(updateTxn, metadata, value,
-                new Transaction.OpListener<Version>() {
-                    @Override
-                    public void onCommit(Version r) {
-                        result.setValue(r);
-                    }
-
-                    @Override
-                    public void onAbort(Throwable t) {
-                        result.setException(t);
-                    }
-                });
-        try {
-            FutureUtils.result(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);
-            fail("Should fail on storing log record transaction id if path doesn't exist");
-        } catch (KeeperException ke) {
-            assertEquals(KeeperException.Code.NONODE, ke.code());
-        }
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKNamespaceWatcher.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKNamespaceWatcher.java
deleted file mode 100644
index c9a2e5b..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/TestZKNamespaceWatcher.java
+++ /dev/null
@@ -1,185 +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 com.twitter.distributedlog.impl;
-
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientUtils;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Utils;
-import org.apache.zookeeper.CreateMode;
-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 java.net.URI;
-import java.util.Iterator;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.junit.Assert.*;
-
-/**
- * Test ZK Namespace Watcher.
- */
-public class TestZKNamespaceWatcher extends TestDistributedLogBase {
-
-    private final static int zkSessionTimeoutMs = 2000;
-
-    @Rule
-    public TestName runtime = new TestName();
-    protected final DistributedLogConfiguration baseConf =
-            new DistributedLogConfiguration();
-    protected ZooKeeperClient zkc;
-    protected OrderedScheduler scheduler;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .uri(createDLMURI("/"))
-                .sessionTimeoutMs(zkSessionTimeoutMs)
-                .build();
-        scheduler = OrderedScheduler.newBuilder()
-                .name("test-zk-namespace-watcher")
-                .corePoolSize(1)
-                .build();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        if (null != zkc) {
-            zkc.close();
-        }
-        if (null != scheduler) {
-            scheduler.shutdown();
-        }
-    }
-
-    private void createLogInNamespace(URI uri, String logName) throws Exception {
-        String logPath = uri.getPath() + "/" + logName;
-        Utils.zkCreateFullPathOptimistic(zkc, logPath, new byte[0],
-                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-    }
-
-    private void deleteLogInNamespace(URI uri, String logName) throws Exception {
-        String logPath = uri.getPath() + "/" + logName;
-        zkc.get().delete(logPath, -1);
-    }
-
-    @Test(timeout = 60000)
-    public void testNamespaceListener() throws Exception {
-        URI uri = createDLMURI("/" + runtime.getMethodName());
-        zkc.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.addConfiguration(baseConf);
-        ZKNamespaceWatcher watcher = new ZKNamespaceWatcher(conf, uri, zkc, scheduler);
-        final CountDownLatch[] latches = new CountDownLatch[10];
-        for (int i = 0; i < 10; i++) {
-            latches[i] = new CountDownLatch(1);
-        }
-        final AtomicInteger numUpdates = new AtomicInteger(0);
-        final AtomicReference<Set<String>> receivedLogs = new AtomicReference<Set<String>>(null);
-        watcher.registerListener(new NamespaceListener() {
-            @Override
-            public void onStreamsChanged(Iterator<String> streams) {
-                Set<String> streamSet = Sets.newHashSet(streams);
-                int updates = numUpdates.incrementAndGet();
-                receivedLogs.set(streamSet);
-                latches[updates - 1].countDown();
-            }
-        });
-        // first update
-        final Set<String> expectedLogs = Sets.newHashSet();
-        latches[0].await();
-        validateReceivedLogs(expectedLogs, receivedLogs.get());
-
-        // create test1
-        expectedLogs.add("test1");
-        createLogInNamespace(uri, "test1");
-        latches[1].await();
-        validateReceivedLogs(expectedLogs, receivedLogs.get());
-
-        // create invalid log
-        createLogInNamespace(uri, ".test1");
-        latches[2].await();
-        validateReceivedLogs(expectedLogs, receivedLogs.get());
-
-        // create test2
-        expectedLogs.add("test2");
-        createLogInNamespace(uri, "test2");
-        latches[3].await();
-        validateReceivedLogs(expectedLogs, receivedLogs.get());
-
-        // delete test1
-        expectedLogs.remove("test1");
-        deleteLogInNamespace(uri, "test1");
-        latches[4].await();
-        validateReceivedLogs(expectedLogs, receivedLogs.get());
-    }
-
-    private void validateReceivedLogs(Set<String> expectedLogs, Set<String> receivedLogs) {
-        assertTrue(Sets.difference(expectedLogs, receivedLogs).isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testSessionExpired() throws Exception {
-        URI uri = createDLMURI("/" + runtime.getMethodName());
-        zkc.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.addConfiguration(baseConf);
-        ZKNamespaceWatcher watcher = new ZKNamespaceWatcher(conf, uri, zkc, scheduler);
-        final CountDownLatch[] latches = new CountDownLatch[10];
-        for (int i = 0; i < 10; i++) {
-            latches[i] = new CountDownLatch(1);
-        }
-        final AtomicInteger numUpdates = new AtomicInteger(0);
-        final AtomicReference<Set<String>> receivedLogs = new AtomicReference<Set<String>>(null);
-        watcher.registerListener(new NamespaceListener() {
-            @Override
-            public void onStreamsChanged(Iterator<String> streams) {
-                Set<String> streamSet = Sets.newHashSet(streams);
-                int updates = numUpdates.incrementAndGet();
-                receivedLogs.set(streamSet);
-                latches[updates - 1].countDown();
-            }
-        });
-        latches[0].await();
-        createLogInNamespace(uri, "test1");
-        latches[1].await();
-        createLogInNamespace(uri, "test2");
-        latches[2].await();
-        assertEquals(2, receivedLogs.get().size());
-        ZooKeeperClientUtils.expireSession(zkc, BKNamespaceDriver.getZKServersFromDLUri(uri), zkSessionTimeoutMs);
-        latches[3].await();
-        assertEquals(2, receivedLogs.get().size());
-        createLogInNamespace(uri, "test3");
-        latches[4].await();
-        assertEquals(3, receivedLogs.get().size());
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
deleted file mode 100644
index 0ce9f46..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
+++ /dev/null
@@ -1,446 +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 com.twitter.distributedlog.impl.federated;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Iterators;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClientUtils;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.exceptions.LogExistsException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.metadata.LogMetadataStore;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import java.net.URI;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Random;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Test ZK based metadata store.
- */
-public class TestFederatedZKLogMetadataStore extends TestDistributedLogBase {
-
-    private final static int zkSessionTimeoutMs = 2000;
-    private final static int maxLogsPerSubnamespace = 10;
-
-    static class TestNamespaceListener implements NamespaceListener {
-
-        final CountDownLatch doneLatch = new CountDownLatch(1);
-        final AtomicReference<Iterator<String>> resultHolder = new AtomicReference<Iterator<String>>();
-
-        @Override
-        public void onStreamsChanged(Iterator<String> streams) {
-            resultHolder.set(streams);
-            if (streams.hasNext()) {
-                doneLatch.countDown();
-            }
-        }
-
-        Iterator<String> getResult() {
-            return resultHolder.get();
-        }
-
-        void waitForDone() throws InterruptedException {
-            doneLatch.await();
-        }
-    }
-
-    static class TestNamespaceListenerWithExpectedSize implements NamespaceListener {
-
-        final int expectedSize;
-        final CountDownLatch doneLatch = new CountDownLatch(1);
-        final AtomicReference<Set<String>> resultHolder = new AtomicReference<Set<String>>();
-
-        TestNamespaceListenerWithExpectedSize(int expectedSize) {
-            this.expectedSize = expectedSize;
-        }
-
-        Set<String> getResult() {
-            return resultHolder.get();
-        }
-
-        @Override
-        public void onStreamsChanged(Iterator<String> logsIter) {
-            List<String> logList = Lists.newArrayList(logsIter);
-            if (logList.size() < expectedSize) {
-                return;
-            }
-            resultHolder.set(Sets.newTreeSet(logList));
-            doneLatch.countDown();
-        }
-
-        void waitForDone() throws InterruptedException {
-            doneLatch.await();
-        }
-    }
-
-    @Rule
-    public TestName runtime = new TestName();
-    protected final DistributedLogConfiguration baseConf =
-            new DistributedLogConfiguration()
-                    .setFederatedMaxLogsPerSubnamespace(maxLogsPerSubnamespace);
-    protected ZooKeeperClient zkc;
-    protected FederatedZKLogMetadataStore metadataStore;
-    protected OrderedScheduler scheduler;
-    protected URI uri;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .uri(createDLMURI("/"))
-                .sessionTimeoutMs(zkSessionTimeoutMs)
-                .build();
-        scheduler = OrderedScheduler.newBuilder()
-                .name("test-zk-logmetadata-store")
-                .corePoolSize(2)
-                .build();
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.addConfiguration(baseConf);
-        this.uri = createDLMURI("/" + runtime.getMethodName());
-        FederatedZKLogMetadataStore.createFederatedNamespace(uri, zkc);
-        metadataStore = new FederatedZKLogMetadataStore(conf, uri, zkc, scheduler);
-    }
-
-    @After
-    public void teardown() throws Exception {
-        if (null != zkc) {
-            zkc.close();
-        }
-        if (null != scheduler) {
-            scheduler.shutdown();
-        }
-    }
-
-    private void deleteLog(String logName) throws Exception {
-        Optional<URI> logUriOptional = FutureUtils.result(metadataStore.getLogLocation(logName));
-        assertTrue(logUriOptional.isPresent());
-        URI logUri = logUriOptional.get();
-        zkc.get().delete(logUri.getPath() + "/" + logName, -1);
-    }
-
-    @Test(timeout = 60000)
-    public void testBasicOperations() throws Exception {
-        TestNamespaceListener listener = new TestNamespaceListener();
-        metadataStore.registerNamespaceListener(listener);
-        String logName = "test-log-1";
-        URI logUri = FutureUtils.result(metadataStore.createLog(logName));
-        assertEquals(uri, logUri);
-        Optional<URI> logLocation = FutureUtils.result(metadataStore.getLogLocation(logName));
-        assertTrue(logLocation.isPresent());
-        assertEquals(uri, logLocation.get());
-        Optional<URI> notExistLogLocation = FutureUtils.result(metadataStore.getLogLocation("non-existent-log"));
-        assertFalse(notExistLogLocation.isPresent());
-        // listener should receive notification
-        listener.waitForDone();
-        Iterator<String> logsIter = listener.getResult();
-        assertTrue(logsIter.hasNext());
-        assertEquals(logName, logsIter.next());
-        assertFalse(logsIter.hasNext());
-        // get logs should return the log
-        Iterator<String> newLogsIter = FutureUtils.result(metadataStore.getLogs());
-        assertTrue(newLogsIter.hasNext());
-        assertEquals(logName, newLogsIter.next());
-        assertFalse(newLogsIter.hasNext());
-    }
-
-    @Test(timeout = 60000)
-    public void testMultipleListeners() throws Exception {
-        TestNamespaceListener listener1 = new TestNamespaceListener();
-        TestNamespaceListener listener2 = new TestNamespaceListener();
-        metadataStore.registerNamespaceListener(listener1);
-        metadataStore.registerNamespaceListener(listener2);
-        String logName = "test-multiple-listeners";
-        URI logUri = FutureUtils.result(metadataStore.createLog(logName));
-        assertEquals(uri, logUri);
-        listener1.waitForDone();
-        listener2.waitForDone();
-        Iterator<String> logsIter1 = listener1.getResult();
-        Iterator<String> logsIter2 = listener2.getResult();
-        assertTrue(Iterators.elementsEqual(logsIter1, logsIter2));
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLog() throws Exception {
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.addConfiguration(baseConf);
-        ZooKeeperClient anotherZkc = TestZooKeeperClientBuilder.newBuilder()
-                .uri(uri)
-                .sessionTimeoutMs(zkSessionTimeoutMs)
-                .build();
-        FederatedZKLogMetadataStore anotherMetadataStore =
-                new FederatedZKLogMetadataStore(conf, uri, anotherZkc, scheduler);
-        for (int i = 0; i < 2 * maxLogsPerSubnamespace; i++) {
-            LogMetadataStore createStore, checkStore;
-            if (i % 2 == 0) {
-                createStore = metadataStore;
-                checkStore = anotherMetadataStore;
-            } else {
-                createStore = anotherMetadataStore;
-                checkStore = metadataStore;
-            }
-            String logName = "test-create-log-" + i;
-            URI logUri = FutureUtils.result(createStore.createLog(logName));
-            Optional<URI> logLocation = FutureUtils.result(checkStore.getLogLocation(logName));
-            assertTrue("Log " + logName + " doesn't exist", logLocation.isPresent());
-            assertEquals("Different log location " + logLocation.get() + " is found",
-                    logUri, logLocation.get());
-        }
-        assertEquals(2, metadataStore.getSubnamespaces().size());
-        assertEquals(2, anotherMetadataStore.getSubnamespaces().size());
-    }
-
-    @Test(timeout = 60000)
-    public void testDuplicatedLogs() throws Exception {
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.addConfiguration(baseConf);
-
-        String logName = "test-log";
-        FutureUtils.result(metadataStore.createLog(logName));
-
-        URI subNs1 = FutureUtils.result(metadataStore.createSubNamespace());
-        URI subNs2 = FutureUtils.result(metadataStore.createSubNamespace());
-
-        String duplicatedLogName = "test-duplicated-logs";
-        // Create same log in different sub namespaces
-        metadataStore.createLogInNamespaceSync(subNs1, duplicatedLogName);
-        metadataStore.createLogInNamespaceSync(subNs2, duplicatedLogName);
-
-        try {
-            FutureUtils.result(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));
-            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"));
-            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));
-            fail("should throw exception when duplicated log found");
-        } catch (UnexpectedException ue) {
-            // should throw unexpected exception
-            assertTrue(metadataStore.duplicatedLogFound.get());
-        }
-        try {
-            FutureUtils.result(metadataStore.getLogs());
-            fail("should throw exception when duplicated log found");
-        } catch (UnexpectedException ue) {
-            // should throw unexpected exception
-            assertTrue(metadataStore.duplicatedLogFound.get());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogLocationWhenCacheMissed() throws Exception {
-        String logName = "test-get-location-when-cache-missed";
-        URI logUri = FutureUtils.result(metadataStore.createLog(logName));
-        assertEquals(uri, logUri);
-        metadataStore.removeLogFromCache(logName);
-        Optional<URI> logLocation = FutureUtils.result(metadataStore.getLogLocation(logName));
-        assertTrue(logLocation.isPresent());
-        assertEquals(logUri, logLocation.get());
-    }
-
-    @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));
-        assertEquals(uri, logUri);
-        metadataStore.removeLogFromCache(logName);
-        FutureUtils.result(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));
-        assertEquals(uri, logUri);
-        FutureUtils.result(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));
-            expectedLogs.add(logName);
-        }
-        return expectedLogs;
-    }
-
-    private Set<String> createLogs(URI uri, int numLogs, String prefix) throws Exception {
-        Set<String> expectedLogs = Sets.newTreeSet();
-        for (int i = 0; i < numLogs; i++) {
-            String logName = prefix + i;
-            metadataStore.createLogInNamespaceSync(uri, logName);
-            expectedLogs.add(logName);
-        }
-        return expectedLogs;
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLogs() throws Exception {
-        int numLogs = 3 * maxLogsPerSubnamespace;
-        Set<String> expectedLogs = createLogs(numLogs, "test-get-logs");
-        Set<String> receivedLogs;
-        do {
-            TimeUnit.MILLISECONDS.sleep(20);
-            receivedLogs = new TreeSet<String>();
-            Iterator<String> logs = FutureUtils.result(metadataStore.getLogs());
-            receivedLogs.addAll(Lists.newArrayList(logs));
-        } while (receivedLogs.size() < numLogs);
-        assertEquals(numLogs, receivedLogs.size());
-        assertTrue(Sets.difference(expectedLogs, receivedLogs).isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testNamespaceListener() throws Exception {
-        int numLogs = 3 * maxLogsPerSubnamespace;
-        TestNamespaceListenerWithExpectedSize listener = new TestNamespaceListenerWithExpectedSize(numLogs);
-        metadataStore.registerNamespaceListener(listener);
-        Set<String> expectedLogs = createLogs(numLogs, "test-namespace-listener");
-        listener.waitForDone();
-        Set<String> receivedLogs = listener.getResult();
-        assertEquals(numLogs, receivedLogs.size());
-        assertTrue(Sets.difference(expectedLogs, receivedLogs).isEmpty());
-
-        Random r = new Random(System.currentTimeMillis());
-        int logId = r.nextInt(numLogs);
-        String logName = "test-namespace-listener" + logId;
-        TestNamespaceListener deleteListener = new TestNamespaceListener();
-        metadataStore.registerNamespaceListener(deleteListener);
-        deleteLog(logName);
-        deleteListener.waitForDone();
-        Set<String> logsAfterDeleted = Sets.newTreeSet(Lists.newArrayList(deleteListener.getResult()));
-        assertEquals(numLogs - 1, logsAfterDeleted.size());
-        expectedLogs.remove(logName);
-        assertTrue(Sets.difference(expectedLogs, receivedLogs).isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogPickingFirstAvailableSubNamespace() throws Exception {
-        URI subNs1 = FutureUtils.result(metadataStore.createSubNamespace());
-        URI subNs2 = FutureUtils.result(metadataStore.createSubNamespace());
-
-        Set<String> logs0 = createLogs(uri, maxLogsPerSubnamespace - 1, "test-ns0-");
-        Set<String> logs1 = createLogs(subNs1, maxLogsPerSubnamespace, "test-ns1-");
-        Set<String> logs2 = createLogs(subNs2, maxLogsPerSubnamespace, "test-ns2-");
-        Set<String> allLogs = Sets.newTreeSet();
-        allLogs.addAll(logs0);
-        allLogs.addAll(logs1);
-        allLogs.addAll(logs2);
-
-        // make sure the metadata store saw all 29 logs
-        Set<String> receivedLogs;
-        do {
-            TimeUnit.MILLISECONDS.sleep(20);
-            receivedLogs = new TreeSet<String>();
-            Iterator<String> logs = FutureUtils.result(metadataStore.getLogs());
-            receivedLogs.addAll(Lists.newArrayList(logs));
-        } while (receivedLogs.size() < 3 * maxLogsPerSubnamespace - 1);
-
-        TestNamespaceListenerWithExpectedSize listener =
-                new TestNamespaceListenerWithExpectedSize(3 * maxLogsPerSubnamespace + 1);
-        metadataStore.registerNamespaceListener(listener);
-
-        Set<URI> uris = FutureUtils.result(metadataStore.fetchSubNamespaces(null));
-        assertEquals(3, uris.size());
-        String testLogName = "test-pick-first-available-ns";
-        URI createdURI = FutureUtils.result(metadataStore.createLog(testLogName));
-        allLogs.add(testLogName);
-        assertEquals(uri, createdURI);
-        uris = FutureUtils.result(metadataStore.fetchSubNamespaces(null));
-        assertEquals(3, uris.size());
-        testLogName = "test-create-new-ns";
-        URI newURI = FutureUtils.result(metadataStore.createLog(testLogName));
-        allLogs.add(testLogName);
-        assertFalse(uris.contains(newURI));
-        uris = FutureUtils.result(metadataStore.fetchSubNamespaces(null));
-        assertEquals(4, uris.size());
-
-        listener.waitForDone();
-        receivedLogs = listener.getResult();
-        assertEquals(3 * maxLogsPerSubnamespace + 1, receivedLogs.size());
-        assertEquals(allLogs, receivedLogs);
-    }
-
-    @Test(timeout = 60000)
-    public void testZooKeeperSessionExpired() throws Exception {
-        Set<String> allLogs = createLogs(2 * maxLogsPerSubnamespace, "test-zookeeper-session-expired-");
-        TestNamespaceListenerWithExpectedSize listener =
-                new TestNamespaceListenerWithExpectedSize(2 * maxLogsPerSubnamespace + 1);
-        metadataStore.registerNamespaceListener(listener);
-        ZooKeeperClientUtils.expireSession(zkc, BKNamespaceDriver.getZKServersFromDLUri(uri), zkSessionTimeoutMs);
-        String testLogName = "test-log-name";
-        allLogs.add(testLogName);
-
-        DistributedLogConfiguration anotherConf = new DistributedLogConfiguration();
-        anotherConf.addConfiguration(baseConf);
-        ZooKeeperClient anotherZkc = TestZooKeeperClientBuilder.newBuilder()
-                .uri(uri)
-                .sessionTimeoutMs(zkSessionTimeoutMs)
-                .build();
-        FederatedZKLogMetadataStore anotherMetadataStore =
-                new FederatedZKLogMetadataStore(anotherConf, uri, anotherZkc, scheduler);
-        FutureUtils.result(anotherMetadataStore.createLog(testLogName));
-
-        listener.waitForDone();
-        Set<String> receivedLogs = listener.getResult();
-        assertEquals(2 * maxLogsPerSubnamespace + 1, receivedLogs.size());
-        assertEquals(allLogs, receivedLogs);
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
deleted file mode 100644
index 183a405..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
+++ /dev/null
@@ -1,561 +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 com.twitter.distributedlog.impl.logsegment;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.BookKeeperClient;
-import com.twitter.distributedlog.BookKeeperClientBuilder;
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.Entry;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.exceptions.EndOfLogSegmentException;
-import com.twitter.distributedlog.exceptions.ReadCancelledException;
-import com.twitter.distributedlog.injector.AsyncFailureInjector;
-import com.twitter.distributedlog.logsegment.LogSegmentEntryStore;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.*;
-
-/**
- * Test Case for {@link BKLogSegmentEntryReader}
- */
-public class TestBKLogSegmentEntryReader extends TestDistributedLogBase {
-
-    @Rule
-    public TestName runtime = new TestName();
-    private OrderedScheduler scheduler;
-    private BookKeeperClient bkc;
-    private ZooKeeperClient zkc;
-
-    @Before
-    public void setup() throws Exception {
-        super.setup();
-        zkc = ZooKeeperClientBuilder.newBuilder()
-                .name("test-zk")
-                .zkServers(bkutil.getZkServers())
-                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
-                .zkAclId(conf.getZkAclId())
-                .build();
-        bkc = BookKeeperClientBuilder.newBuilder()
-                .name("test-bk")
-                .dlConfig(conf)
-                .ledgersPath("/ledgers")
-                .zkServers(bkutil.getZkServers())
-                .build();
-        scheduler = OrderedScheduler.newBuilder()
-                .name("test-bk-logsegment-entry-reader")
-                .corePoolSize(1)
-                .build();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        if (null != bkc) {
-            bkc.close();
-        }
-        if (null != scheduler) {
-            scheduler.shutdown();
-        }
-        if (null != zkc) {
-            zkc.close();
-        }
-        super.teardown();
-    }
-
-    BKLogSegmentEntryReader createEntryReader(LogSegmentMetadata segment,
-                                              long startEntryId,
-                                              DistributedLogConfiguration conf)
-            throws Exception {
-        LogSegmentEntryStore store = new BKLogSegmentEntryStore(
-                conf,
-                ConfUtils.getConstDynConf(conf),
-                zkc,
-                bkc,
-                scheduler,
-                null,
-                NullStatsLogger.INSTANCE,
-                AsyncFailureInjector.NULL);
-        return (BKLogSegmentEntryReader) FutureUtils.result(store.openReader(segment, startEntryId));
-    }
-
-    void generateCompletedLogSegments(DistributedLogManager dlm,
-                                      DistributedLogConfiguration conf,
-                                      long numCompletedSegments,
-                                      long segmentSize) throws Exception {
-        long txid = 1L;
-        for (long i = 0; i < numCompletedSegments; i++) {
-            AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
-            for (long j = 1; j <= segmentSize; j++) {
-                FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
-                LogRecord ctrlRecord = DLMTestUtil.getLogRecordInstance(txid);
-                ctrlRecord.setControl();
-                FutureUtils.result(writer.write(ctrlRecord));
-            }
-            Utils.close(writer);
-        }
-    }
-
-    AsyncLogWriter createInprogressLogSegment(DistributedLogManager dlm,
-                                              DistributedLogConfiguration conf,
-                                              long segmentSize) throws Exception {
-        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
-        for (long i = 1L; i <= segmentSize; i++) {
-            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
-            LogRecord ctrlRecord = DLMTestUtil.getLogRecordInstance(i);
-            ctrlRecord.setControl();
-            FutureUtils.result(writer.write(ctrlRecord));
-        }
-        return writer;
-    }
-
-    @Test(timeout = 60000)
-    public void testReadEntriesFromCompleteLogSegment() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setNumPrefetchEntriesPerLogSegment(10);
-        confLocal.setMaxPrefetchEntriesPerLogSegment(10);
-        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
-        generateCompletedLogSegments(dlm, confLocal, 1, 20);
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(segments.size() + " log segments found, expected to be only one",
-                1, segments.size());
-
-        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
-        reader.start();
-        boolean done = false;
-        long txId = 1L;
-        long entryId = 0L;
-        while (!done) {
-            Entry.Reader entryReader;
-            try {
-                entryReader = FutureUtils.result(reader.readNext(1)).get(0);
-            } catch (EndOfLogSegmentException eol) {
-                done = true;
-                continue;
-            }
-            LogRecordWithDLSN record = entryReader.nextRecord();
-            while (null != record) {
-                if (!record.isControl()) {
-                    DLMTestUtil.verifyLogRecord(record);
-                    assertEquals(txId, record.getTransactionId());
-                    ++txId;
-                }
-                DLSN dlsn = record.getDlsn();
-                assertEquals(1L, dlsn.getLogSegmentSequenceNo());
-                assertEquals(entryId, dlsn.getEntryId());
-                record = entryReader.nextRecord();
-            }
-            ++entryId;
-        }
-        assertEquals(21, txId);
-        assertFalse(reader.hasCaughtUpOnInprogress());
-        Utils.close(reader);
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseReaderToCancelPendingReads() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setNumPrefetchEntriesPerLogSegment(10);
-        confLocal.setMaxPrefetchEntriesPerLogSegment(10);
-        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
-        DLMTestUtil.generateCompletedLogSegments(dlm, confLocal, 1, 20);
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(segments.size() + " log segments found, expected to be only one",
-                1, segments.size());
-
-        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
-        List<Future<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) {
-            try {
-                FutureUtils.result(future);
-                fail("The read request should be cancelled");
-            } catch (ReadCancelledException rce) {
-                // expected
-            }
-        }
-        assertFalse(reader.hasCaughtUpOnInprogress());
-        assertTrue("Reader should be closed yet", reader.isClosed());
-    }
-
-    @Test(timeout = 60000)
-    public void testMaxPrefetchEntriesSmallBatch() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setNumPrefetchEntriesPerLogSegment(2);
-        confLocal.setMaxPrefetchEntriesPerLogSegment(10);
-        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
-        generateCompletedLogSegments(dlm, confLocal, 1, 20);
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(segments.size() + " log segments found, expected to be only one",
-                1, segments.size());
-
-        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
-        reader.start();
-
-        // wait for the read ahead entries to become available
-        while (reader.readAheadEntries.size() < 10) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-
-        long txId = 1L;
-        long entryId = 0L;
-
-        assertEquals(10, reader.readAheadEntries.size());
-        assertEquals(10, reader.getNextEntryId());
-        assertFalse(reader.hasCaughtUpOnInprogress());
-        // read first entry
-        Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
-        LogRecordWithDLSN record = entryReader.nextRecord();
-        while (null != record) {
-            if (!record.isControl()) {
-                DLMTestUtil.verifyLogRecord(record);
-                assertEquals(txId, record.getTransactionId());
-                ++txId;
-            }
-            DLSN dlsn = record.getDlsn();
-            assertEquals(1L, dlsn.getLogSegmentSequenceNo());
-            assertEquals(entryId, dlsn.getEntryId());
-            record = entryReader.nextRecord();
-        }
-        ++entryId;
-        assertEquals(2L, txId);
-        // wait for the read ahead entries to become 10 again
-        while (reader.readAheadEntries.size() < 10) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-
-        assertEquals(10, reader.readAheadEntries.size());
-        assertEquals(11, reader.getNextEntryId());
-        assertFalse(reader.hasCaughtUpOnInprogress());
-
-        Utils.close(reader);
-    }
-
-    @Test(timeout = 60000)
-    public void testMaxPrefetchEntriesLargeBatch() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setNumPrefetchEntriesPerLogSegment(10);
-        confLocal.setMaxPrefetchEntriesPerLogSegment(5);
-        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
-        generateCompletedLogSegments(dlm, confLocal, 1, 20);
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(segments.size() + " log segments found, expected to be only one",
-                1, segments.size());
-
-        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
-        reader.start();
-
-        // wait for the read ahead entries to become available
-        while (reader.readAheadEntries.size() < 5) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-
-        long txId = 1L;
-        long entryId = 0L;
-
-        assertEquals(5, reader.readAheadEntries.size());
-        assertEquals(5, reader.getNextEntryId());
-        // read first entry
-        Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
-        LogRecordWithDLSN record = entryReader.nextRecord();
-        while (null != record) {
-            if (!record.isControl()) {
-                DLMTestUtil.verifyLogRecord(record);
-                assertEquals(txId, record.getTransactionId());
-                ++txId;
-            }
-            DLSN dlsn = record.getDlsn();
-            assertEquals(1L, dlsn.getLogSegmentSequenceNo());
-            assertEquals(entryId, dlsn.getEntryId());
-            record = entryReader.nextRecord();
-        }
-        ++entryId;
-        assertEquals(2L, txId);
-        // wait for the read ahead entries to become 10 again
-        while (reader.readAheadEntries.size() < 5) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-
-        assertEquals(5, reader.readAheadEntries.size());
-        assertEquals(6, reader.getNextEntryId());
-        assertFalse(reader.hasCaughtUpOnInprogress());
-
-        Utils.close(reader);
-    }
-
-    @Test(timeout = 60000)
-    public void testMaxPrefetchEntriesSmallSegment() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setNumPrefetchEntriesPerLogSegment(10);
-        confLocal.setMaxPrefetchEntriesPerLogSegment(20);
-        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
-        generateCompletedLogSegments(dlm, confLocal, 1, 5);
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(segments.size() + " log segments found, expected to be only one",
-                1, segments.size());
-
-        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
-        reader.start();
-
-        // wait for the read ahead entries to become available
-        while (reader.readAheadEntries.size() < (reader.getLastAddConfirmed() + 1)) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-
-        long txId = 1L;
-        long entryId = 0L;
-
-        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);
-        LogRecordWithDLSN record = entryReader.nextRecord();
-        while (null != record) {
-            if (!record.isControl()) {
-                DLMTestUtil.verifyLogRecord(record);
-                assertEquals(txId, record.getTransactionId());
-                ++txId;
-            }
-            DLSN dlsn = record.getDlsn();
-            assertEquals(1L, dlsn.getLogSegmentSequenceNo());
-            assertEquals(entryId, dlsn.getEntryId());
-            record = entryReader.nextRecord();
-        }
-        ++entryId;
-        assertEquals(2L, txId);
-        assertEquals(reader.getLastAddConfirmed(), reader.readAheadEntries.size());
-        assertEquals((reader.getLastAddConfirmed() + 1), reader.getNextEntryId());
-        assertFalse(reader.hasCaughtUpOnInprogress());
-
-        Utils.close(reader);
-    }
-
-    @Test(timeout = 60000)
-    public void testReadEntriesFromInprogressSegment() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setNumPrefetchEntriesPerLogSegment(20);
-        confLocal.setMaxPrefetchEntriesPerLogSegment(20);
-        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
-        AsyncLogWriter writer = createInprogressLogSegment(dlm, confLocal, 5);
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(segments.size() + " log segments found, expected to be only one",
-                1, segments.size());
-
-        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
-        reader.start();
-
-        long expectedLastAddConfirmed = 8L;
-        // wait until sending out all prefetch requests
-        while (reader.readAheadEntries.size() < expectedLastAddConfirmed + 2) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals(expectedLastAddConfirmed + 2, reader.getNextEntryId());
-
-        long txId = 1L;
-        long entryId = 0L;
-        while (true) {
-            Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
-            LogRecordWithDLSN record = entryReader.nextRecord();
-            while (null != record) {
-                if (!record.isControl()) {
-                    DLMTestUtil.verifyLogRecord(record);
-                    assertEquals(txId, record.getTransactionId());
-                    ++txId;
-                }
-                DLSN dlsn = record.getDlsn();
-                assertEquals(1L, dlsn.getLogSegmentSequenceNo());
-                assertEquals(entryId, dlsn.getEntryId());
-                record = entryReader.nextRecord();
-            }
-            ++entryId;
-            if (entryId == expectedLastAddConfirmed + 1) {
-                break;
-            }
-        }
-        assertEquals(6L, txId);
-
-        Future<List<Entry.Reader>> nextReadFuture = reader.readNext(1);
-        // write another record to commit previous writes
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId)));
-        // the long poll will be satisfied
-        List<Entry.Reader> nextReadEntries = FutureUtils.result(nextReadFuture);
-        assertEquals(1, nextReadEntries.size());
-        assertTrue(reader.hasCaughtUpOnInprogress());
-        Entry.Reader entryReader = nextReadEntries.get(0);
-        LogRecordWithDLSN record = entryReader.nextRecord();
-        assertNotNull(record);
-        assertTrue(record.isControl());
-        assertNull(entryReader.nextRecord());
-        // once the read is advanced, we will prefetch next record
-        while (reader.getNextEntryId() <= entryId) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals(entryId + 2, reader.getNextEntryId());
-        assertEquals(1, reader.readAheadEntries.size());
-
-        Utils.close(reader);
-        Utils.close(writer);
-    }
-
-    @Test(timeout = 60000)
-    public void testReadEntriesOnStateChange() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setOutputBufferSize(0);
-        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
-        confLocal.setImmediateFlushEnabled(false);
-        confLocal.setNumPrefetchEntriesPerLogSegment(20);
-        confLocal.setMaxPrefetchEntriesPerLogSegment(20);
-        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
-        AsyncLogWriter writer = createInprogressLogSegment(dlm, confLocal, 5);
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        assertEquals(segments.size() + " log segments found, expected to be only one",
-                1, segments.size());
-
-        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
-        reader.start();
-
-        long expectedLastAddConfirmed = 8L;
-        // wait until sending out all prefetch requests
-        while (reader.readAheadEntries.size() < expectedLastAddConfirmed + 2) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals(expectedLastAddConfirmed + 2, reader.getNextEntryId());
-
-        long txId = 1L;
-        long entryId = 0L;
-        while (true) {
-            Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
-            LogRecordWithDLSN record = entryReader.nextRecord();
-            while (null != record) {
-                if (!record.isControl()) {
-                    DLMTestUtil.verifyLogRecord(record);
-                    assertEquals(txId, record.getTransactionId());
-                    ++txId;
-                }
-                DLSN dlsn = record.getDlsn();
-                assertEquals(1L, dlsn.getLogSegmentSequenceNo());
-                assertEquals(entryId, dlsn.getEntryId());
-                record = entryReader.nextRecord();
-            }
-            ++entryId;
-            if (entryId == expectedLastAddConfirmed + 1) {
-                break;
-            }
-        }
-        assertEquals(6L, txId);
-
-        Future<List<Entry.Reader>> nextReadFuture = reader.readNext(1);
-        // write another record to commit previous writes
-        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId)));
-        // the long poll will be satisfied
-        List<Entry.Reader> nextReadEntries = FutureUtils.result(nextReadFuture);
-        assertEquals(1, nextReadEntries.size());
-        Entry.Reader entryReader = nextReadEntries.get(0);
-        LogRecordWithDLSN record = entryReader.nextRecord();
-        assertNotNull(record);
-        assertTrue(record.isControl());
-        assertNull(entryReader.nextRecord());
-        // once the read is advanced, we will prefetch next record
-        while (reader.getNextEntryId() <= entryId) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals(entryId + 2, reader.getNextEntryId());
-        assertEquals(1, reader.readAheadEntries.size());
-
-        // advance the entry id
-        ++entryId;
-        // close the writer, the write will be committed
-        Utils.close(writer);
-        entryReader = FutureUtils.result(reader.readNext(1)).get(0);
-        record = entryReader.nextRecord();
-        assertNotNull(record);
-        assertFalse(record.isControl());
-        assertNull(entryReader.nextRecord());
-        while (reader.getNextEntryId() <= entryId + 1) {
-            TimeUnit.MILLISECONDS.sleep(10);
-        }
-        assertEquals(entryId + 2, reader.getNextEntryId());
-        assertEquals(1, reader.readAheadEntries.size());
-
-        // get the new log segment
-        List<LogSegmentMetadata> newSegments = dlm.getLogSegments();
-        assertEquals(1, newSegments.size());
-        assertFalse(newSegments.get(0).isInProgress());
-        reader.onLogSegmentMetadataUpdated(newSegments.get(0));
-        // when reader received the new log segments. the outstanding long poll
-        // should be cancelled and end of log segment should be signaled correctly
-        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));
-            fail("Should reach end of log segment");
-        } catch (EndOfLogSegmentException eol) {
-            // expected
-        }
-        Utils.close(reader);
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
deleted file mode 100644
index 1b19b2e..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
+++ /dev/null
@@ -1,327 +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 com.twitter.distributedlog.impl.metadata;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.MetadataAccessor;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.DLMetadata;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClusterTestCase;
-import com.twitter.distributedlog.util.DLUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.Utils;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.util.ZkUtils;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Transaction;
-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 org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-import java.util.List;
-
-import static com.twitter.distributedlog.metadata.LogMetadata.*;
-import static com.twitter.distributedlog.impl.metadata.ZKLogStreamMetadataStore.*;
-import static org.junit.Assert.*;
-
-/**
- * Test {@link ZKLogStreamMetadataStore}
- */
-public class TestZKLogStreamMetadataStore extends ZooKeeperClusterTestCase {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestZKLogStreamMetadataStore.class);
-
-    private final static int sessionTimeoutMs = 30000;
-
-    @Rule
-    public TestName testName = new TestName();
-
-    private ZooKeeperClient zkc;
-    private URI uri;
-
-    private static void createLog(ZooKeeperClient zk, URI uri, String logName, String logIdentifier)
-            throws Exception {
-        final String logRootPath = getLogRootPath(uri, logName, logIdentifier);
-        final String logSegmentsPath = logRootPath + LOGSEGMENTS_PATH;
-        final String maxTxIdPath = logRootPath + MAX_TXID_PATH;
-        final String lockPath = logRootPath + LOCK_PATH;
-        final String readLockPath = logRootPath + READ_LOCK_PATH;
-        final String versionPath = logRootPath + VERSION_PATH;
-        final String allocationPath = logRootPath + ALLOCATION_PATH;
-
-        Utils.zkCreateFullPathOptimistic(zk, logRootPath, new byte[0],
-                zk.getDefaultACL(), CreateMode.PERSISTENT);
-        Transaction txn = zk.get().transaction();
-        txn.create(logSegmentsPath, DLUtils.serializeLogSegmentSequenceNumber(
-                        DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO),
-                zk.getDefaultACL(), CreateMode.PERSISTENT);
-        txn.create(maxTxIdPath, DLUtils.serializeTransactionId(0L),
-                zk.getDefaultACL(), CreateMode.PERSISTENT);
-        txn.create(lockPath, DistributedLogConstants.EMPTY_BYTES,
-                zk.getDefaultACL(), CreateMode.PERSISTENT);
-        txn.create(readLockPath, DistributedLogConstants.EMPTY_BYTES,
-                zk.getDefaultACL(), CreateMode.PERSISTENT);
-        txn.create(versionPath, intToBytes(LAYOUT_VERSION),
-                zk.getDefaultACL(), CreateMode.PERSISTENT);
-        txn.create(allocationPath, DistributedLogConstants.EMPTY_BYTES,
-                zk.getDefaultACL(), CreateMode.PERSISTENT);
-        txn.commit();
-    }
-
-    @Before
-    public void setup() throws Exception {
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .name("zkc")
-                .uri(DLMTestUtil.createDLMURI(zkPort, "/"))
-                .sessionTimeoutMs(sessionTimeoutMs)
-                .build();
-        uri = DLMTestUtil.createDLMURI(zkPort, "");
-        try {
-            ZkUtils.createFullPathOptimistic(
-                    zkc.get(),
-                    uri.getPath(),
-                    new byte[0],
-                    ZooDefs.Ids.OPEN_ACL_UNSAFE,
-                    CreateMode.PERSISTENT);
-        } catch (KeeperException.NodeExistsException nee) {
-            logger.debug("The namespace uri already exists.");
-        }
-    }
-
-    @After
-    public void teardown() throws Exception {
-        zkc.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testCheckLogMetadataPathsWithAllocator() throws Exception {
-        String logRootPath = "/" + testName.getMethodName();
-        List<Versioned<byte[]>> metadatas =
-                FutureUtils.result(checkLogMetadataPaths(
-                        zkc.get(), logRootPath, true));
-        assertEquals("Should have 8 paths",
-                8, metadatas.size());
-        for (Versioned<byte[]> path : metadatas.subList(2, metadatas.size())) {
-            assertNull(path.getValue());
-            assertNull(path.getVersion());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testCheckLogMetadataPathsWithoutAllocator() throws Exception {
-        String logRootPath = "/" + testName.getMethodName();
-        List<Versioned<byte[]>> metadatas =
-                FutureUtils.result(checkLogMetadataPaths(
-                        zkc.get(), logRootPath, false));
-        assertEquals("Should have 7 paths",
-                7, metadatas.size());
-        for (Versioned<byte[]> path : metadatas.subList(2, metadatas.size())) {
-            assertNull(path.getValue());
-            assertNull(path.getVersion());
-        }
-    }
-
-    private void testCreateLogMetadataWithMissingPaths(URI uri,
-                                                       String logName,
-                                                       String logIdentifier,
-                                                       List<String> pathsToDelete,
-                                                       boolean ownAllocator,
-                                                       boolean createLogFirst)
-            throws Exception {
-        if (createLogFirst) {
-            createLog(zkc, uri, logName, logIdentifier);
-        }
-        // delete a path
-        for (String path : pathsToDelete) {
-            zkc.get().delete(path, -1);
-        }
-
-        LogMetadataForWriter logMetadata =
-                FutureUtils.result(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));
-
-        if (ownAllocator) {
-            assertEquals("Should have 8 paths : ownAllocator = " + ownAllocator,
-                    8, metadatas.size());
-        } else {
-            assertEquals("Should have 7 paths : ownAllocator = " + ownAllocator,
-                    7, metadatas.size());
-        }
-
-        for (Versioned<byte[]> metadata : metadatas) {
-            assertTrue(pathExists(metadata));
-            assertTrue(((ZkVersion) metadata.getVersion()).getZnodeVersion() >= 0);
-        }
-
-        Versioned<byte[]> logSegmentsData = logMetadata.getMaxLSSNData();
-
-        assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO,
-                DLUtils.deserializeLogSegmentSequenceNumber(logSegmentsData.getValue()));
-
-        Versioned<byte[]> maxTxIdData = logMetadata.getMaxTxIdData();
-
-        assertEquals(0L, DLUtils.deserializeTransactionId(maxTxIdData.getValue()));
-
-        if (ownAllocator) {
-            Versioned<byte[]> allocationData = logMetadata.getAllocationData();
-            assertEquals(0, allocationData.getValue().length);
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogMetadataMissingLogSegmentsPath() throws Exception {
-        String logName = testName.getMethodName();
-        String logIdentifier = "<default>";
-        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
-        List<String> pathsToDelete = Lists.newArrayList(
-                logRootPath + LOGSEGMENTS_PATH);
-        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, false, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogMetadataMissingMaxTxIdPath() throws Exception {
-        String logName = testName.getMethodName();
-        String logIdentifier = "<default>";
-        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
-        List<String> pathsToDelete = Lists.newArrayList(
-                logRootPath + MAX_TXID_PATH);
-        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, false, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogMetadataMissingLockPath() throws Exception {
-        String logName = testName.getMethodName();
-        String logIdentifier = "<default>";
-        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
-        List<String> pathsToDelete = Lists.newArrayList(
-                logRootPath + LOCK_PATH);
-        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, false, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogMetadataMissingReadLockPath() throws Exception {
-        String logName = testName.getMethodName();
-        String logIdentifier = "<default>";
-        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
-        List<String> pathsToDelete = Lists.newArrayList(
-                logRootPath + READ_LOCK_PATH);
-        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, false, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogMetadataMissingVersionPath() throws Exception {
-        String logName = testName.getMethodName();
-        String logIdentifier = "<default>";
-        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
-        List<String> pathsToDelete = Lists.newArrayList(
-                logRootPath + VERSION_PATH);
-        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, false, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogMetadataMissingAllocatorPath() throws Exception {
-        URI uri = DLMTestUtil.createDLMURI(zkPort, "");
-        String logName = testName.getMethodName();
-        String logIdentifier = "<default>";
-        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
-        List<String> pathsToDelete = Lists.newArrayList(
-                logRootPath + ALLOCATION_PATH);
-        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, true, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogMetadataMissingAllPath() throws Exception {
-        String logName = testName.getMethodName();
-        String logIdentifier = "<default>";
-        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
-        List<String> pathsToDelete = Lists.newArrayList(
-                logRootPath + LOGSEGMENTS_PATH,
-                logRootPath + MAX_TXID_PATH,
-                logRootPath + LOCK_PATH,
-                logRootPath + READ_LOCK_PATH,
-                logRootPath + VERSION_PATH,
-                logRootPath + ALLOCATION_PATH);
-        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, true, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogMetadataOnExistedLog() throws Exception {
-        String logName = testName.getMethodName();
-        String logIdentifier = "<default>";
-        List<String> pathsToDelete = Lists.newArrayList();
-        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, true, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogMetadata() throws Exception {
-        String logName = testName.getMethodName();
-        String logIdentifier = "<default>";
-        List<String> pathsToDelete = Lists.newArrayList();
-
-        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, true, false);
-    }
-
-    @Test(timeout = 60000, expected = LogNotFoundException.class)
-    public void testCreateLogMetadataWithCreateIfNotExistsSetToFalse() throws Exception {
-        String logName = testName.getMethodName();
-        String logIdentifier = "<default>";
-        FutureUtils.result(getLog(uri, logName, logIdentifier, zkc, true, false));
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateLogMetadataWithCustomMetadata() throws Exception {
-        String logName = testName.getMethodName();
-        String logIdentifier = "<default>";
-        List<String> pathsToDelete = Lists.newArrayList();
-
-        DLMetadata.create(new BKDLConfig(zkServers, "/ledgers")).update(uri);
-
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-            .conf(new DistributedLogConfiguration())
-            .uri(uri)
-            .build();
-
-        MetadataAccessor accessor = namespace.getNamespaceDriver().getMetadataAccessor(logName);
-        accessor.createOrUpdateMetadata(logName.getBytes("UTF-8"));
-        accessor.close();
-
-        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, true, false);
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStoreUtils.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStoreUtils.java
deleted file mode 100644
index eb9174f..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZKLogStreamMetadataStoreUtils.java
+++ /dev/null
@@ -1,208 +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 com.twitter.distributedlog.impl.metadata;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.metadata.LogMetadata;
-import com.twitter.distributedlog.metadata.LogMetadataForWriter;
-import com.twitter.distributedlog.util.DLUtils;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.junit.Test;
-
-import java.net.URI;
-import java.util.List;
-
-import static com.twitter.distributedlog.impl.metadata.ZKLogStreamMetadataStore.*;
-import static org.junit.Assert.*;
-
-public class TestZKLogStreamMetadataStoreUtils {
-
-    @SuppressWarnings("unchecked")
-    @Test(timeout = 60000, expected = UnexpectedException.class)
-    public void testProcessLogMetadatasMissingMaxTxnId() throws Exception {
-        String rootPath = "/test-missing-max-txn-id";
-        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
-        String logName = "test-log";
-        String logIdentifier = "<default>";
-        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(null, null));
-        processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test(timeout = 60000, expected = UnexpectedException.class)
-    public void testProcessLogMetadatasMissingVersion() throws Exception {
-        String rootPath = "/test-missing-version";
-        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
-        String logName = "test-log";
-        String logIdentifier = "<default>";
-        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1)),
-                new Versioned<byte[]>(null, null));
-        processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test(timeout = 60000, expected = UnexpectedException.class)
-    public void testProcessLogMetadatasWrongVersion() throws Exception {
-        String rootPath = "/test-missing-version";
-        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
-        String logName = "test-log";
-        String logIdentifier = "<default>";
-        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1)),
-                new Versioned<byte[]>(intToBytes(9999), null));
-        processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test(timeout = 60000, expected = UnexpectedException.class)
-    public void testProcessLogMetadatasMissingLockPath() throws Exception {
-        String rootPath = "/test-missing-version";
-        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
-        String logName = "test-log";
-        String logIdentifier = "<default>";
-        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1)),
-                new Versioned<byte[]>(intToBytes(LogMetadata.LAYOUT_VERSION), null),
-                new Versioned<byte[]>(null, null));
-        processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test(timeout = 60000, expected = UnexpectedException.class)
-    public void testProcessLogMetadatasMissingReadLockPath() throws Exception {
-        String rootPath = "/test-missing-version";
-        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
-        String logName = "test-log";
-        String logIdentifier = "<default>";
-        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1)),
-                new Versioned<byte[]>(intToBytes(LogMetadata.LAYOUT_VERSION), null),
-                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
-                new Versioned<byte[]>(null, null));
-        processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test(timeout = 60000, expected = UnexpectedException.class)
-    public void testProcessLogMetadatasMissingLogSegmentsPath() throws Exception {
-        String rootPath = "/test-missing-version";
-        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
-        String logName = "test-log";
-        String logIdentifier = "<default>";
-        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1)),
-                new Versioned<byte[]>(intToBytes(LogMetadata.LAYOUT_VERSION), null),
-                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
-                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
-                new Versioned<byte[]>(null, null));
-        processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test(timeout = 60000, expected = UnexpectedException.class)
-    public void testProcessLogMetadatasMissingAllocatorPath() throws Exception {
-        String rootPath = "/test-missing-version";
-        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
-        String logName = "test-log";
-        String logIdentifier = "<default>";
-        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1)),
-                new Versioned<byte[]>(intToBytes(LogMetadata.LAYOUT_VERSION), null),
-                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
-                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
-                new Versioned<byte[]>(DLUtils.serializeLogSegmentSequenceNumber(1L), new ZkVersion(1)),
-                new Versioned<byte[]>(null, null));
-        processLogMetadatas(uri, logName, logIdentifier, metadatas, true);
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test(timeout = 60000)
-    public void testProcessLogMetadatasNoAllocatorPath() throws Exception {
-        String rootPath = "/test-missing-version";
-        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
-        String logName = "test-log";
-        String logIdentifier = "<default>";
-        Versioned<byte[]> maxTxnIdData =
-                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1));
-        Versioned<byte[]> logSegmentsData =
-                new Versioned<byte[]>(DLUtils.serializeLogSegmentSequenceNumber(1L), new ZkVersion(1));
-        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(null, null),
-                maxTxnIdData,
-                new Versioned<byte[]>(intToBytes(LogMetadata.LAYOUT_VERSION), null),
-                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
-                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
-                logSegmentsData);
-        LogMetadataForWriter metadata =
-                processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
-        assertTrue(maxTxnIdData == metadata.getMaxTxIdData());
-        assertTrue(logSegmentsData == metadata.getMaxLSSNData());
-        assertNull(metadata.getAllocationData().getValue());
-        assertNull(metadata.getAllocationData().getVersion());
-    }
-
-    @SuppressWarnings("unchecked")
-    @Test(timeout = 60000)
-    public void testProcessLogMetadatasAllocatorPath() throws Exception {
-        String rootPath = "/test-missing-version";
-        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
-        String logName = "test-log";
-        String logIdentifier = "<default>";
-        Versioned<byte[]> maxTxnIdData =
-                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1));
-        Versioned<byte[]> logSegmentsData =
-                new Versioned<byte[]>(DLUtils.serializeLogSegmentSequenceNumber(1L), new ZkVersion(1));
-        Versioned<byte[]> allocationData =
-                new Versioned<byte[]>(DLUtils.logSegmentId2Bytes(1L), new ZkVersion(1));
-        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
-                new Versioned<byte[]>(null, null),
-                new Versioned<byte[]>(null, null),
-                maxTxnIdData,
-                new Versioned<byte[]>(intToBytes(LogMetadata.LAYOUT_VERSION), null),
-                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
-                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
-                logSegmentsData,
-                allocationData);
-        LogMetadataForWriter metadata =
-                processLogMetadatas(uri, logName, logIdentifier, metadatas, true);
-        assertTrue(maxTxnIdData == metadata.getMaxTxIdData());
-        assertTrue(logSegmentsData == metadata.getMaxLSSNData());
-        assertTrue(allocationData == metadata.getAllocationData());
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZkMetadataResolver.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZkMetadataResolver.java
deleted file mode 100644
index bbabbb2..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/impl/metadata/TestZkMetadataResolver.java
+++ /dev/null
@@ -1,203 +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 com.twitter.distributedlog.impl.metadata;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.impl.metadata.ZkMetadataResolver;
-import com.twitter.distributedlog.metadata.DLMetadata;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClusterTestCase;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.ZooDefs;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.net.URI;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-public class TestZkMetadataResolver extends ZooKeeperClusterTestCase {
-
-    private static final BKDLConfig bkdlConfig = new BKDLConfig("127.0.0.1:7000", "ledgers");
-    private static final BKDLConfig bkdlConfig2 = new BKDLConfig("127.0.0.1:7000", "ledgers2");
-
-    private ZooKeeperClient zkc;
-    private ZkMetadataResolver resolver;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .uri(createURI("/"))
-                .sessionTimeoutMs(10000)
-                .build();
-        resolver = new ZkMetadataResolver(zkc);
-    }
-
-    @After
-    public void tearDown() throws Exception {
-        zkc.close();
-    }
-
-    private URI createURI(String path) {
-        return URI.create("distributedlog://127.0.0.1:" + zkPort + path);
-    }
-
-    @Test(timeout = 60000)
-    public void testResolveFailures() throws Exception {
-        // resolve unexisted path
-        try {
-            resolver.resolve(createURI("/unexisted/path"));
-            fail("Should fail if no metadata resolved.");
-        } catch (IOException e) {
-            // expected
-        }
-        // resolve existed unbound path
-        Utils.zkCreateFullPathOptimistic(zkc, "/existed/path", new byte[0],
-                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        try {
-            resolver.resolve(createURI("/existed/path"));
-            fail("Should fail if no metadata resolved.");
-        } catch (IOException e) {
-            // expected
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testResolve() throws Exception {
-        DLMetadata dlMetadata = DLMetadata.create(bkdlConfig);
-        dlMetadata.create(createURI("/messaging/distributedlog-testresolve"));
-        DLMetadata dlMetadata2 = DLMetadata.create(bkdlConfig2);
-        dlMetadata2.create(createURI("/messaging/distributedlog-testresolve/child"));
-        assertEquals(dlMetadata,
-                resolver.resolve(createURI("/messaging/distributedlog-testresolve")));
-        assertEquals(dlMetadata2,
-                resolver.resolve(createURI("/messaging/distributedlog-testresolve/child")));
-        assertEquals(dlMetadata2,
-                resolver.resolve(createURI("/messaging/distributedlog-testresolve/child/unknown")));
-        Utils.zkCreateFullPathOptimistic(zkc, "/messaging/distributedlog-testresolve/child/child2", new byte[0],
-                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        assertEquals(dlMetadata2,
-                resolver.resolve(createURI("/messaging/distributedlog-testresolve/child/child2")));
-    }
-
-    @Test(timeout = 60000)
-    public void testEncodeRegionID() throws Exception {
-        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
-
-        URI uri = createURI("/messaging/distributedlog-testencoderegionid/dl1");
-        DLMetadata meta1 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers"));
-        meta1.create(uri);
-        BKDLConfig read1 = BKDLConfig.resolveDLConfig(zkc, uri);
-        BKDLConfig.propagateConfiguration(read1, dlConf);
-        assertFalse(dlConf.getEncodeRegionIDInLogSegmentMetadata());
-
-        BKDLConfig.clearCachedDLConfigs();
-
-        DLMetadata meta2 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers").setEncodeRegionID(true));
-        meta2.update(uri);
-        BKDLConfig read2 = BKDLConfig.resolveDLConfig(zkc, uri);
-        BKDLConfig.propagateConfiguration(read2, dlConf);
-        assertTrue(dlConf.getEncodeRegionIDInLogSegmentMetadata());
-
-        BKDLConfig.clearCachedDLConfigs();
-
-        DLMetadata meta3 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers").setEncodeRegionID(false));
-        meta3.update(uri);
-        BKDLConfig read3 = BKDLConfig.resolveDLConfig(zkc, uri);
-        BKDLConfig.propagateConfiguration(read3, dlConf);
-        assertFalse(dlConf.getEncodeRegionIDInLogSegmentMetadata());
-
-        BKDLConfig.clearCachedDLConfigs();
-    }
-
-    @Test(timeout = 60000)
-    public void testFirstLogSegmentSequenceNumber() throws Exception {
-        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
-
-        URI uri = createURI("/messaging/distributedlog-testfirstledgerseqno/dl1");
-        DLMetadata meta1 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers"));
-        meta1.create(uri);
-        BKDLConfig read1 = BKDLConfig.resolveDLConfig(zkc, uri);
-        BKDLConfig.propagateConfiguration(read1, dlConf);
-        assertEquals(DistributedLogConstants.FIRST_LOGSEGMENT_SEQNO, dlConf.getFirstLogSegmentSequenceNumber());
-
-        BKDLConfig.clearCachedDLConfigs();
-
-        DLMetadata meta2 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers")
-                .setFirstLogSegmentSeqNo(9999L));
-        meta2.update(uri);
-        BKDLConfig read2 = BKDLConfig.resolveDLConfig(zkc, uri);
-        BKDLConfig.propagateConfiguration(read2, dlConf);
-        assertEquals(9999L, dlConf.getFirstLogSegmentSequenceNumber());
-
-        BKDLConfig.clearCachedDLConfigs();
-
-        DLMetadata meta3 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers")
-                .setFirstLogSegmentSeqNo(99L));
-        meta3.update(uri);
-        BKDLConfig read3 = BKDLConfig.resolveDLConfig(zkc, uri);
-        BKDLConfig.propagateConfiguration(read3, dlConf);
-        assertEquals(99L, dlConf.getFirstLogSegmentSequenceNumber());
-
-        BKDLConfig.clearCachedDLConfigs();
-    }
-
-    @Test(timeout = 60000)
-    public void testFederatedNamespace() throws Exception {
-        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
-
-        URI uri = createURI("/messaging/distributedlog-testfederatednamespace/dl1");
-        DLMetadata meta1 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers"));
-        meta1.create(uri);
-        BKDLConfig read1 = BKDLConfig.resolveDLConfig(zkc, uri);
-        BKDLConfig.propagateConfiguration(read1, dlConf);
-        assertTrue(dlConf.getCreateStreamIfNotExists());
-
-        BKDLConfig.clearCachedDLConfigs();
-
-        DLMetadata meta2 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers")
-                .setFederatedNamespace(true));
-        meta2.update(uri);
-        BKDLConfig read2 = BKDLConfig.resolveDLConfig(zkc, uri);
-        BKDLConfig.propagateConfiguration(read2, dlConf);
-        assertFalse(dlConf.getCreateStreamIfNotExists());
-
-        BKDLConfig.clearCachedDLConfigs();
-
-        DLMetadata meta3 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers")
-                .setFederatedNamespace(false));
-        meta3.update(uri);
-        BKDLConfig read3 = BKDLConfig.resolveDLConfig(zkc, uri);
-        BKDLConfig.propagateConfiguration(read3, dlConf);
-        // if it is non-federated namespace, it won't change the create stream behavior.
-        assertFalse(dlConf.getCreateStreamIfNotExists());
-
-        BKDLConfig.clearCachedDLConfigs();
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/limiter/TestRequestLimiter.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/limiter/TestRequestLimiter.java
deleted file mode 100644
index 589539b..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/limiter/TestRequestLimiter.java
+++ /dev/null
@@ -1,60 +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 com.twitter.distributedlog.limiter;
-
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class TestRequestLimiter {
-
-    class MockRequest {
-    }
-
-    class MockRequestLimiter implements RequestLimiter<MockRequest> {
-        int count;
-        MockRequestLimiter() {
-            this.count = 0;
-        }
-        public void apply(MockRequest request) {
-            count++;
-        }
-        public int getCount() {
-            return count;
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testChainedRequestLimiter() throws Exception {
-        MockRequestLimiter limiter1 = new MockRequestLimiter();
-        MockRequestLimiter limiter2 = new MockRequestLimiter();
-        ChainedRequestLimiter.Builder<MockRequest> limiterBuilder =
-                new ChainedRequestLimiter.Builder<MockRequest>();
-        limiterBuilder.addLimiter(limiter1)
-                      .addLimiter(limiter2);
-        ChainedRequestLimiter<MockRequest> limiter = limiterBuilder.build();
-        assertEquals(0, limiter1.getCount());
-        assertEquals(0, limiter2.getCount());
-        limiter.apply(new MockRequest());
-        assertEquals(1, limiter1.getCount());
-        assertEquals(1, limiter2.getCount());
-        limiter.apply(new MockRequest());
-        assertEquals(2, limiter1.getCount());
-        assertEquals(2, limiter2.getCount());
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/lock/TestDistributedLock.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/lock/TestDistributedLock.java
deleted file mode 100644
index 1be7fe3..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/lock/TestDistributedLock.java
+++ /dev/null
@@ -1,835 +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 com.twitter.distributedlog.lock;
-
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.Utils;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClientUtils;
-import com.twitter.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;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.ZooKeeper;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-
-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 com.twitter.distributedlog.lock.ZKSessionLock.asyncParseClientID;
-
-/**
- * Distributed Lock Tests
- */
-public class TestDistributedLock extends TestDistributedLogBase {
-
-    static final Logger logger = LoggerFactory.getLogger(TestDistributedLock.class);
-
-    @Rule
-    public TestName runtime = new TestName();
-
-    private final static int sessionTimeoutMs = 2000;
-
-    private ZooKeeperClient zkc;
-    private ZooKeeperClient zkc0; // used for checking
-    private OrderedScheduler lockStateExecutor;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = ZooKeeperClientBuilder.newBuilder()
-                .name("zkc")
-                .uri(createDLMURI("/"))
-                .sessionTimeoutMs(sessionTimeoutMs)
-                .zkAclId(null)
-                .build();
-        zkc0 = ZooKeeperClientBuilder.newBuilder()
-                .name("zkc0")
-                .uri(createDLMURI("/"))
-                .sessionTimeoutMs(sessionTimeoutMs)
-                .zkAclId(null)
-                .build();
-        lockStateExecutor = OrderedScheduler.newBuilder()
-                .name("test-scheduer")
-                .corePoolSize(1)
-                .build();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        zkc.close();
-        zkc0.close();
-        lockStateExecutor.shutdown();
-    }
-
-    private static void createLockPath(ZooKeeper zk, String lockPath) throws Exception {
-        zk.create(lockPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-    }
-
-    private static List<String> getLockWaiters(ZooKeeperClient zkc, String lockPath) throws Exception {
-        List<String> children = zkc.get().getChildren(lockPath, false);
-        Collections.sort(children, ZKSessionLock.MEMBER_COMPARATOR);
-        return children;
-    }
-
-    static class TestLockFactory {
-        final String lockPath;
-        final String clientId;
-        final OrderedScheduler lockStateExecutor;
-
-        public TestLockFactory(String name,
-                               ZooKeeperClient defaultZkc,
-                               OrderedScheduler lockStateExecutor)
-                throws Exception {
-            this.lockPath = "/" + name + System.currentTimeMillis();
-            this.clientId = name;
-            createLockPath(defaultZkc.get(), lockPath);
-            this.lockStateExecutor = lockStateExecutor;
-
-        }
-        public ZKDistributedLock createLock(int id, ZooKeeperClient zkc) throws Exception {
-            SessionLockFactory lockFactory = new ZKSessionLockFactory(
-                    zkc,
-                    clientId + id,
-                    lockStateExecutor,
-                    0,
-                    Long.MAX_VALUE,
-                    sessionTimeoutMs,
-                    NullStatsLogger.INSTANCE);
-            return new ZKDistributedLock(
-                    this.lockStateExecutor,
-                    lockFactory,
-                    this.lockPath,
-                    Long.MAX_VALUE,
-                    NullStatsLogger.INSTANCE);
-        }
-        public String getLockPath() {
-            return lockPath;
-        }
-    }
-
-    static class CountDownThrowFailPointAction extends FailpointUtils.AbstractFailPointAction {
-
-        final AtomicInteger successCounter;
-        final AtomicInteger failureCounter;
-
-        CountDownThrowFailPointAction(int successCount, int failureCount) {
-            this.successCounter = new AtomicInteger(successCount);
-            this.failureCounter = new AtomicInteger(failureCount);
-        }
-
-        @Override
-        public boolean checkFailPoint() throws IOException {
-            int successCount = successCounter.getAndDecrement();
-            if (successCount > 0) {
-                return true;
-            }
-            int count = failureCounter.getAndDecrement();
-            if (count > 0) {
-                throw new IOException("counter = " + count);
-            }
-            return true;
-        }
-    }
-
-    private SessionLockFactory createLockFactory(String clientId,
-                                                 ZooKeeperClient zkc) {
-        return createLockFactory(clientId, zkc, Long.MAX_VALUE, 0);
-    }
-    private SessionLockFactory createLockFactory(String clientId,
-                                                 ZooKeeperClient zkc,
-                                                 long lockTimeoutMs,
-                                                 int recreationTimes) {
-        return new ZKSessionLockFactory(
-                zkc,
-                clientId,
-                lockStateExecutor,
-                recreationTimes,
-                lockTimeoutMs,
-                sessionTimeoutMs,
-                NullStatsLogger.INSTANCE);
-    }
-
-    private static void checkLockAndReacquire(ZKDistributedLock lock, boolean sync) throws Exception {
-        lock.checkOwnershipAndReacquire();
-        Future<ZKDistributedLock> reacquireFuture = lock.getLockReacquireFuture();
-        if (null != reacquireFuture && sync) {
-            FutureUtils.result(reacquireFuture);
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testZooKeeperConnectionLossOnLockCreation() throws Exception {
-        String lockPath = "/test-zookeeper-connection-loss-on-lock-creation-" + System.currentTimeMillis();
-        String clientId = "zookeeper-connection-loss";
-
-        createLockPath(zkc.get(), lockPath);
-
-        FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss,
-                new CountDownThrowFailPointAction(0, Integer.MAX_VALUE));
-        SessionLockFactory lockFactory = createLockFactory(clientId, zkc, Long.MAX_VALUE, 0);
-        try {
-            try {
-                ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
-                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-                FutureUtils.result(lock.asyncAcquire());
-                fail("Should fail on creating lock if couldn't establishing connections to zookeeper");
-            } catch (IOException ioe) {
-                // expected.
-            }
-        } finally {
-            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss);
-        }
-
-        FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss,
-                new CountDownThrowFailPointAction(0, Integer.MAX_VALUE));
-        lockFactory = createLockFactory(clientId, zkc, Long.MAX_VALUE, 3);
-        try {
-            try {
-                ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
-                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-                FutureUtils.result(lock.asyncAcquire());
-                fail("Should fail on creating lock if couldn't establishing connections to zookeeper after 3 retries");
-            } catch (IOException ioe) {
-                // expected.
-            }
-        } finally {
-            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss);
-        }
-
-        FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss,
-                new CountDownThrowFailPointAction(0, 3));
-        lockFactory = createLockFactory(clientId, zkc, Long.MAX_VALUE, 5);
-        try {
-            ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
-                Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-            FutureUtils.result(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))));
-
-            lock.asyncClose();
-        } finally {
-            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss);
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testBasicAcquireRelease() throws Exception {
-        String lockPath = "/test-basic-acquire-release-" + System.currentTimeMillis();
-        String clientId = "basic-acquire-release";
-
-        createLockPath(zkc.get(), lockPath);
-
-        SessionLockFactory lockFactory = createLockFactory(clientId, zkc);
-        ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
-                Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-        FutureUtils.result(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))));
-
-        FutureUtils.result(lock.asyncClose());
-
-        children = getLockWaiters(zkc, lockPath);
-        assertEquals(0, children.size());
-        assertFalse(lock.haveLock());
-
-        lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
-                Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-        FutureUtils.result(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(lockId1, lockId2);
-
-        FutureUtils.result(lock.asyncClose());
-
-        children = getLockWaiters(zkc, lockPath);
-        assertEquals(0, children.size());
-        assertFalse(lock.haveLock());
-
-        try {
-            FutureUtils.result(lock.asyncAcquire());
-            fail("Should fail on acquiring a closed lock");
-        } catch (UnexpectedException le) {
-            // expected.
-        }
-        children = getLockWaiters(zkc, lockPath);
-        assertEquals(0, children.size());
-        assertFalse(lock.haveLock());
-    }
-
-    @Test(timeout = 60000)
-    public void testCheckWriteLockFailureWhenLockIsAcquiredByOthers() throws Exception {
-        String lockPath = "/test-check-write-lock-failure-when-lock-is-acquired-by-others-" + System.currentTimeMillis();
-        String clientId = "test-check-write-lock-failure";
-
-        createLockPath(zkc.get(), lockPath);
-
-        SessionLockFactory lockFactory0 = createLockFactory(clientId, zkc0);
-        ZKDistributedLock lock0 =
-                new ZKDistributedLock(lockStateExecutor, lockFactory0, lockPath,
-                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-        FutureUtils.result(lock0.asyncAcquire());
-
-        Pair<String, Long> lockId0_1 = ((ZKSessionLock) lock0.getInternalLock()).getLockId();
-
-        List<String> children = getLockWaiters(zkc, lockPath);
-        assertEquals(1, children.size());
-        assertTrue(lock0.haveLock());
-        assertEquals(lockId0_1,
-                Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
-
-        // expire the session
-        ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
-
-        // reacquire the lock and wait reacquire completed
-        checkLockAndReacquire(lock0, true);
-
-        Pair<String, Long> lockId0_2 = ((ZKSessionLock) lock0.getInternalLock()).getLockId();
-        assertFalse("New lock should be created under different session", lockId0_1.equals(lockId0_2));
-
-        children = getLockWaiters(zkc, lockPath);
-        assertEquals(1, children.size());
-        assertTrue(lock0.haveLock());
-        assertEquals(lockId0_2,
-                Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
-
-
-        SessionLockFactory lockFactory = createLockFactory(clientId, zkc);
-        final ZKDistributedLock lock1 =
-                new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
-                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-        final CountDownLatch lockLatch = new CountDownLatch(1);
-        Thread lockThread = new Thread(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    FutureUtils.result(lock1.asyncAcquire());
-                    lockLatch.countDown();
-                } catch (IOException e) {
-                    logger.error("Failed on locking lock1 : ", e);
-                }
-            }
-        }, "lock-thread");
-        lockThread.start();
-
-        // ensure lock1 is waiting for lock0
-        do {
-            Thread.sleep(1);
-            children = getLockWaiters(zkc, lockPath);
-        } while (children.size() < 2);
-
-        // expire the session
-        ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
-
-        lockLatch.await();
-        lockThread.join();
-
-        try {
-            checkLockAndReacquire(lock0, true);
-            fail("Should fail on checking write lock since lock is acquired by lock1");
-        } catch (LockingException le) {
-            // expected
-        }
-
-        try {
-            checkLockAndReacquire(lock0, false);
-            fail("Should fail on checking write lock since lock is acquired by lock1");
-        } catch (LockingException le) {
-            // expected
-        }
-    }
-
-    /**
-     * If no lock is acquired between session expired and re-acquisition, check write lock will acquire the lock.
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testLockReacquireSuccessAfterCheckWriteLock() throws Exception {
-        testLockReacquireSuccess(true);
-    }
-
-    /**
-     * If no lock is acquired between session expired and re-acquisition, check write lock will acquire the lock.
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testLockReacquireSuccessWithoutCheckWriteLock() throws Exception {
-        testLockReacquireSuccess(false);
-    }
-
-    private void testLockReacquireSuccess(boolean checkOwnershipAndReacquire) throws Exception {
-        String lockPath = "/test-lock-re-acquire-success-" + checkOwnershipAndReacquire
-                + "-" + System.currentTimeMillis();
-        String clientId = "test-lock-re-acquire";
-
-        createLockPath(zkc.get(), lockPath);
-
-        SessionLockFactory lockFactory0 = createLockFactory(clientId, zkc0);
-        ZKDistributedLock lock0 =
-                new ZKDistributedLock(lockStateExecutor, lockFactory0, lockPath,
-                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-        FutureUtils.result(lock0.asyncAcquire());
-
-        Pair<String, Long> lockId0_1 = ((ZKSessionLock) lock0.getInternalLock()).getLockId();
-
-        List<String> children = getLockWaiters(zkc, lockPath);
-        assertEquals(1, children.size());
-        assertTrue(lock0.haveLock());
-        assertEquals(lockId0_1,
-                Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
-
-        ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
-
-        if (checkOwnershipAndReacquire) {
-            checkLockAndReacquire(lock0, true);
-            checkLockAndReacquire(lock0, false);
-        } else {
-            // session expire will trigger lock re-acquisition
-            Future<ZKDistributedLock> asyncLockAcquireFuture;
-            do {
-                Thread.sleep(1);
-                asyncLockAcquireFuture = lock0.getLockReacquireFuture();
-            } while (null == asyncLockAcquireFuture && lock0.getReacquireCount() < 1);
-            if (null != asyncLockAcquireFuture) {
-                Await.result(asyncLockAcquireFuture);
-            }
-            checkLockAndReacquire(lock0, false);
-        }
-        children = getLockWaiters(zkc, lockPath);
-        assertEquals(1, children.size());
-        assertTrue(lock0.haveLock());
-        Pair<String, Long> lock0_2 = ((ZKSessionLock) lock0.getInternalLock()).getLockId();
-        assertEquals(lock0_2,
-                Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
-        assertEquals(clientId, lock0_2.getLeft());
-        assertFalse(lockId0_1.equals(lock0_2));
-
-        FutureUtils.result(lock0.asyncClose());
-
-        children = getLockWaiters(zkc, lockPath);
-        assertEquals(0, children.size());
-    }
-
-    /**
-     * If lock is acquired between session expired and re-acquisition, check write lock will be failed.
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testLockReacquireFailureAfterCheckWriteLock() throws Exception {
-        testLockReacquireFailure(true);
-    }
-
-    /**
-     * If lock is acquired between session expired and re-acquisition, check write lock will be failed.
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testLockReacquireFailureWithoutCheckWriteLock() throws Exception {
-        testLockReacquireFailure(false);
-    }
-
-    private void testLockReacquireFailure(boolean checkOwnershipAndReacquire) throws Exception {
-        String lockPath = "/test-lock-re-acquire-failure-" + checkOwnershipAndReacquire
-                + "-" + System.currentTimeMillis();
-        String clientId = "test-lock-re-acquire";
-
-        createLockPath(zkc.get(), lockPath);
-
-        SessionLockFactory lockFactory0 = createLockFactory(clientId, zkc0);
-        ZKDistributedLock lock0 =
-                new ZKDistributedLock(lockStateExecutor, lockFactory0, lockPath,
-                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-        FutureUtils.result(lock0.asyncAcquire());
-
-        final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
-        SessionLockFactory lockFactory1 = createLockFactory(clientId, zkc);
-        final ZKDistributedLock lock1 =
-                new ZKDistributedLock(lockStateExecutor, lockFactory1, lockPath,
-                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
-        Thread lock1Thread = new Thread(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    FutureUtils.result(lock1.asyncAcquire());
-                    lock1DoneLatch.countDown();
-                } catch (IOException e) {
-                    logger.error("Error on acquiring lock1 : ", e);
-                }
-            }
-        }, "lock1-thread");
-        lock1Thread.start();
-
-        List<String> children;
-        do {
-            Thread.sleep(1);
-            children = getLockWaiters(zkc, lockPath);
-        } while (children.size() < 2);
-        assertEquals(2, children.size());
-        assertTrue(lock0.haveLock());
-        assertFalse(lock1.haveLock());
-        assertEquals(((ZKSessionLock) lock0.getInternalLock()).getLockId(),
-                Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
-        assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId(),
-                Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
-
-        logger.info("Expiring session on lock0");
-        ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
-        logger.info("Session on lock0 is expired");
-        lock1DoneLatch.await();
-        assertFalse(lock0.haveLock());
-        assertTrue(lock1.haveLock());
-
-        if (checkOwnershipAndReacquire) {
-            try {
-                checkLockAndReacquire(lock0, true);
-                fail("Should fail check write lock since lock is already held by other people");
-            } catch (OwnershipAcquireFailedException oafe) {
-                assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId().getLeft(),
-                        oafe.getCurrentOwner());
-            }
-            try {
-                checkLockAndReacquire(lock0, false);
-                fail("Should fail check write lock since lock is already held by other people");
-            } catch (OwnershipAcquireFailedException oafe) {
-                assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId().getLeft(),
-                        oafe.getCurrentOwner());
-            }
-        } else {
-            logger.info("Waiting lock0 to attempt acquisition after session expired");
-            // session expire will trigger lock re-acquisition
-            Future<ZKDistributedLock> asyncLockAcquireFuture;
-            do {
-                Thread.sleep(1);
-                asyncLockAcquireFuture = lock0.getLockReacquireFuture();
-            } while (null == asyncLockAcquireFuture);
-
-            try {
-                Await.result(asyncLockAcquireFuture);
-                fail("Should fail check write lock since lock is already held by other people");
-            } catch (OwnershipAcquireFailedException oafe) {
-                assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId().getLeft(),
-                        oafe.getCurrentOwner());
-            }
-            try {
-                checkLockAndReacquire(lock0, false);
-                fail("Should fail check write lock since lock is already held by other people");
-            } catch (OwnershipAcquireFailedException oafe) {
-                assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId().getLeft(),
-                        oafe.getCurrentOwner());
-            }
-        }
-        children = getLockWaiters(zkc, lockPath);
-        assertEquals(1, children.size());
-        assertFalse(lock0.haveLock());
-        assertTrue(lock1.haveLock());
-        assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId(),
-                Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
-
-        FutureUtils.result(lock0.asyncClose());
-        FutureUtils.result(lock1.asyncClose());
-
-        children = getLockWaiters(zkc, lockPath);
-        assertEquals(0, children.size());
-    }
-
-    @Test(timeout = 60000)
-    public void testLockReacquire() throws Exception {
-        String lockPath = "/reacquirePath";
-        Utils.zkCreateFullPathOptimistic(zkc, lockPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE,
-                CreateMode.PERSISTENT);
-        String clientId = "lockHolder";
-        SessionLockFactory lockFactory = createLockFactory(clientId, zkc, conf.getLockTimeoutMilliSeconds(), 0);
-        ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
-            conf.getLockTimeoutMilliSeconds(), NullStatsLogger.INSTANCE);
-        FutureUtils.result(lock.asyncAcquire());
-
-        // try and cleanup the underlying lock
-        lock.getInternalLock().unlock();
-
-        // This should reacquire the lock
-        checkLockAndReacquire(lock, true);
-
-        assertEquals(true, lock.haveLock());
-        assertEquals(true, lock.getInternalLock().isLockHeld());
-
-        lockFactory = createLockFactory(clientId + "_2", zkc, conf.getLockTimeoutMilliSeconds(), 0);
-        ZKDistributedLock lock2 = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
-            0, NullStatsLogger.INSTANCE);
-
-        boolean exceptionEncountered = false;
-        try {
-            FutureUtils.result(lock2.asyncAcquire());
-        } catch (OwnershipAcquireFailedException exc) {
-            assertEquals(clientId, exc.getCurrentOwner());
-            exceptionEncountered = true;
-        }
-        assertTrue(exceptionEncountered);
-        FutureUtils.result(lock.asyncClose());
-        FutureUtils.result(lock2.asyncClose());
-    }
-
-    @Test(timeout = 60000)
-    public void testLockReacquireMultiple() throws Exception {
-        String lockPath = "/reacquirePathMultiple";
-        Utils.zkCreateFullPathOptimistic(zkc, lockPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE,
-            CreateMode.PERSISTENT);
-        String clientId = "lockHolder";
-        SessionLockFactory factory = createLockFactory(clientId, zkc, conf.getLockTimeoutMilliSeconds(), 0);
-        ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, factory, lockPath,
-            conf.getLockTimeoutMilliSeconds(), NullStatsLogger.INSTANCE);
-        FutureUtils.result(lock.asyncAcquire());
-
-        // try and cleanup the underlying lock
-        lock.getInternalLock().unlock();
-
-        // This should reacquire the lock
-        checkLockAndReacquire(lock, true);
-
-        assertEquals(true, lock.haveLock());
-        assertEquals(true, lock.getInternalLock().isLockHeld());
-
-        factory = createLockFactory(clientId + "_2", zkc, 0, 0);
-        ZKDistributedLock lock2 = new ZKDistributedLock(lockStateExecutor, factory, lockPath,
-            0, NullStatsLogger.INSTANCE);
-
-        boolean exceptionEncountered = false;
-        try {
-            FutureUtils.result(lock2.asyncAcquire());
-        } catch (OwnershipAcquireFailedException exc) {
-            assertEquals(clientId, exc.getCurrentOwner());
-            exceptionEncountered = true;
-        }
-        assertTrue(exceptionEncountered);
-        FutureUtils.result(lock2.asyncClose());
-
-        FutureUtils.result(lock.asyncClose());
-        assertEquals(false, lock.haveLock());
-        assertEquals(false, lock.getInternalLock().isLockHeld());
-
-        factory = createLockFactory(clientId + "_3", zkc, 0, 0);
-        ZKDistributedLock lock3 = new ZKDistributedLock(lockStateExecutor, factory, lockPath,
-            0, NullStatsLogger.INSTANCE);
-
-        FutureUtils.result(lock3.asyncAcquire());
-        assertEquals(true, lock3.haveLock());
-        assertEquals(true, lock3.getInternalLock().isLockHeld());
-        FutureUtils.result(lock3.asyncClose());
-    }
-
-    void assertLatchesSet(CountDownLatch[] latches, int endIndex) {
-        for (int i = 0; i < endIndex; i++) {
-            assertEquals("latch " + i + " should have been set", 0, latches[i].getCount());
-        }
-        for (int i = endIndex; i < latches.length; i++) {
-            assertEquals("latch " + i + " should not have been set", 1, latches[i].getCount());
-        }
-    }
-
-    // Assert key lock state (is locked, is internal locked, lock count, etc.) for two dlocks.
-    void assertLockState(ZKDistributedLock lock0, boolean owned0, boolean intOwned0,
-                         ZKDistributedLock lock1, boolean owned1, boolean intOwned1,
-                         int waiters, String lockPath) throws Exception {
-        assertEquals(owned0, lock0.haveLock());
-        assertEquals(intOwned0, lock0.getInternalLock() != null && lock0.getInternalLock().isLockHeld());
-        assertEquals(owned1, lock1.haveLock());
-        assertEquals(intOwned1, lock1.getInternalLock() != null && lock1.getInternalLock().isLockHeld());
-        assertEquals(waiters, getLockWaiters(zkc, lockPath).size());
-    }
-
-    @Test(timeout = 60000)
-    public void testAsyncAcquireBasics() throws Exception {
-        TestLockFactory locks = new TestLockFactory(runtime.getMethodName(), zkc, lockStateExecutor);
-
-        int count = 3;
-        ArrayList<Future<ZKDistributedLock>> results =
-                new ArrayList<Future<ZKDistributedLock>>(count);
-        ZKDistributedLock[] lockArray = new ZKDistributedLock[count];
-        final CountDownLatch[] latches = new CountDownLatch[count];
-
-        // Set up <count> waiters, save async results, count down a latch when lock is acquired in
-        // the future.
-        for (int i = 0; i < count; i++) {
-            latches[i] = new CountDownLatch(1);
-            lockArray[i] = locks.createLock(i, zkc);
-            final int index = i;
-            results.add(lockArray[i].asyncAcquire().addEventListener(
-                new FutureEventListener<ZKDistributedLock>() {
-                    @Override
-                    public void onSuccess(ZKDistributedLock lock) {
-                        latches[index].countDown();
-                    }
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        fail("unexpected failure " + cause);
-                    }
-                }
-            ));
-        }
-
-        // Now await ownership and release ownership of locks one by one (in the order they were
-        // acquired).
-        for (int i = 0; i < count; i++) {
-            latches[i].await();
-            assertLatchesSet(latches, i+1);
-            Await.result(results.get(i));
-            FutureUtils.result(lockArray[i].asyncClose());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testAsyncAcquireSyncThenAsyncOnSameLock() throws Exception {
-        TestLockFactory locks = new TestLockFactory(runtime.getMethodName(), zkc, lockStateExecutor);
-        final ZKDistributedLock lock0 = locks.createLock(0, zkc);
-        final ZKDistributedLock lock1 = locks.createLock(1, zkc0);
-
-        FutureUtils.result(lock0.asyncAcquire());
-
-        // Initial state.
-        assertLockState(lock0, true, true, lock1, false, false, 1, locks.getLockPath());
-
-        Thread lock1Thread = new Thread(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    FutureUtils.result(lock1.asyncAcquire());
-                } catch (IOException e) {
-                    fail("shouldn't fail to acquire");
-                }
-            }
-        }, "lock1-thread");
-        lock1Thread.start();
-
-        // Wait for lock count to increase, indicating background acquire has succeeded.
-        while (getLockWaiters(zkc, locks.getLockPath()).size() < 2) {
-            Thread.sleep(1);
-        }
-        assertLockState(lock0, true, true, lock1, false, false, 2, locks.getLockPath());
-
-        FutureUtils.result(lock0.asyncClose());
-        Await.result(lock1.getLockAcquireFuture());
-
-        assertLockState(lock0, false, false, lock1, true, true, 1, locks.getLockPath());
-
-        // Release lock1
-        FutureUtils.result(lock1.asyncClose());
-        assertLockState(lock0, false, false, lock1, false, false, 0, locks.getLockPath());
-    }
-
-    @Test(timeout = 60000)
-    public void testAsyncAcquireExpireDuringWait() throws Exception {
-        TestLockFactory locks = new TestLockFactory(runtime.getMethodName(), zkc, lockStateExecutor);
-        final ZKDistributedLock lock0 = locks.createLock(0, zkc);
-        final ZKDistributedLock lock1 = locks.createLock(1, zkc0);
-
-        FutureUtils.result(lock0.asyncAcquire());
-        Future<ZKDistributedLock> result = lock1.asyncAcquire();
-        // make sure we place a waiter for lock1
-        while (null == lock1.getLockWaiter()) {
-            TimeUnit.MILLISECONDS.sleep(20);
-        }
-
-        // Expire causes acquire future to be failed and unset.
-        ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
-        try {
-            Await.result(result);
-            fail("future should have been failed");
-        } catch (OwnershipAcquireFailedException ex) {
-        }
-
-        assertLockState(lock0, true, true, lock1, false, false, 1, locks.getLockPath());
-        lock0.asyncClose();
-        lock1.asyncClose();
-    }
-
-    @Test(timeout = 60000)
-    public void testAsyncAcquireCloseDuringWait() throws Exception {
-        TestLockFactory locks = new TestLockFactory(runtime.getMethodName(), zkc, lockStateExecutor);
-        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());
-        try {
-            Await.result(result);
-            fail("future should have been failed");
-        } catch (LockClosedException ex) {
-        }
-
-        assertLockState(lock0, true, true, lock1, false, false, 1, locks.getLockPath());
-        lock0.asyncClose();
-    }
-
-    @Test(timeout = 60000)
-    public void testAsyncAcquireCloseAfterAcquire() throws Exception {
-        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());
-
-        // Already have this, stays satisfied.
-        Await.result(result);
-
-        // But we no longer have the lock.
-        assertEquals(false, lock0.haveLock());
-        assertEquals(false, lock0.getInternalLock().isLockHeld());
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/lock/TestZKSessionLock.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/lock/TestZKSessionLock.java
deleted file mode 100644
index 054d714..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/lock/TestZKSessionLock.java
+++ /dev/null
@@ -1,1224 +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 com.twitter.distributedlog.lock;
-
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClientUtils;
-import com.twitter.distributedlog.ZooKeeperClusterTestCase;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.distributedlog.lock.ZKSessionLock.State;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.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.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooDefs;
-import org.apache.zookeeper.ZooKeeper;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-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;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static com.twitter.distributedlog.lock.ZKSessionLock.*;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-/**
- * Distributed Lock Tests
- */
-public class TestZKSessionLock extends ZooKeeperClusterTestCase {
-
-    @Rule
-    public TestName testNames = new TestName();
-
-    static final Logger logger = LoggerFactory.getLogger(TestZKSessionLock.class);
-
-    private final static int sessionTimeoutMs = 2000;
-
-    private ZooKeeperClient zkc;
-    private ZooKeeperClient zkc0; // used for checking
-    private OrderedScheduler lockStateExecutor;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = ZooKeeperClientBuilder.newBuilder()
-                .name("zkc")
-                .uri(DLMTestUtil.createDLMURI(zkPort, "/"))
-                .sessionTimeoutMs(sessionTimeoutMs)
-                .zkServers(zkServers)
-                .zkAclId(null)
-                .build();
-        zkc0 = ZooKeeperClientBuilder.newBuilder()
-                .name("zkc0")
-                .uri(DLMTestUtil.createDLMURI(zkPort, "/"))
-                .sessionTimeoutMs(sessionTimeoutMs)
-                .zkServers(zkServers)
-                .zkAclId(null)
-                .build();
-        lockStateExecutor = OrderedScheduler.newBuilder()
-                .corePoolSize(1)
-                .build();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        zkc.close();
-        zkc0.close();
-        lockStateExecutor.shutdown();
-    }
-
-    private static void createLockPath(ZooKeeper zk, String lockPath) throws Exception {
-        zk.create(lockPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-    }
-
-    private static String createLockNodeV1(ZooKeeper zk, String lockPath, String clientId) throws Exception {
-        return zk.create(getLockPathPrefixV1(lockPath), serializeClientId(clientId),
-                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
-    }
-
-    private static String createLockNodeV2(ZooKeeper zk, String lockPath, String clientId) throws Exception {
-        return zk.create(getLockPathPrefixV2(lockPath, clientId), serializeClientId(clientId),
-                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
-    }
-
-    private static String createLockNodeV3(ZooKeeper zk, String lockPath, String clientId) throws Exception {
-        return zk.create(getLockPathPrefixV3(lockPath, clientId, zk.getSessionId()), serializeClientId(clientId),
-                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
-    }
-
-    private static String createLockNodeWithBadNodeName(ZooKeeper zk, String lockPath, String clientId, String badNodeName)
-            throws Exception {
-        return zk.create(lockPath + "/" + badNodeName, serializeClientId(clientId),
-                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
-    }
-
-    private static List<String> getLockWaiters(ZooKeeperClient zkc, String lockPath) throws Exception {
-        List<String> children = zkc.get().getChildren(lockPath, false);
-        Collections.sort(children, ZKSessionLock.MEMBER_COMPARATOR);
-        return children;
-    }
-
-    @Test(timeout = 60000)
-    public void testParseClientID() throws Exception {
-        ZooKeeper zk = zkc.get();
-
-        String lockPath = "/test-parse-clientid";
-        String clientId = "test-parse-clientid-" + System.currentTimeMillis();
-        Pair<String, Long> lockId = Pair.of(clientId, zk.getSessionId());
-
-        createLockPath(zk, lockPath);
-
-        // Correct data
-        String node1 = getLockIdFromPath(createLockNodeV1(zk, lockPath, clientId));
-        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)));
-
-        // Bad Lock Node Name
-        String node4 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member"));
-        String node5 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode"));
-        String node6 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode_badnode"));
-        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)));
-
-        // 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)));
-    }
-
-    @Test(timeout = 60000)
-    public void testParseMemberID() throws Exception {
-        assertEquals(Integer.MAX_VALUE, parseMemberID("badnode"));
-        assertEquals(Integer.MAX_VALUE, parseMemberID("badnode_badnode"));
-        assertEquals(0, parseMemberID("member_000000"));
-        assertEquals(123, parseMemberID("member_000123"));
-    }
-
-    @Test(timeout = 60000)
-    public void testAreLockWaitersInSameSession() throws Exception {
-        ZooKeeper zk = zkc.get();
-
-        String lockPath = "/test-are-lock-waiters-in-same-session";
-        String clientId1 = "test-are-lock-waiters-in-same-session-1";
-        String clientId2 = "test-are-lock-waiters-in-same-session-2";
-
-        createLockPath(zk, lockPath);
-
-        String node1 = getLockIdFromPath(createLockNodeV3(zk, lockPath, clientId1));
-        String node2 = getLockIdFromPath(createLockNodeV3(zk, lockPath, clientId2));
-        String node3 = getLockIdFromPath(createLockNodeV3(zk, lockPath, clientId1));
-
-        assertEquals(node1 + " and " + node3 + " should be in same session.",
-                true, areLockWaitersInSameSession(node1, node3));
-        assertEquals(node1 + " and " + node2 + " should be not in same session.",
-                false, areLockWaitersInSameSession(node1, node2));
-        assertEquals(node3 + " and " + node2 + " should be not in same session.",
-                false, areLockWaitersInSameSession(node3, node2));
-    }
-
-    @Test(timeout = 60000)
-    public void testExecuteLockAction() throws Exception {
-        String lockPath = "/test-execute-lock-action";
-        String clientId = "test-execute-lock-action-" + System.currentTimeMillis();
-
-        ZKSessionLock lock =
-                new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
-
-        final AtomicInteger counter = new AtomicInteger(0);
-
-        // lock action would be executed in same epoch
-        final CountDownLatch latch1 = new CountDownLatch(1);
-        lock.executeLockAction(lock.getEpoch().get(), new LockAction() {
-            @Override
-            public void execute() {
-                counter.incrementAndGet();
-                latch1.countDown();
-            }
-
-            @Override
-            public String getActionName() {
-                return "increment1";
-            }
-        });
-        latch1.await();
-        assertEquals("counter should be increased in same epoch", 1, counter.get());
-
-        // lock action would not be executed in same epoch
-        final CountDownLatch latch2 = new CountDownLatch(1);
-        lock.executeLockAction(lock.getEpoch().get() + 1, new LockAction() {
-            @Override
-            public void execute() {
-                counter.incrementAndGet();
-            }
-
-            @Override
-            public String getActionName() {
-                return "increment2";
-            }
-        });
-        lock.executeLockAction(lock.getEpoch().get(), new LockAction() {
-            @Override
-            public void execute() {
-                latch2.countDown();
-            }
-
-            @Override
-            public String getActionName() {
-                return "countdown";
-            }
-        });
-        latch2.await();
-        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>();
-        lock.executeLockAction(lock.getEpoch().get() + 1, new LockAction() {
-            @Override
-            public void execute() {
-                counter.incrementAndGet();
-            }
-
-            @Override
-            public String getActionName() {
-                return "increment3";
-            }
-        }, promise);
-        try {
-            Await.result(promise);
-            fail("Should satisfy promise with epoch changed exception.");
-        } catch (EpochChangedException ece) {
-            // expected
-        }
-        assertEquals("counter should not be increased in different epochs", 1, counter.get());
-
-        lockStateExecutor.shutdown();
-    }
-
-    /**
-     * Test lock after unlock is called.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testLockAfterUnlock() throws Exception {
-        String lockPath = "/test-lock-after-unlock";
-        String clientId = "test-lock-after-unlock";
-
-        ZKSessionLock lock = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
-        lock.unlock();
-        assertEquals(State.CLOSED, lock.getLockState());
-
-        try {
-            lock.tryLock(0, TimeUnit.MILLISECONDS);
-            fail("Should fail on tryLock since lock state has changed.");
-        } catch (LockStateChangedException lsce) {
-            // expected
-        }
-        assertEquals(State.CLOSED, lock.getLockState());
-
-        try {
-            lock.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-            fail("Should fail on tryLock immediately if lock state has changed.");
-        } catch (LockStateChangedException lsce) {
-            // expected
-        }
-        assertEquals(State.CLOSED, lock.getLockState());
-    }
-
-    class DelayFailpointAction extends FailpointUtils.AbstractFailPointAction {
-        long timeout;
-        DelayFailpointAction(long timeout) {
-            this.timeout = timeout;
-        }
-        @Override
-        public boolean checkFailPoint() throws IOException {
-            try {
-                Thread.sleep(timeout);
-            } catch (InterruptedException ie) {
-            }
-            return true;
-        }
-    }
-
-    /**
-     * Test unlock timeout.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testUnlockTimeout() throws Exception {
-        String name = testNames.getMethodName();
-        String lockPath = "/" + name;
-        String clientId = name;
-
-        createLockPath(zkc.get(), lockPath);
-
-        ZKSessionLock lock = new ZKSessionLock(
-                zkc, lockPath, clientId, lockStateExecutor,
-                1*1000 /* op timeout */, NullStatsLogger.INSTANCE,
-                new DistributedLockContext());
-
-        lock.tryLock(0, TimeUnit.MILLISECONDS);
-        assertEquals(State.CLAIMED, lock.getLockState());
-
-        try {
-            FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_LockUnlockCleanup,
-                                        new DelayFailpointAction(60*60*1000));
-
-            lock.unlock();
-            assertEquals(State.CLOSING, lock.getLockState());
-        } finally {
-            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_LockUnlockCleanup);
-        }
-    }
-
-    /**
-     * Test try-create after close race condition.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testTryCloseRaceCondition() throws Exception {
-        String name = testNames.getMethodName();
-        String lockPath = "/" + name;
-        String clientId = name;
-
-        createLockPath(zkc.get(), lockPath);
-
-        ZKSessionLock lock = new ZKSessionLock(
-                zkc, lockPath, clientId, lockStateExecutor,
-                1*1000 /* op timeout */, NullStatsLogger.INSTANCE,
-                new DistributedLockContext());
-
-        try {
-            FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_LockTryCloseRaceCondition,
-                                        FailpointUtils.DEFAULT_ACTION);
-
-            lock.tryLock(0, TimeUnit.MILLISECONDS);
-        } catch (LockClosedException ex) {
-            ;
-        } finally {
-            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_LockTryCloseRaceCondition);
-        }
-
-        assertEquals(State.CLOSED, lock.getLockState());
-        List<String> children = getLockWaiters(zkc, lockPath);
-        assertEquals(0, children.size());
-    }
-
-    /**
-     * Test try acquire timeout.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testTryAcquireTimeout() throws Exception {
-        String name = testNames.getMethodName();
-        String lockPath = "/" + name;
-        String clientId = name;
-
-        createLockPath(zkc.get(), lockPath);
-
-        ZKSessionLock lock = new ZKSessionLock(
-                zkc, lockPath, clientId, lockStateExecutor,
-                1 /* op timeout */, NullStatsLogger.INSTANCE,
-                new DistributedLockContext());
-
-        try {
-            FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_LockTryAcquire,
-                                        new DelayFailpointAction(60*60*1000));
-
-            lock.tryLock(0, TimeUnit.MILLISECONDS);
-            assertEquals(State.CLOSED, lock.getLockState());
-        } catch (LockingException le) {
-        } catch (Exception e) {
-            fail("expected locking exception");
-        } finally {
-            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_LockTryAcquire);
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testBasicLockUnlock0() throws Exception {
-        testBasicLockUnlock(0);
-    }
-
-    @Test(timeout = 60000)
-    public void testBasicLockUnlock1() throws Exception {
-        testBasicLockUnlock(Long.MAX_VALUE);
-    }
-
-    /**
-     * Test Basic Lock and Unlock
-     *
-     * - lock should succeed if there is no lock held
-     * - lock should fail on a success lock
-     * - unlock should release the held lock
-     *
-     * @param timeout
-     *          timeout to wait for the lock
-     * @throws Exception
-     */
-    private void testBasicLockUnlock(long timeout) throws Exception {
-        String lockPath = "/test-basic-lock-unlock-" + timeout + System.currentTimeMillis();
-        String clientId = "test-basic-lock-unlock";
-
-        createLockPath(zkc.get(), lockPath);
-
-        ZKSessionLock lock = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
-        // lock
-        lock.tryLock(timeout, TimeUnit.MILLISECONDS);
-        // verification after lock
-        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))));
-
-        // lock should fail on a success lock
-        try {
-            lock.tryLock(timeout, TimeUnit.MILLISECONDS);
-            fail("Should fail on locking a failure lock.");
-        } catch (LockStateChangedException lsce) {
-            // expected
-        }
-        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))));
-
-        // unlock
-        lock.unlock();
-        // verification after unlock
-        assertEquals(State.CLOSED, lock.getLockState());
-        assertEquals(0, getLockWaiters(zkc, lockPath).size());
-    }
-
-    /**
-     * Test lock on non existed lock.
-     *
-     * - lock should fail on a non existed lock.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testLockOnNonExistedLock() throws Exception {
-        String lockPath = "/test-lock-on-non-existed-lock";
-        String clientId = "test-lock-on-non-existed-lock";
-
-        ZKSessionLock lock = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
-        // lock
-        try {
-            lock.tryLock(0, TimeUnit.MILLISECONDS);
-            fail("Should fail on locking a non-existed lock.");
-        } catch (LockingException le) {
-            Throwable cause = le.getCause();
-            assertTrue(cause instanceof KeeperException);
-            assertEquals(KeeperException.Code.NONODE, ((KeeperException) cause).code());
-        }
-        assertEquals(State.CLOSED, lock.getLockState());
-
-        // lock should failed on a failure lock
-        try {
-            lock.tryLock(0, TimeUnit.MILLISECONDS);
-            fail("Should fail on locking a failure lock.");
-        } catch (LockStateChangedException lsce) {
-            // expected
-        }
-        assertEquals(State.CLOSED, lock.getLockState());
-    }
-
-    @Test(timeout = 60000)
-    public void testLockWhenSomeoneHeldLock0() throws Exception {
-        testLockWhenSomeoneHeldLock(0);
-    }
-
-    @Test(timeout = 60000)
-    public void testLockWhenSomeoneHeldLock1() throws Exception {
-        testLockWhenSomeoneHeldLock(500);
-    }
-
-    /**
-     * Test lock if the lock is already held by someone else. Any lock in this situation will
-     * fail with current owner.
-     *
-     * @param timeout
-     *          timeout to wait for the lock
-     * @throws Exception
-     */
-    private void testLockWhenSomeoneHeldLock(long timeout) throws Exception {
-        String lockPath = "/test-lock-nowait-" + timeout + "-" + System.currentTimeMillis();
-        String clientId0 = "test-lock-nowait-0-" + System.currentTimeMillis();
-        String clientId1 = "test-lock-nowait-1-" + System.currentTimeMillis();
-        String clientId2 = "test-lock-nowait-2-" + System.currentTimeMillis();
-
-        createLockPath(zkc.get(), lockPath);
-
-        ZKSessionLock lock0 = new ZKSessionLock(zkc0, lockPath, clientId0, lockStateExecutor);
-        ZKSessionLock lock1 = new ZKSessionLock(zkc, lockPath, clientId1, lockStateExecutor);
-
-        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        // verification after lock0 lock
-        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))));
-
-        try {
-            lock1.tryLock(timeout, TimeUnit.MILLISECONDS);
-            fail("lock1 should fail on locking since lock0 is holding the lock.");
-        } catch (OwnershipAcquireFailedException oafe) {
-            assertEquals(lock0.getLockId().getLeft(), oafe.getCurrentOwner());
-        }
-        // verification after lock1 tryLock
-        assertEquals(State.CLAIMED, lock0.getLockState());
-        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))));
-
-        lock0.unlock();
-        // verification after unlock lock0
-        assertEquals(State.CLOSED, lock0.getLockState());
-        assertEquals(0, getLockWaiters(zkc, lockPath).size());
-
-        ZKSessionLock lock2 = new ZKSessionLock(zkc, lockPath, clientId2, lockStateExecutor);
-        lock2.tryLock(timeout, TimeUnit.MILLISECONDS);
-        // verification after lock2 lock
-        assertEquals(State.CLOSED, lock0.getLockState());
-        assertEquals(State.CLOSED, lock1.getLockState());
-        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))));
-
-        lock2.unlock();
-    }
-
-    @Test(timeout = 60000)
-    public void testLockWhenPreviousLockZnodeStillExists() throws Exception {
-        String lockPath = "/test-lock-when-previous-lock-znode-still-exists-" +
-                System.currentTimeMillis();
-        String clientId = "client-id";
-
-        ZooKeeper zk = zkc.get();
-
-        createLockPath(zk, lockPath);
-
-        final ZKSessionLock lock0 = new ZKSessionLock(zkc0, lockPath, clientId, lockStateExecutor);
-        // lock0 lock
-        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-
-        // simulate lock0 expires but znode still exists
-        final DistributedLockContext context1 = new DistributedLockContext();
-        context1.addLockId(lock0.getLockId());
-
-        final ZKSessionLock lock1 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor,
-                60000, NullStatsLogger.INSTANCE, context1);
-        lock1.tryLock(0L, TimeUnit.MILLISECONDS);
-        assertEquals(State.CLAIMED, lock1.getLockState());
-        lock1.unlock();
-
-        final DistributedLockContext context2 = new DistributedLockContext();
-        context2.addLockId(lock0.getLockId());
-
-        final ZKSessionLock lock2 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor,
-                60000, NullStatsLogger.INSTANCE, context2);
-        lock2.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        assertEquals(State.CLAIMED, lock2.getLockState());
-        lock2.unlock();
-
-        lock0.unlock();
-    }
-
-    @Test(timeout = 60000)
-    public void testWaitForLockUnlock() throws Exception {
-        testWaitForLockReleased("/test-wait-for-lock-unlock", true);
-    }
-
-    @Test(timeout = 60000)
-    public void testWaitForLockExpired() throws Exception {
-        testWaitForLockReleased("/test-wait-for-lock-expired", false);
-    }
-
-    /**
-     * Test lock wait for the lock owner to release the lock. The lock waiter should acquire lock successfully
-     * if the lock owner unlock or it is expired.
-     *
-     * @param lockPath
-     *          lock path
-     * @param isUnlock
-     *          whether to unlock or expire the lock
-     * @throws Exception
-     */
-    private void testWaitForLockReleased(String lockPath, boolean isUnlock) throws Exception {
-        String clientId0 = "test-wait-for-lock-released-0-" + System.currentTimeMillis();
-        String clientId1 = "test-wait-for-lock-released-1-" + System.currentTimeMillis();
-
-        createLockPath(zkc.get(), lockPath);
-
-        final ZKSessionLock lock0 = new ZKSessionLock(zkc0, lockPath, clientId0, lockStateExecutor);
-        final ZKSessionLock lock1 = new ZKSessionLock(zkc, lockPath, clientId1, lockStateExecutor);
-
-        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        // verification after lock0 lock
-        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))));
-
-        final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
-
-        Thread lock1Thread = new Thread(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    lock1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-                    lock1DoneLatch.countDown();
-                } catch (LockingException e) {
-                    logger.error("Failed on locking lock1 : ", e);
-                }
-            }
-        }, "lock1-thread");
-        lock1Thread.start();
-
-        // ensure lock1 is waiting for lock0
-        children = awaitWaiters(2, zkc, lockPath);
-
-        if (isUnlock) {
-            lock0.unlock();
-        } else {
-            ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
-        }
-
-        lock1DoneLatch.await();
-        lock1Thread.join();
-
-        // verification after lock2 lock
-        if (isUnlock) {
-            assertEquals(State.CLOSED, lock0.getLockState());
-        } else {
-            assertEquals(State.EXPIRED, lock0.getLockState());
-        }
-        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))));
-
-        lock1.unlock();
-    }
-
-    /**
-     * Test session expired after claimed the lock: lock state should be changed to expired and notify
-     * the lock listener about expiry.
-     *
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testLockListenerOnExpired() throws Exception {
-        String lockPath = "/test-lock-listener-on-expired";
-        String clientId = "test-lock-listener-on-expired-" + System.currentTimeMillis();
-
-        createLockPath(zkc.get(), lockPath);
-
-        final CountDownLatch expiredLatch = new CountDownLatch(1);
-        LockListener listener = new LockListener() {
-            @Override
-            public void onExpired() {
-                expiredLatch.countDown();
-            }
-        };
-        final ZKSessionLock lock =
-                new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor).setLockListener(listener);
-        lock.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        // verification after lock
-        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))));
-
-        ZooKeeperClientUtils.expireSession(zkc, zkServers, sessionTimeoutMs);
-        expiredLatch.await();
-        assertEquals(State.EXPIRED, lock.getLockState());
-        children = getLockWaiters(zkc, lockPath);
-        assertEquals(0, children.size());
-
-        try {
-            lock.tryLock(0, TimeUnit.MILLISECONDS);
-            fail("Should fail on tryLock since lock state has changed.");
-        } catch (LockStateChangedException lsce) {
-            // expected
-        }
-
-        lock.unlock();
-    }
-
-    @Test(timeout = 60000)
-    public void testSessionExpiredBeforeLock0() throws Exception {
-        testSessionExpiredBeforeLock(0);
-    }
-
-    @Test(timeout = 60000)
-    public void testSessionExpiredBeforeLock1() throws Exception {
-        testSessionExpiredBeforeLock(Long.MAX_VALUE);
-    }
-
-    /**
-     * Test Session Expired Before Lock does locking. The lock should be closed since
-     * all zookeeper operations would be failed.
-     *
-     * @param timeout
-     *          timeout to wait for the lock
-     * @throws Exception
-     */
-    private void testSessionExpiredBeforeLock(long timeout) throws Exception {
-        String lockPath = "/test-session-expired-before-lock-" + timeout + "-" + System.currentTimeMillis();
-        String clientId = "test-session-expired-before-lock-" + System.currentTimeMillis();
-
-        createLockPath(zkc.get(), lockPath);
-        final AtomicInteger expireCounter = new AtomicInteger(0);
-        final CountDownLatch expiredLatch = new CountDownLatch(1);
-        LockListener listener = new LockListener() {
-            @Override
-            public void onExpired() {
-                expireCounter.incrementAndGet();
-            }
-        };
-        final ZKSessionLock lock = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor)
-                .setLockListener(listener);
-        // expire session
-        ZooKeeperClientUtils.expireSession(zkc, zkServers, sessionTimeoutMs);
-        // submit a runnable to lock state executor to ensure any state changes happened when session expired
-        lockStateExecutor.submit(lockPath, new SafeRunnable() {
-            @Override
-            public void safeRun() {
-                expiredLatch.countDown();
-            }
-        });
-        expiredLatch.await();
-        // no watcher was registered if never acquired lock successfully
-        assertEquals(State.INIT, lock.getLockState());
-        try {
-            lock.tryLock(timeout, TimeUnit.MILLISECONDS);
-            fail("Should fail locking using an expired lock");
-        } catch (LockingException le) {
-            assertTrue(le.getCause() instanceof KeeperException.SessionExpiredException);
-        }
-        assertEquals(State.CLOSED, lock.getLockState());
-        List<String> children = getLockWaiters(zkc, lockPath);
-        assertEquals(0, children.size());
-    }
-
-    @Test(timeout = 60000)
-    public void testSessionExpiredForLockWaiter() throws Exception {
-        String lockPath = "/test-session-expired-for-lock-waiter";
-        String clientId0 = "test-session-expired-for-lock-waiter-0";
-        String clientId1 = "test-session-expired-for-lock-waiter-1";
-
-        createLockPath(zkc.get(), lockPath);
-
-        final ZKSessionLock lock0 = new ZKSessionLock(zkc0, lockPath, clientId0, lockStateExecutor);
-        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        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))));
-
-        final ZKSessionLock lock1 = new ZKSessionLock(zkc, lockPath, clientId1, lockStateExecutor);
-        final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
-
-        Thread lock1Thread = new Thread(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    lock1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-                } catch (OwnershipAcquireFailedException oafe) {
-                    lock1DoneLatch.countDown();
-                } catch (LockingException e) {
-                    logger.error("Failed on locking lock1 : ", e);
-                }
-            }
-        }, "lock1-thread");
-        lock1Thread.start();
-
-        // check lock1 is waiting for lock0
-        children = awaitWaiters(2, zkc, lockPath);
-
-        assertEquals(2, children.size());
-        assertEquals(State.CLAIMED, lock0.getLockState());
-        assertEquals(lock0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
-        awaitState(State.WAITING, lock1);
-        assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
-
-        // expire lock1
-        ZooKeeperClientUtils.expireSession(zkc, zkServers, sessionTimeoutMs);
-
-        lock1DoneLatch.countDown();
-        lock1Thread.join();
-        assertEquals(State.CLAIMED, lock0.getLockState());
-        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))));
-    }
-
-    public void awaitState(State state, ZKSessionLock lock) throws InterruptedException {
-        while (lock.getLockState() != state) {
-            Thread.sleep(50);
-        }
-    }
-
-    public List<String> awaitWaiters(int waiters, ZooKeeperClient zkc, String lockPath) throws Exception {
-        List<String> children = getLockWaiters(zkc, lockPath);
-        while (children.size() < waiters) {
-            Thread.sleep(50);
-            children = getLockWaiters(zkc, lockPath);
-        }
-        return children;
-    }
-
-    @Test(timeout = 60000)
-    public void testLockUseSameClientIdButDifferentSessions0() throws Exception {
-        testLockUseSameClientIdButDifferentSessions(true);
-    }
-
-    @Test(timeout = 60000)
-    public void testLockUseSameClientIdButDifferentSessions1() throws Exception {
-        testLockUseSameClientIdButDifferentSessions(false);
-    }
-
-    private void testLockUseSameClientIdButDifferentSessions(boolean isUnlock) throws Exception {
-        String lockPath = "/test-lock-use-same-client-id-but-different-sessions-" + isUnlock + System.currentTimeMillis();
-        String clientId = "test-lock-use-same-client-id-but-different-sessions";
-
-        createLockPath(zkc.get(), lockPath);
-
-        final ZKSessionLock lock0 = new ZKSessionLock(zkc0, lockPath, clientId, lockStateExecutor);
-
-        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        // lock1_0 couldn't claim ownership since owner is in a different zk session.
-        final ZKSessionLock lock1_0 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
-        try {
-            lock1_0.tryLock(0, TimeUnit.MILLISECONDS);
-            fail("Should fail locking since the lock is held in a different zk session.");
-        } catch (OwnershipAcquireFailedException oafe) {
-            assertEquals(clientId, oafe.getCurrentOwner());
-        }
-        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))));
-
-        // lock1_1 would wait the ownership
-        final ZKSessionLock lock1_1 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
-        final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
-
-        Thread lock1Thread = new Thread(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    lock1_1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-                    lock1DoneLatch.countDown();
-                } catch (LockingException e) {
-                    logger.error("Failed on locking lock1 : ", e);
-                }
-            }
-        }, "lock1-thread");
-        lock1Thread.start();
-
-        // check lock1 is waiting for lock0
-        children = awaitWaiters(2, zkc, lockPath);
-
-        logger.info("Found {} lock waiters : {}", children.size(), children);
-
-        assertEquals(2, children.size());
-        assertEquals(State.CLAIMED, lock0.getLockState());
-        assertEquals(lock0.getLockId(), Await.result(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))));
-
-        if (isUnlock) {
-            lock0.unlock();
-        } else {
-            ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
-        }
-        lock1DoneLatch.await();
-        lock1Thread.join();
-
-        // verification
-        if (isUnlock) {
-            assertEquals(State.CLOSED, lock0.getLockState());
-        } else {
-            assertEquals(State.EXPIRED, lock0.getLockState());
-        }
-        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))));
-
-        lock1_1.unlock();
-    }
-
-    @Test(timeout = 60000)
-    public void testLockWithMultipleSiblingWaiters() throws Exception {
-        String lockPath = "/test-lock-with-multiple-sibling-waiters";
-        String clientId = "client-id";
-
-        createLockPath(zkc.get(), lockPath);
-
-        final ZKSessionLock lock0 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
-        final ZKSessionLock lock1 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
-        final ZKSessionLock lock2 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
-
-        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        lock1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        lock2.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-
-        List<String> children = awaitWaiters(3, zkc, lockPath);
-
-        assertEquals(3, children.size());
-        assertEquals(State.CLAIMED, lock0.getLockState());
-        assertEquals(State.CLAIMED, lock1.getLockState());
-        assertEquals(State.CLAIMED, lock2.getLockState());
-
-        lock0.unlock();
-        lock1.unlock();
-        lock2.unlock();
-    }
-
-    /**
-     * Immediate lock and unlock first lock
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testLockWhenSiblingUseDifferentLockId0() throws Exception {
-        testLockWhenSiblingUseDifferentLockId(0, true);
-    }
-
-    /**
-     * Immediate lock and expire first lock
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testLockWhenSiblingUseDifferentLockId1() throws Exception {
-        testLockWhenSiblingUseDifferentLockId(0, false);
-    }
-
-    /**
-     * Wait Lock and unlock lock0_0 and lock1
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testLockWhenSiblingUseDifferentLockId2() throws Exception {
-        testLockWhenSiblingUseDifferentLockId(Long.MAX_VALUE, true);
-    }
-
-    /**
-     * Wait Lock and expire first & third lock
-     * @throws Exception
-     */
-    @Test(timeout = 60000)
-    public void testLockWhenSiblingUseDifferentLockId3() throws Exception {
-        testLockWhenSiblingUseDifferentLockId(Long.MAX_VALUE, false);
-    }
-
-    private void testLockWhenSiblingUseDifferentLockId(long timeout, final boolean isUnlock) throws Exception {
-        String lockPath = "/test-lock-when-sibling-use-different-lock-id-" + timeout
-                + "-" + isUnlock + "-" + System.currentTimeMillis();
-        String clientId0 = "client-id-0";
-        String clientId1 = "client-id-1";
-
-        createLockPath(zkc.get(), lockPath);
-
-        final ZKSessionLock lock0_0 = new ZKSessionLock(zkc0, lockPath, clientId0, lockStateExecutor);
-        final ZKSessionLock lock0_1 = new ZKSessionLock(zkc0, lockPath, clientId0, lockStateExecutor);
-        final ZKSessionLock lock1   = new ZKSessionLock(zkc, lockPath, clientId1, lockStateExecutor);
-
-        lock0_0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-
-        // lock1 wait for the lock ownership.
-        final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
-        Thread lock1Thread = new Thread(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    lock1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-                    lock1DoneLatch.countDown();
-                } catch (LockingException e) {
-                    logger.error("Failed on locking lock1 : ", e);
-                }
-            }
-        }, "lock1-thread");
-        lock1Thread.start();
-
-        // check lock1 is waiting for lock0_0
-        List<String> children = awaitWaiters(2, 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))));
-        awaitState(State.WAITING, lock1);
-        assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
-
-        final CountDownLatch lock0DoneLatch = new CountDownLatch(1);
-        final AtomicReference<String> ownerFromLock0 = new AtomicReference<String>(null);
-        Thread lock0Thread = null;
-        if (timeout == 0) {
-            try {
-                lock0_1.tryLock(0, TimeUnit.MILLISECONDS);
-                fail("Should fail on locking if sibling is using differnt lock id.");
-            } catch (OwnershipAcquireFailedException oafe) {
-                assertEquals(clientId0, oafe.getCurrentOwner());
-            }
-            assertEquals(State.CLOSED, lock0_1.getLockState());
-            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(State.WAITING, lock1.getLockState());
-            assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
-        } else {
-            lock0Thread = new Thread(new Runnable() {
-                @Override
-                public void run() {
-                    try {
-                        lock0_1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-                        if (isUnlock) {
-                            lock0DoneLatch.countDown();
-                        }
-                    } catch (OwnershipAcquireFailedException oafe) {
-                        if (!isUnlock) {
-                            ownerFromLock0.set(oafe.getCurrentOwner());
-                            lock0DoneLatch.countDown();
-                        }
-                    } catch (LockingException le) {
-                        logger.error("Failed on locking lock0_1 : ", le);
-                    }
-                }
-            }, "lock0-thread");
-            lock0Thread.start();
-
-            // check lock1 is waiting for lock0_0
-            children = awaitWaiters(3, zkc, lockPath);
-
-            assertEquals(3, children.size());
-            assertEquals(State.CLAIMED, lock0_0.getLockState());
-            assertEquals(lock0_0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
-            awaitState(State.WAITING, lock1);
-            assertEquals(lock1.getLockId(), Await.result(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))));
-        }
-
-        if (isUnlock) {
-            lock0_0.unlock();
-        } else {
-            ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
-        }
-
-        lock1DoneLatch.await();
-        lock1Thread.join();
-
-        // check the state of lock0_0
-        if (isUnlock) {
-            assertEquals(State.CLOSED, lock0_0.getLockState());
-        } else {
-            assertEquals(State.EXPIRED, lock0_0.getLockState());
-        }
-
-        if (timeout == 0) {
-            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))));
-        } else {
-            assertNotNull(lock0Thread);
-            if (!isUnlock) {
-                // both lock0_0 and lock0_1 would be expired
-                lock0DoneLatch.await();
-                lock0Thread.join();
-
-                assertEquals(clientId0, ownerFromLock0.get());
-                assertEquals(State.CLOSED, lock0_1.getLockState());
-
-                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))));
-            } 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(State.WAITING, lock0_1.getLockState());
-                assertEquals(lock0_1.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(1))));
-            }
-        }
-
-        lock1.unlock();
-
-        if (timeout != 0 && isUnlock) {
-            lock0DoneLatch.await();
-            lock0Thread.join();
-
-            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))));
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testLockWhenSiblingUseSameLockId0() throws Exception {
-        testLockWhenSiblingUseSameLockId(0, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testLockWhenSiblingUseSameLockId1() throws Exception {
-        testLockWhenSiblingUseSameLockId(0, false);
-    }
-
-    @Test(timeout = 60000)
-    public void testLockWhenSiblingUseSameLockId2() throws Exception {
-        testLockWhenSiblingUseSameLockId(Long.MAX_VALUE, true);
-    }
-
-    @Test(timeout = 60000)
-    public void testLockWhenSiblingUseSameLockId3() throws Exception {
-        testLockWhenSiblingUseSameLockId(Long.MAX_VALUE, false);
-    }
-
-    private void testLockWhenSiblingUseSameLockId(long timeout, final boolean isUnlock) throws Exception {
-        String lockPath = "/test-lock-when-sibling-use-same-lock-id-" + timeout
-                + "-" + isUnlock + "-" + System.currentTimeMillis();
-        String clientId = "client-id";
-
-        createLockPath(zkc.get(), lockPath);
-
-        final ZKSessionLock lock0 = new ZKSessionLock(zkc0, lockPath, clientId, lockStateExecutor);
-        final ZKSessionLock lock1 = new ZKSessionLock(zkc0, lockPath, clientId, lockStateExecutor);
-
-        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
-        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))));
-
-        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(State.CLAIMED, lock1.getLockState());
-        assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(1))));
-
-        if (isUnlock) {
-            lock0.unlock();
-            assertEquals(State.CLOSED, lock0.getLockState());
-            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))));
-            lock1.unlock();
-        } else {
-            ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
-            final CountDownLatch latch = new CountDownLatch(1);
-            lockStateExecutor.submit(lockPath, new SafeRunnable() {
-                @Override
-                public void safeRun() {
-                    latch.countDown();
-                }
-            });
-            latch.await();
-            children = getLockWaiters(zkc, lockPath);
-            assertEquals(0, children.size());
-            assertEquals(State.EXPIRED, lock0.getLockState());
-            assertEquals(State.EXPIRED, lock1.getLockState());
-        }
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/logsegment/TestPerStreamLogSegmentCache.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/logsegment/TestPerStreamLogSegmentCache.java
deleted file mode 100644
index 456ed68..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/logsegment/TestPerStreamLogSegmentCache.java
+++ /dev/null
@@ -1,186 +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 com.twitter.distributedlog.logsegment;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.junit.Test;
-
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-
-import static org.junit.Assert.*;
-
-/**
- * Test Case for Per Stream Log Segment Cache.
- */
-public class TestPerStreamLogSegmentCache {
-
-    @Test(timeout = 60000)
-    public void testBasicOperations() {
-        LogSegmentMetadata metadata =
-                DLMTestUtil.completedLogSegment("/segment1", 1L, 1L, 100L, 100, 1L, 99L, 0L);
-        String name = DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(1L);
-
-        PerStreamLogSegmentCache cache = new PerStreamLogSegmentCache("test-basic-operations");
-        assertNull("No log segment " + name + " should be cached", cache.get(name));
-        cache.add(name, metadata);
-        LogSegmentMetadata metadataRetrieved = cache.get(name);
-        assertNotNull("log segment " + name + " should be cached", metadataRetrieved);
-        assertEquals("Wrong log segment metadata returned for " + name,
-                metadata, metadataRetrieved);
-        LogSegmentMetadata metadataRemoved = cache.remove(name);
-        assertNull("log segment " + name + " should be removed from cache", cache.get(name));
-        assertEquals("Wrong log segment metadata removed for " + name,
-                metadata, metadataRemoved);
-        assertNull("No log segment " + name + " to be removed", cache.remove(name));
-    }
-
-    @Test(timeout = 60000)
-    public void testDiff() {
-        PerStreamLogSegmentCache cache = new PerStreamLogSegmentCache("test-diff");
-        // add 5 completed log segments
-        for (int i = 1; i <= 5; i++) {
-            LogSegmentMetadata metadata =
-                    DLMTestUtil.completedLogSegment("/segment" + i, i, i, i * 100L, 100, i, 99L, 0L);
-            String name = DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i);
-            cache.add(name, metadata);
-        }
-        // add one inprogress log segment
-        LogSegmentMetadata inprogress =
-                DLMTestUtil.inprogressLogSegment("/inprogress-6", 6, 600L, 6);
-        String name = DLMTestUtil.inprogressZNodeName(6);
-        cache.add(name, inprogress);
-
-        // deleted first 2 completed log segments and completed the last one
-        Set<String> segmentRemoved = Sets.newHashSet();
-        for (int i = 1; i <= 2; i++) {
-            segmentRemoved.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i));
-        }
-        segmentRemoved.add((DLMTestUtil.inprogressZNodeName(6)));
-        Set<String> segmentReceived = Sets.newHashSet();
-        Set<String> segmentAdded = Sets.newHashSet();
-        for (int i = 3; i <= 6; i++) {
-            segmentReceived.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i));
-            if (i == 6) {
-                segmentAdded.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i));
-            }
-        }
-
-        Pair<Set<String>, Set<String>> segmentChanges = cache.diff(segmentReceived);
-        assertTrue("Should remove " + segmentRemoved + ", but removed " + segmentChanges.getRight(),
-                Sets.difference(segmentRemoved, segmentChanges.getRight()).isEmpty());
-        assertTrue("Should add " + segmentAdded + ", but added " + segmentChanges.getLeft(),
-                Sets.difference(segmentAdded, segmentChanges.getLeft()).isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testUpdate() {
-        PerStreamLogSegmentCache cache = new PerStreamLogSegmentCache("test-update");
-        // add 5 completed log segments
-        for (int i = 1; i <= 5; i++) {
-            LogSegmentMetadata metadata =
-                    DLMTestUtil.completedLogSegment("/segment" + i, i, i, i * 100L, 100, i, 99L, 0L);
-            String name = DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i);
-            cache.add(name, metadata);
-        }
-        // add one inprogress log segment
-        LogSegmentMetadata inprogress =
-                DLMTestUtil.inprogressLogSegment("/inprogress-6", 6, 600L, 6);
-        String name = DLMTestUtil.inprogressZNodeName(6);
-        cache.add(name, inprogress);
-
-        // deleted first 2 completed log segments and completed the last one
-        Set<String> segmentRemoved = Sets.newHashSet();
-        for (int i = 1; i <= 2; i++) {
-            segmentRemoved.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i));
-        }
-        segmentRemoved.add((DLMTestUtil.inprogressZNodeName(6)));
-        Set<String> segmentReceived = Sets.newHashSet();
-        Map<String, LogSegmentMetadata> segmentAdded = Maps.newHashMap();
-        for (int i = 3; i <= 6; i++) {
-            segmentReceived.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i));
-            if (i == 6) {
-                segmentAdded.put(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i),
-                        DLMTestUtil.completedLogSegment("/segment" + i, i, i, i * 100L, 100, i, 99L, 0L));
-            }
-        }
-
-        // update the cache
-        cache.update(segmentRemoved, segmentAdded);
-        for (String segment : segmentRemoved) {
-            assertNull("Segment " + segment + " should be removed.", cache.get(segment));
-        }
-        for (String segment : segmentReceived) {
-            assertNotNull("Segment " + segment + " should not be removed", cache.get(segment));
-        }
-        for (Map.Entry<String, LogSegmentMetadata> entry : segmentAdded.entrySet()) {
-            assertEquals("Segment " + entry.getKey() + " should be added.",
-                    entry.getValue(), entry.getValue());
-        }
-    }
-
-    @Test(timeout = 60000, expected = UnexpectedException.class)
-    public void testGapDetection() throws Exception {
-        PerStreamLogSegmentCache cache = new PerStreamLogSegmentCache("test-gap-detection");
-        cache.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(1L),
-                DLMTestUtil.completedLogSegment("/segment-1", 1L, 1L, 100L, 100, 1L, 99L, 0L));
-        cache.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(3L),
-                DLMTestUtil.completedLogSegment("/segment-3", 3L, 3L, 300L, 100, 3L, 99L, 0L));
-        cache.getLogSegments(LogSegmentMetadata.COMPARATOR);
-    }
-
-    @Test(timeout = 60000)
-    public void testGapDetectionOnLogSegmentsWithoutLogSegmentSequenceNumber() throws Exception {
-        PerStreamLogSegmentCache cache = new PerStreamLogSegmentCache("test-gap-detection");
-        LogSegmentMetadata segment1 =
-                DLMTestUtil.completedLogSegment("/segment-1", 1L, 1L, 100L, 100, 1L, 99L, 0L)
-                        .mutator().setVersion(LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V1_ORIGINAL).build();
-        cache.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(1L), segment1);
-        LogSegmentMetadata segment3 =
-                DLMTestUtil.completedLogSegment("/segment-3", 3L, 3L, 300L, 100, 3L, 99L, 0L)
-                        .mutator().setVersion(LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO).build();
-        cache.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(3L), segment3);
-        List<LogSegmentMetadata> expectedList = Lists.asList(segment1, new LogSegmentMetadata[] { segment3 });
-        List<LogSegmentMetadata> resultList = cache.getLogSegments(LogSegmentMetadata.COMPARATOR);
-        assertEquals(expectedList, resultList);
-    }
-
-    @Test(timeout = 60000, expected = UnexpectedException.class)
-    public void testSameLogSegment() throws Exception {
-        PerStreamLogSegmentCache cache = new PerStreamLogSegmentCache("test-same-log-segment");
-        List<LogSegmentMetadata> expectedList = Lists.newArrayListWithExpectedSize(2);
-        LogSegmentMetadata inprogress =
-                DLMTestUtil.inprogressLogSegment("/inprogress-1", 1L, 1L, 1L);
-        expectedList.add(inprogress);
-        cache.add(DLMTestUtil.inprogressZNodeName(1L), inprogress);
-        LogSegmentMetadata completed =
-                DLMTestUtil.completedLogSegment("/segment-1", 1L, 1L, 100L, 100, 1L, 99L, 0L);
-        expectedList.add(completed);
-        cache.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(1L), completed);
-
-        cache.getLogSegments(LogSegmentMetadata.COMPARATOR);
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/logsegment/TestRollingPolicy.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/logsegment/TestRollingPolicy.java
deleted file mode 100644
index f3518f4..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/logsegment/TestRollingPolicy.java
+++ /dev/null
@@ -1,63 +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 com.twitter.distributedlog.logsegment;
-
-import com.twitter.distributedlog.util.Sizable;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-/**
- * Test Case for {@link RollingPolicy}s.
- */
-public class TestRollingPolicy {
-
-    static class TestSizable implements Sizable {
-
-        long size;
-
-        TestSizable(long size) {
-            this.size = size;
-        }
-
-        @Override
-        public long size() {
-            return size;
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testTimeBasedRollingPolicy() {
-        TimeBasedRollingPolicy policy1 = new TimeBasedRollingPolicy(Long.MAX_VALUE);
-        TestSizable maxSize = new TestSizable(Long.MAX_VALUE);
-        assertFalse(policy1.shouldRollover(maxSize, System.currentTimeMillis()));
-
-        long currentMs = System.currentTimeMillis();
-        TimeBasedRollingPolicy policy2 = new TimeBasedRollingPolicy(1000);
-        assertTrue(policy2.shouldRollover(maxSize, currentMs - 2 * 1000));
-    }
-
-    @Test(timeout = 60000)
-    public void testSizeBasedRollingPolicy() {
-        SizeBasedRollingPolicy policy = new SizeBasedRollingPolicy(1000);
-        TestSizable sizable1 = new TestSizable(10);
-        assertFalse(policy.shouldRollover(sizable1, 0L));
-        TestSizable sizable2 = new TestSizable(10000);
-        assertTrue(policy.shouldRollover(sizable2, 0L));
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestDLMetadata.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestDLMetadata.java
deleted file mode 100644
index e3cc239..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestDLMetadata.java
+++ /dev/null
@@ -1,195 +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 com.twitter.distributedlog.metadata;
-
-import com.twitter.distributedlog.LocalDLMEmulator;
-import com.twitter.distributedlog.ZooKeeperClusterTestCase;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.Stat;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.net.URI;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-public class TestDLMetadata extends ZooKeeperClusterTestCase {
-
-    private static final BKDLConfig bkdlConfig =
-            new BKDLConfig("127.0.0.1:7000", "127.0.0.1:7000",
-                           "127.0.0.1:7000", "127.0.0.1:7000", "ledgers");
-    private static final BKDLConfig bkdlConfig2 =
-            new BKDLConfig("127.0.0.1:7001", "127.0.0.1:7002",
-                           "127.0.0.1:7003", "127.0.0.1:7004", "ledgers2");
-
-    private ZooKeeper zkc;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = LocalDLMEmulator.connectZooKeeper("127.0.0.1", zkPort);
-    }
-
-    @After
-    public void teardown() throws Exception {
-        zkc.close();
-    }
-
-    private URI createURI(String path) {
-        return URI.create("distributedlog://127.0.0.1:" + zkPort + path);
-    }
-
-    @Test(timeout = 60000)
-    public void testBadMetadata() throws Exception {
-        URI uri = createURI("/");
-        try {
-            DLMetadata.deserialize(uri, new byte[0]);
-            fail("Should fail to deserialize invalid metadata");
-        } catch (IOException ie) {
-            // expected
-        }
-        try {
-            DLMetadata.deserialize(uri, new DLMetadata("unknown", bkdlConfig).serialize());
-            fail("Should fail to deserialize due to unknown dl type.");
-        } catch (IOException ie) {
-            // expected
-        }
-        try {
-            DLMetadata.deserialize(uri, new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig, 9999).serialize());
-            fail("Should fail to deserialize due to invalid version.");
-        } catch (IOException ie) {
-            // expected
-        }
-        byte[] data = new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig).serialize();
-        // truncate data
-        byte[] badData = new byte[data.length - 3];
-        System.arraycopy(data, 0, badData, 0, badData.length);
-        try {
-            DLMetadata.deserialize(uri, badData);
-            fail("Should fail to deserialize truncated data.");
-        } catch (IOException ie) {
-            // expected
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testGoodMetadata() throws Exception {
-        URI uri = createURI("/");
-        byte[] data = new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig).serialize();
-        DLMetadata deserailized = DLMetadata.deserialize(uri, data);
-        assertEquals(bkdlConfig, deserailized.getDLConfig());
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteMetadata() throws Exception {
-        DLMetadata metadata = new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig);
-        try {
-            metadata.create(createURI("//metadata"));
-            fail("Should fail due to invalid uri.");
-        } catch (IllegalArgumentException e) {
-            // expected
-        }
-        URI uri = createURI("/metadata");
-        metadata.create(uri);
-        // create on existed path
-        try {
-            metadata.create(uri);
-            fail("Should fail when create on existed path");
-        } catch (IOException e) {
-            // expected
-        }
-        // update on unexisted path
-        try {
-            metadata.update(createURI("/unexisted"));
-            fail("Should fail when update on unexisted path");
-        } catch (IOException e) {
-            // expected
-        }
-        byte[] data = zkc.getData("/metadata", false, new Stat());
-        assertEquals(bkdlConfig, DLMetadata.deserialize(uri, data).getDLConfig());
-        // update on existed path
-        DLMetadata newMetadata = new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig2);
-        newMetadata.update(createURI("/metadata"));
-        byte[] newData = zkc.getData("/metadata", false, new Stat());
-        assertEquals(bkdlConfig2, DLMetadata.deserialize(uri, newData).getDLConfig());
-    }
-
-    // Missing dlZkServersForWriter, dlZkServersForReader default to configured server.
-    @Test(timeout = 60000)
-    public void testMetadataWithoutDLZKServers() throws Exception {
-        testMetadataWithOrWithoutZkServers(
-                "/metadata-without-dlzk-servers",
-                null, null, "127.0.0.1:7003", "127.0.0.1:7004",
-                "127.0.0.1:" + zkPort, "127.0.0.1:" + zkPort, "127.0.0.1:7003", "127.0.0.1:7004");
-    }
-
-    @Test(timeout = 60000)
-    public void testMetadataWithoutDLZKServersForRead() throws Exception {
-        testMetadataWithOrWithoutZkServers(
-                "/metadata-without-dlzk-servers-for-read",
-                "127.0.0.1:7001", null, "127.0.0.1:7003", "127.0.0.1:7004",
-                "127.0.0.1:7001", "127.0.0.1:7001", "127.0.0.1:7003", "127.0.0.1:7004");
-    }
-
-    @Test(timeout = 60000)
-    public void testMetadataWithoutBKZKServersForRead() throws Exception {
-        testMetadataWithOrWithoutZkServers(
-                "/metadata-without-bkzk-servers-for-read",
-                "127.0.0.1:7001", null, "127.0.0.1:7003", null,
-                "127.0.0.1:7001", "127.0.0.1:7001", "127.0.0.1:7003", "127.0.0.1:7003");
-    }
-
-    private void testMetadataWithOrWithoutZkServers(
-            String metadataPath,
-            String dlZkServersForWriter, String dlZkServersForReader,
-            String bkZkServersForWriter, String bkZkServersForReader,
-            String expectedDlZkServersForWriter, String expectedDlZkServersForReader,
-            String expectedBkZkServersForWriter, String expectedBkZkServersForReader
-    ) throws Exception {
-        BKDLConfig bkdlConfig = new BKDLConfig(dlZkServersForWriter, dlZkServersForReader,
-                                               bkZkServersForWriter, bkZkServersForReader, "ledgers");
-        BKDLConfig expectedBKDLConfig =
-                new BKDLConfig(expectedDlZkServersForWriter, expectedDlZkServersForReader,
-                               expectedBkZkServersForWriter, expectedBkZkServersForReader, "ledgers");
-        URI uri = createURI(metadataPath);
-        DLMetadata metadata = new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig);
-        metadata.create(uri);
-        // read serialized metadata
-        byte[] data = zkc.getData(metadataPath, false, new Stat());
-        assertEquals(expectedBKDLConfig, DLMetadata.deserialize(uri, data).getDLConfig());
-    }
-
-    @Test(timeout = 60000)
-    public void testMetadataMissingRequiredFields() throws Exception {
-        BKDLConfig bkdlConfig = new BKDLConfig(null, null, null, null, "ledgers");
-        URI uri = createURI("/metadata-missing-fields");
-        DLMetadata metadata = new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig);
-        metadata.create(uri);
-        // read serialized metadata
-        byte[] data = zkc.getData("/metadata-missing-fields", false, new Stat());
-        try {
-            DLMetadata.deserialize(uri, data);
-            fail("Should fail on deserializing metadata missing fields");
-        } catch (IOException ioe) {
-            // expected
-        }
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestLogMetadata.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestLogMetadata.java
deleted file mode 100644
index 315b983..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestLogMetadata.java
+++ /dev/null
@@ -1,59 +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 com.twitter.distributedlog.metadata;
-
-import com.twitter.distributedlog.DLMTestUtil;
-import org.junit.Test;
-
-import java.net.URI;
-
-import static com.twitter.distributedlog.metadata.LogMetadata.*;
-import static org.junit.Assert.*;
-
-public class TestLogMetadata {
-
-    @Test(timeout = 60000)
-    public void testGetPaths() throws Exception {
-        String rootPath = "/test-get-paths";
-        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
-        String logName = "test-log";
-        String logIdentifier = "<default>";
-        String logRootPath = uri.getPath() + "/" + logName + "/" + logIdentifier;
-        String logSegmentName = "test-segment";
-
-        LogMetadata logMetadata = new LogMetadata(uri, logName, logIdentifier);
-        assertEquals("wrong log name", logName, logMetadata.getLogName());
-        assertEquals("wrong root path", logRootPath, logMetadata.getLogRootPath());
-        assertEquals("wrong log segments path",
-                logRootPath + LOGSEGMENTS_PATH,
-                logMetadata.getLogSegmentsPath());
-        assertEquals("wrong log segment path",
-                logRootPath + LOGSEGMENTS_PATH + "/" + logSegmentName,
-                logMetadata.getLogSegmentPath(logSegmentName));
-        assertEquals("wrong lock path",
-                logRootPath + LOCK_PATH, logMetadata.getLockPath());
-        assertEquals("wrong max tx id path",
-                logRootPath + MAX_TXID_PATH, logMetadata.getMaxTxIdPath());
-        assertEquals("wrong allocation path",
-                logRootPath + ALLOCATION_PATH, logMetadata.getAllocationPath());
-        assertEquals("wrong qualified name",
-                logName + ":" + logIdentifier, logMetadata.getFullyQualifiedName());
-    }
-
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java
deleted file mode 100644
index 51bdaa5..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java
+++ /dev/null
@@ -1,284 +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 com.twitter.distributedlog.metadata;
-
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClusterTestCase;
-import com.twitter.distributedlog.impl.ZKLogSegmentMetadataStore;
-import com.twitter.distributedlog.logsegment.LogSegmentMetadataStore;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.ZooDefs;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import java.net.URI;
-import java.util.HashMap;
-import java.util.Map;
-
-import static org.junit.Assert.*;
-
-public class TestLogSegmentMetadataStoreUpdater extends ZooKeeperClusterTestCase {
-
-    static final Logger LOG = LoggerFactory.getLogger(TestLogSegmentMetadataStoreUpdater.class);
-
-    private ZooKeeperClient zkc;
-    private OrderedScheduler scheduler;
-    private LogSegmentMetadataStore metadataStore;
-    private DistributedLogConfiguration conf = new DistributedLogConfiguration()
-            .setDLLedgerMetadataLayoutVersion(LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
-
-    @Before
-    public void setup() throws Exception {
-        scheduler = OrderedScheduler.newBuilder()
-                .name("test-logsegment-metadata-store-updater")
-                .corePoolSize(1)
-                .build();
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .uri(createURI("/"))
-                .sessionTimeoutMs(10000)
-                .build();
-        metadataStore = new ZKLogSegmentMetadataStore(conf, zkc, scheduler);
-    }
-
-    @After
-    public void tearDown() throws Exception {
-        metadataStore.close();
-        scheduler.shutdown();
-        zkc.close();
-    }
-
-    private URI createURI(String path) {
-        return URI.create("distributedlog://127.0.0.1:" + zkPort + path);
-    }
-
-    Map<Long, LogSegmentMetadata> readLogSegments(String ledgerPath) throws Exception {
-        return DLMTestUtil.readLogSegments(zkc, ledgerPath);
-    }
-
-    @Test(timeout = 60000)
-    public void testChangeSequenceNumber() throws Exception {
-        String ledgerPath = "/testChangeSequenceNumber";
-        zkc.get().create(ledgerPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        Map<Long, LogSegmentMetadata> completedLogSegments = new HashMap<Long, LogSegmentMetadata>();
-        // Create 5 completed log segments
-        for (int i = 1; i <= 5; i++) {
-            LogSegmentMetadata segment = DLMTestUtil.completedLogSegment(ledgerPath, i, (i - 1) * 100, i * 100 - 1, 100, i, 100, 0);
-            completedLogSegments.put(((long)i), segment);
-            LOG.info("Create completed segment {} : {}", segment.getZkPath(), segment);
-            segment.write(zkc);
-        }
-        // Create a smaller inprogress log segment
-        long inprogressSeqNo = 3;
-        LogSegmentMetadata segment = DLMTestUtil.inprogressLogSegment(ledgerPath, inprogressSeqNo, 5 * 100, inprogressSeqNo);
-        LOG.info("Create inprogress segment {} : {}", segment.getZkPath(), segment);
-        segment.write(zkc);
-
-        Map<Long, LogSegmentMetadata> segmentList = readLogSegments(ledgerPath);
-        assertEquals(5, segmentList.size());
-
-        // Dryrun
-        MetadataUpdater dryrunUpdater = new DryrunLogSegmentMetadataStoreUpdater(conf, metadataStore);
-        FutureUtils.result(dryrunUpdater.changeSequenceNumber(segment, 6L));
-
-        segmentList = readLogSegments(ledgerPath);
-        assertEquals(5, segmentList.size());
-
-        // Fix the inprogress log segments
-
-        MetadataUpdater updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
-        FutureUtils.result(updater.changeSequenceNumber(segment, 6L));
-
-        segmentList = readLogSegments(ledgerPath);
-        assertEquals(6, segmentList.size());
-
-        // check first 5 log segments
-        for (int i = 1; i <= 5; i++) {
-            LogSegmentMetadata s = segmentList.get((long)i);
-            assertNotNull(s);
-            assertEquals(completedLogSegments.get((long)i), s);
-        }
-
-        // get log segment 6
-        LogSegmentMetadata segmentChanged = segmentList.get(6L);
-        assertNotNull(segmentChanged);
-        assertEquals(6L, segmentChanged.getLogSegmentSequenceNumber());
-        assertTrue(segmentChanged.isInProgress());
-        assertEquals(5 * 100, segmentChanged.getFirstTxId());
-        assertEquals(3L, segmentChanged.getLogSegmentId());
-    }
-
-    @Test(timeout = 60000)
-    public void testUpdateLastDLSN() throws Exception {
-        String ledgerPath = "/testUpdateLastDLSN";
-        zkc.get().create(ledgerPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        // Create 1 completed log segment
-        LogSegmentMetadata completedLogSegment = DLMTestUtil.completedLogSegment(ledgerPath, 1L, 0L, 99L, 100, 1L, 99L, 0L);
-        completedLogSegment.write(zkc);
-        // Create 1 inprogress log segment
-        LogSegmentMetadata inprogressLogSegment = DLMTestUtil.inprogressLogSegment(ledgerPath, 2L, 100L, 2L);
-        inprogressLogSegment.write(zkc);
-
-        DLSN badLastDLSN = new DLSN(99L, 0L, 0L);
-        DLSN goodLastDLSN1 = new DLSN(1L, 100L, 0L);
-        DLSN goodLastDLSN2 = new DLSN(2L, 200L, 0L);
-
-        LogRecordWithDLSN badRecord = DLMTestUtil.getLogRecordWithDLSNInstance(badLastDLSN, 100L);
-        LogRecordWithDLSN goodRecord1 = DLMTestUtil.getLogRecordWithDLSNInstance(goodLastDLSN1, 100L);
-        LogRecordWithDLSN goodRecord2 = DLMTestUtil.getLogRecordWithDLSNInstance(goodLastDLSN2, 200L);
-
-        // Dryrun
-        MetadataUpdater dryrunUpdater = new DryrunLogSegmentMetadataStoreUpdater(conf, metadataStore);
-        try {
-            FutureUtils.result(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));
-            fail("Should fail on updating dlsn for an inprogress log segment");
-        } catch (IllegalStateException ise) {
-            // expected
-        }
-        LogSegmentMetadata updatedCompletedLogSegment =
-                FutureUtils.result(dryrunUpdater.updateLastRecord(completedLogSegment, goodRecord1));
-        assertEquals(goodLastDLSN1, updatedCompletedLogSegment.getLastDLSN());
-        assertEquals(goodRecord1.getTransactionId(), updatedCompletedLogSegment.getLastTxId());
-        assertTrue(updatedCompletedLogSegment.isRecordLastPositioninThisSegment(goodRecord1));
-
-        Map<Long, LogSegmentMetadata> segmentList = readLogSegments(ledgerPath);
-        assertEquals(2, segmentList.size());
-
-        LogSegmentMetadata readCompletedLogSegment = segmentList.get(1L);
-        assertNotNull(readCompletedLogSegment);
-        assertEquals(completedLogSegment, readCompletedLogSegment);
-
-        LogSegmentMetadata readInprogressLogSegment = segmentList.get(2L);
-        assertNotNull(readInprogressLogSegment);
-        assertEquals(inprogressLogSegment, readInprogressLogSegment);
-
-        // Fix the last dlsn
-        MetadataUpdater updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
-        try {
-            FutureUtils.result(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));
-            fail("Should fail on updating dlsn for an inprogress log segment");
-        } catch (IllegalStateException ise) {
-            // expected
-        }
-        updatedCompletedLogSegment = FutureUtils.result(updater.updateLastRecord(completedLogSegment, goodRecord1));
-        assertEquals(goodLastDLSN1, updatedCompletedLogSegment.getLastDLSN());
-        assertEquals(goodRecord1.getTransactionId(), updatedCompletedLogSegment.getLastTxId());
-        assertTrue(updatedCompletedLogSegment.isRecordLastPositioninThisSegment(goodRecord1));
-
-        segmentList = readLogSegments(ledgerPath);
-        assertEquals(2, segmentList.size());
-
-        readCompletedLogSegment = segmentList.get(1L);
-        assertNotNull(readCompletedLogSegment);
-        assertEquals(goodLastDLSN1, readCompletedLogSegment.getLastDLSN());
-        assertEquals(goodRecord1.getTransactionId(), readCompletedLogSegment.getLastTxId());
-        assertTrue(readCompletedLogSegment.isRecordLastPositioninThisSegment(goodRecord1));
-        assertEquals(updatedCompletedLogSegment, readCompletedLogSegment);
-        assertEquals(completedLogSegment.getCompletionTime(), readCompletedLogSegment.getCompletionTime());
-        assertEquals(completedLogSegment.getFirstTxId(), readCompletedLogSegment.getFirstTxId());
-        assertEquals(completedLogSegment.getLogSegmentId(), readCompletedLogSegment.getLogSegmentId());
-        assertEquals(completedLogSegment.getLogSegmentSequenceNumber(), readCompletedLogSegment.getLogSegmentSequenceNumber());
-        assertEquals(completedLogSegment.getRegionId(), readCompletedLogSegment.getRegionId());
-        assertEquals(completedLogSegment.getZkPath(), readCompletedLogSegment.getZkPath());
-        assertEquals(completedLogSegment.getZNodeName(), readCompletedLogSegment.getZNodeName());
-
-        readInprogressLogSegment = segmentList.get(2L);
-        assertNotNull(readInprogressLogSegment);
-        assertEquals(inprogressLogSegment, readInprogressLogSegment);
-    }
-
-    @Test(timeout = 60000)
-    public void testChangeTruncationStatus() throws Exception {
-        String ledgerPath = "/ledgers2";
-        zkc.get().create(ledgerPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-        Map<Long, LogSegmentMetadata> completedLogSegments = new HashMap<Long, LogSegmentMetadata>();
-        // Create 5 completed log segments
-        for (int i = 1; i <= 5; i++) {
-            LogSegmentMetadata segment = DLMTestUtil.completedLogSegment(ledgerPath, i, (i - 1) * 100, i * 100 - 1, 100, i, 100, 0);
-            completedLogSegments.put(((long)i), segment);
-            LOG.info("Create completed segment {} : {}", segment.getZkPath(), segment);
-            segment.write(zkc);
-        }
-
-        Map<Long, LogSegmentMetadata> segmentList = readLogSegments(ledgerPath);
-        assertEquals(5, segmentList.size());
-
-        long segmentToModify = 1L;
-
-        // Dryrun
-        MetadataUpdater dryrunUpdater = new DryrunLogSegmentMetadataStoreUpdater(conf, metadataStore);
-        FutureUtils.result(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)));
-
-        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)));
-
-        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),
-                segmentList.get(segmentToModify).getFirstDLSN()));
-
-        segmentList = readLogSegments(ledgerPath);
-        assertEquals(false, segmentList.get(segmentToModify).isTruncated());
-        assertEquals(true, segmentList.get(segmentToModify).isPartiallyTruncated());
-
-        updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
-        FutureUtils.result(updater.setLogSegmentActive(segmentList.get(segmentToModify)));
-
-        segmentList = readLogSegments(ledgerPath);
-        assertEquals(false, segmentList.get(segmentToModify).isTruncated());
-        assertEquals(false, segmentList.get(segmentToModify).isPartiallyTruncated());
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/namespace/TestDistributedLogNamespaceBuilder.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/namespace/TestDistributedLogNamespaceBuilder.java
deleted file mode 100644
index 6e48deb..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/namespace/TestDistributedLogNamespaceBuilder.java
+++ /dev/null
@@ -1,124 +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 com.twitter.distributedlog.namespace;
-
-import com.twitter.distributedlog.BKDistributedLogNamespace;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import org.junit.Test;
-
-import java.net.URI;
-
-import static com.twitter.distributedlog.LocalDLMEmulator.DLOG_NAMESPACE;
-import static org.junit.Assert.assertTrue;
-
-/**
- * Test Namespace Builder
- */
-public class TestDistributedLogNamespaceBuilder extends TestDistributedLogBase {
-
-    @Test(timeout = 60000, expected = NullPointerException.class)
-    public void testEmptyBuilder() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder().build();
-    }
-
-    @Test(timeout = 60000, expected = NullPointerException.class)
-    public void testMissingUri() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
-                .conf(new DistributedLogConfiguration())
-                .build();
-    }
-
-    @Test(timeout = 60000, expected = NullPointerException.class)
-    public void testMissingSchemeInUri() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
-                .conf(new DistributedLogConfiguration())
-                .uri(new URI("/test"))
-                .build();
-    }
-
-    @Test(timeout = 60000, expected = IllegalArgumentException.class)
-    public void testInvalidSchemeInUri() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
-                .conf(new DistributedLogConfiguration())
-                .uri(new URI("dist://invalid/scheme/in/uri"))
-                .build();
-    }
-
-    @Test(timeout = 60000, expected = IllegalArgumentException.class)
-    public void testInvalidSchemeCorrectBackendInUri() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
-                .conf(new DistributedLogConfiguration())
-                .uri(new URI("dist-bk://invalid/scheme/in/uri"))
-                .build();
-    }
-
-    @Test(timeout = 60000, expected = IllegalArgumentException.class)
-    public void testUnknownBackendInUri() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
-                .conf(new DistributedLogConfiguration())
-                .uri(new URI("distributedlog-unknown://invalid/scheme/in/uri"))
-                .build();
-    }
-
-    @Test(timeout = 60000, expected = NullPointerException.class)
-    public void testNullStatsLogger() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
-                .conf(new DistributedLogConfiguration())
-                .uri(new URI("distributedlog-bk://localhost/distributedlog"))
-                .statsLogger(null)
-                .build();
-    }
-
-    @Test(timeout = 60000, expected = NullPointerException.class)
-    public void testNullClientId() throws Exception {
-        DistributedLogNamespaceBuilder.newBuilder()
-                .conf(new DistributedLogConfiguration())
-                .uri(new URI("distributedlog-bk://localhost/distributedlog"))
-                .clientId(null)
-                .build();
-    }
-
-    @Test(timeout = 60000)
-    public void testBuildBKDistributedLogNamespace() throws Exception {
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(new DistributedLogConfiguration())
-                .uri(new URI("distributedlog-bk://" + zkServers + DLOG_NAMESPACE + "/bknamespace"))
-                .build();
-        try {
-            assertTrue("distributedlog-bk:// should build bookkeeper based distributedlog namespace",
-                    namespace instanceof BKDistributedLogNamespace);
-        } finally {
-            namespace.close();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testBuildWhenMissingBackendInUri() throws Exception {
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(new DistributedLogConfiguration())
-                .uri(new URI("distributedlog://" + zkServers + DLOG_NAMESPACE + "/defaultnamespace"))
-                .build();
-        try {
-            assertTrue("distributedlog:// should build bookkeeper based distributedlog namespace",
-                    namespace instanceof BKDistributedLogNamespace);
-        } finally {
-            namespace.close();
-        }
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/net/TestDNSResolver.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/net/TestDNSResolver.java
deleted file mode 100644
index e5b8f04..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/net/TestDNSResolver.java
+++ /dev/null
@@ -1,96 +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 com.twitter.distributedlog.net;
-
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-public class TestDNSResolver {
-
-    private static final String host1 = "r1-w1rack1-1111-2222.distributedlog.io";
-    private static final String host2 = "r2-w2rack2-3333-4444.distributedlog.io";
-
-    @Test(timeout = 20000)
-    public void testDNSResolverForRacks() {
-        DNSResolver dnsResolver = new DNSResolverForRacks("");
-
-        List<String> ipList = new ArrayList<String>();
-        ipList.add("192.0.0.1");
-        List<String> racks = dnsResolver.resolve(ipList);
-        assertEquals(DNSResolverForRacks.DEFAULT_RACK, racks.get(0));
-
-        List<String> unknownList = new ArrayList<String>();
-        unknownList.add("unknown");
-        racks = dnsResolver.resolve(unknownList);
-        assertEquals(DNSResolverForRacks.DEFAULT_RACK, racks.get(0));
-
-        List<String> r1List = new ArrayList<String>();
-        r1List.add(host1);
-        racks = dnsResolver.resolve(r1List);
-        assertEquals("/r1/w1rack1", racks.get(0));
-
-        List<String> r2List = new ArrayList<String>();
-        r2List.add(host2);
-        racks = dnsResolver.resolve(r2List);
-        assertEquals("/r2/w2rack2", racks.get(0));
-    }
-
-    @Test(timeout = 20000)
-    public void testDNSResolverForRows() {
-        DNSResolver dnsResolver = new DNSResolverForRows("");
-
-        List<String> ipList = new ArrayList<String>();
-        ipList.add("192.0.0.1");
-        List<String> rows = dnsResolver.resolve(ipList);
-        assertEquals(DNSResolverForRows.DEFAULT_ROW, rows.get(0));
-
-        List<String> unknownList = new ArrayList<String>();
-        unknownList.add("unknown");
-        rows = dnsResolver.resolve(unknownList);
-        assertEquals(DNSResolverForRows.DEFAULT_ROW, rows.get(0));
-
-        List<String> r1List = new ArrayList<String>();
-        r1List.add(host1);
-        rows = dnsResolver.resolve(r1List);
-        assertEquals("/r1/w1", rows.get(0));
-
-        List<String> r2List = new ArrayList<String>();
-        r2List.add(host2);
-        rows = dnsResolver.resolve(r2List);
-        assertEquals("/r2/w2", rows.get(0));
-    }
-
-    @Test(timeout = 20000)
-    public void testDNSResolverOverrides() {
-        DNSResolver dnsResolver = new DNSResolverForRacks("r1-w1rack1-1111-2222:r3;r2-w2rack2-3333-4444:r3");
-
-        List<String> r1List = new ArrayList<String>();
-        r1List.add(host1);
-        List<String> racks = dnsResolver.resolve(r1List);
-        assertEquals("/r3/w1rack1", racks.get(0));
-
-        List<String> r2List = new ArrayList<String>();
-        r2List.add(host2);
-        racks = dnsResolver.resolve(r2List);
-        assertEquals("/r3/w2rack2", racks.get(0));
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/net/TestNetUtils.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/net/TestNetUtils.java
deleted file mode 100644
index 0a928a0..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/net/TestNetUtils.java
+++ /dev/null
@@ -1,82 +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 com.twitter.distributedlog.net;
-
-import org.apache.bookkeeper.net.DNSToSwitchMapping;
-import org.junit.Test;
-
-import java.util.List;
-
-import static org.junit.Assert.*;
-
-/**
- * Test Cases of {@link NetUtils}.
- */
-public class TestNetUtils {
-
-    static class DNSResolverWithDefaultConstructor implements DNSToSwitchMapping {
-
-        public DNSResolverWithDefaultConstructor() {}
-
-        @Override
-        public List<String> resolve(List<String> list) {
-            return list;
-        }
-
-        @Override
-        public void reloadCachedMappings() {
-            // no-op
-        }
-    }
-
-    static class DNSResolverWithUnknownConstructor implements DNSToSwitchMapping {
-
-        public DNSResolverWithUnknownConstructor(int var1, int var2, int var3) {}
-
-        @Override
-        public List<String> resolve(List<String> list) {
-            return list;
-        }
-
-        @Override
-        public void reloadCachedMappings() {
-            // no-op
-        }
-    }
-
-    @Test(timeout = 20000)
-    public void testGetDNSResolverWithOverrides() throws Exception {
-        DNSToSwitchMapping dnsResolver =
-                NetUtils.getDNSResolver(DNSResolverForRacks.class, "");
-        assertEquals("Should succeed to load " + DNSResolverForRacks.class,
-                dnsResolver.getClass(), DNSResolverForRacks.class);
-    }
-
-    @Test(timeout = 20000)
-    public void testGetDNSResolverWithDefaultConstructor() throws Exception {
-        DNSToSwitchMapping dnsResolver =
-                NetUtils.getDNSResolver(DNSResolverWithDefaultConstructor.class, "");
-        assertEquals("Should succeed to load " + DNSResolverWithDefaultConstructor.class,
-                dnsResolver.getClass(), DNSResolverWithDefaultConstructor.class);
-    }
-
-    @Test(timeout = 20000, expected = RuntimeException.class)
-    public void testGetDNSResolverWithUnknownConstructor() throws Exception {
-        NetUtils.getDNSResolver(DNSResolverWithUnknownConstructor.class, "");
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/rate/TestMovingAverageRate.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/rate/TestMovingAverageRate.java
deleted file mode 100644
index 85c82e5..0000000
--- a/distributedlog-core/src/test/java/com/twitter/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 com.twitter.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/com/twitter/distributedlog/selector/TestLogRecordSelectors.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/selector/TestLogRecordSelectors.java
deleted file mode 100644
index c06c935..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/selector/TestLogRecordSelectors.java
+++ /dev/null
@@ -1,128 +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 com.twitter.distributedlog.selector;
-
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.DLSN;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-/**
- * Test Case for {@link LogRecordSelector}s.
- */
-public class TestLogRecordSelectors {
-
-    @Test(timeout = 60000)
-    public void testFirstRecordSelector() {
-        FirstRecordSelector selectorIncludeControlRecord =
-                new FirstRecordSelector(true);
-
-        for (int i = 0; i < 5; i++) {
-            selectorIncludeControlRecord.process(
-                    DLMTestUtil.getLogRecordWithDLSNInstance(
-                            new DLSN(1L, i * 2, 0L), i * 2, true));
-            selectorIncludeControlRecord.process(
-                    DLMTestUtil.getLogRecordWithDLSNInstance(
-                            new DLSN(1L, i * 2 + 1, 0L), i * 2 + 1));
-        }
-        assertEquals(new DLSN(1L, 0L, 0L), selectorIncludeControlRecord.result().getDlsn());
-
-        FirstRecordSelector selectorExcludeControlRecord =
-                new FirstRecordSelector(false);
-
-        for (int i = 0; i < 5; i++) {
-            selectorExcludeControlRecord.process(
-                    DLMTestUtil.getLogRecordWithDLSNInstance(
-                            new DLSN(1L, i * 2, 0L), i * 2, true));
-            selectorExcludeControlRecord.process(
-                    DLMTestUtil.getLogRecordWithDLSNInstance(
-                            new DLSN(1L, i * 2 + 1, 0L), i * 2 + 1));
-        }
-        assertEquals(new DLSN(1L, 1L, 0L), selectorExcludeControlRecord.result().getDlsn());
-    }
-
-    @Test(timeout = 60000)
-    public void testLastRecordSelector() {
-        LastRecordSelector selector = new LastRecordSelector();
-
-        for (int i = 0; i < 10; i++) {
-            selector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
-                    new DLSN(1L, i, 0L), i));
-        }
-        assertEquals(new DLSN(1L, 9L, 0L), selector.result().getDlsn());
-    }
-
-    @Test(timeout = 60000)
-    public void testFirstDLSNNotLessThanSelector() {
-        DLSN dlsn = new DLSN(5L, 5L, 0L);
-
-        FirstDLSNNotLessThanSelector largerSelector =
-                new FirstDLSNNotLessThanSelector(dlsn);
-        for (int i = 0; i < 10; i++) {
-            largerSelector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
-                    new DLSN(4L, i, 0L), i));
-        }
-        assertNull(largerSelector.result());
-
-        FirstDLSNNotLessThanSelector smallerSelector =
-                new FirstDLSNNotLessThanSelector(dlsn);
-        for (int i = 0; i < 10; i++) {
-            smallerSelector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
-                    new DLSN(6L, i, 0L), i));
-        }
-        assertEquals(new DLSN(6L, 0L, 0L), smallerSelector.result().getDlsn());
-
-        FirstDLSNNotLessThanSelector selector =
-                new FirstDLSNNotLessThanSelector(dlsn);
-        for (int i = 0; i < 10; i++) {
-            selector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
-                    new DLSN(5L, i, 0L), i));
-        }
-        assertEquals(dlsn, selector.result().getDlsn());
-    }
-
-    @Test(timeout = 60000)
-    public void testFirstTxIdNotLessThanSelector() {
-        long txId = 5 * 10 + 5;
-
-        FirstTxIdNotLessThanSelector largerSelector =
-                new FirstTxIdNotLessThanSelector(txId);
-        for (int i = 0; i < 10; i++) {
-            largerSelector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
-                    new DLSN(4L, i, 0L), 4 * 10 + i));
-        }
-        assertEquals(49, largerSelector.result().getTransactionId());
-
-        FirstTxIdNotLessThanSelector smallerSelector =
-                new FirstTxIdNotLessThanSelector(txId);
-        for (int i = 0; i < 10; i++) {
-            smallerSelector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
-                    new DLSN(6L, i, 0L), 6 * 10 + i));
-        }
-        assertEquals(6 * 10, smallerSelector.result().getTransactionId());
-
-        FirstTxIdNotLessThanSelector selector =
-                new FirstTxIdNotLessThanSelector(txId);
-        for (int i = 0; i < 10; i++) {
-            selector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
-                    new DLSN(5L, i, 0L), 5 * 10 + i));
-        }
-        assertEquals(txId, selector.result().getTransactionId());
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/tools/TestDistributedLogTool.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/tools/TestDistributedLogTool.java
deleted file mode 100644
index cbabf2a..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/tools/TestDistributedLogTool.java
+++ /dev/null
@@ -1,231 +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 com.twitter.distributedlog.tools;
-
-import java.net.URI;
-
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import com.twitter.distributedlog.LocalDLMEmulator;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.LogReader;
-import com.twitter.distributedlog.exceptions.ZKException;
-import com.twitter.distributedlog.tools.DistributedLogTool.*;
-import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.zookeeper.KeeperException;
-import org.junit.BeforeClass;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-import static org.junit.Assert.*;
-
-public class TestDistributedLogTool extends TestDistributedLogBase {
-
-    static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogTool.class);
-
-    static final String defaultLedgerPath = LocalDLMEmulator.getBkLedgerPath();
-    static final String defaultPath = "/test/namespace";
-    static final String defaultHost = "127.0.0.1";
-    static final String defaultPrivilegedZkAclId = "NathanielP";
-    static URI defaultUri = null;
-
-    static final String ADMIN_TOOL = com.twitter.distributedlog.admin.DistributedLogAdmin.class.getName();
-
-    @BeforeClass
-    public static void setupDefaults() throws Exception {
-        defaultUri = DLMTestUtil.createDLMURI(zkPort, defaultPath);
-        DistributedLogManager dlm = DLMTestUtil.createNewDLM("DefaultStream", conf, defaultUri);
-        bindStream(defaultUri, defaultLedgerPath, defaultHost);
-        DLMTestUtil.generateCompletedLogSegments(dlm, conf, 3, 8192);
-        dlm.close();
-    }
-
-    private static int runTool(String[] args) throws Exception {
-        Tool tool = ReflectionUtils.newInstance(args[0], Tool.class);
-        String[] newArgs = new String[args.length - 1];
-        System.arraycopy(args, 1, newArgs, 0, newArgs.length);
-        int rc = tool.run(newArgs);
-        assertTrue(0 == rc);
-        return rc;
-    }
-
-    static void bindStream(URI uri, String ledgerPath, String zkHosts) throws Exception {
-        String[] args = new String[] { ADMIN_TOOL,
-            "bind",
-            "-c",
-            "-l", ledgerPath,
-            "-s", zkHosts,
-            "-f", uri.toString() };
-        runTool(args);
-    }
-
-    static void createStream(URI uri, String prefix, String expression, String zkAclId) throws Exception {
-        CreateCommand cmd = new CreateCommand();
-        cmd.setUri(defaultUri);
-        cmd.setPrefix(prefix);
-        cmd.setExpression(expression);
-        cmd.setForce(true);
-        cmd.setZkAclId(zkAclId);
-        assertEquals(0, cmd.runCmd());
-    }
-
-    void deleteStream(URI uri, String stream) throws Exception {
-        DeleteCommand cmd = new DeleteCommand();
-        cmd.setUri(defaultUri);
-        cmd.setStreamName(stream);
-        assertEquals(0, cmd.runCmd());
-    }
-
-    void list(URI uri) throws Exception {
-        ListCommand cmd = new ListCommand();
-        cmd.setUri(defaultUri);
-        assertEquals(0, cmd.runCmd());
-    }
-
-    @Test(timeout = 60000)
-    public void testToolCreate() throws Exception {
-        createStream(defaultUri, "0", "TestPrefix", null);
-    }
-
-    @Test(timeout = 60000)
-    public void testToolCreateZkAclId() throws Exception {
-        createStream(defaultUri, "0", "CreateAclStream", defaultPrivilegedZkAclId);
-        try {
-            DistributedLogManager dlm = DLMTestUtil.createNewDLM("0CreateAclStream", conf, defaultUri);
-            DLMTestUtil.generateCompletedLogSegments(dlm, conf, 3, 1000);
-            dlm.close();
-        } catch (ZKException ex) {
-            assertEquals(KeeperException.Code.NOAUTH, ex.getKeeperExceptionCode());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testToolDelete() throws Exception {
-        createStream(defaultUri, "1", "TestPrefix", null);
-        deleteStream(defaultUri, "1TestPrefix");
-    }
-
-    @Test(timeout = 60000)
-    public void testToolDeleteAllocPool() throws Exception {
-        try {
-            DeleteAllocatorPoolCommand cmd = new DeleteAllocatorPoolCommand();
-            cmd.setUri(defaultUri);
-            assertEquals(0, cmd.runCmd());
-            fail("should have failed");
-        } catch (org.apache.zookeeper.KeeperException.NoNodeException ex) {
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testToolList() throws Exception {
-        list(defaultUri);
-    }
-
-    @Test(timeout = 60000)
-    public void testToolDump() throws Exception {
-        DumpCommand cmd = new DumpCommand();
-        cmd.setUri(defaultUri);
-        cmd.setStreamName("DefaultStream");
-        cmd.setFromTxnId(Long.valueOf(0));
-        assertEquals(0, cmd.runCmd());
-    }
-
-    @Test(timeout = 60000)
-    public void testToolShow() throws Exception {
-        ShowCommand cmd = new ShowCommand();
-        cmd.setUri(defaultUri);
-        cmd.setStreamName("DefaultStream");
-        assertEquals(0, cmd.runCmd());
-    }
-
-    @Test(timeout = 60000)
-    public void testToolTruncate() throws Exception {
-        DistributedLogManager dlm = DLMTestUtil.createNewDLM("TruncateStream", conf, defaultUri);
-        DLMTestUtil.generateCompletedLogSegments(dlm, conf, 3, 1000);
-        dlm.close();
-
-        TruncateCommand cmd = new TruncateCommand();
-        cmd.setUri(defaultUri);
-        cmd.setFilter("TruncateStream");
-        cmd.setForce(true);
-        assertEquals(0, cmd.runCmd());
-    }
-
-    @Test(timeout = 60000)
-    public void testToolInspect() throws Exception {
-        InspectCommand cmd = new InspectCommand();
-        cmd.setUri(defaultUri);
-        cmd.setForce(true);
-        assertEquals(0, cmd.runCmd());
-    }
-
-    @Test(timeout = 60000)
-    public void testToolReadLastConfirmed() throws Exception {
-        ReadLastConfirmedCommand cmd = new ReadLastConfirmedCommand();
-        cmd.setUri(defaultUri);
-        cmd.setLedgerId(99999999);
-
-        // Too hard to predict ledger entry id. Settle for basicaly
-        // correct functionality.
-        try {
-            cmd.runCmd();
-        } catch (BKNoSuchLedgerExistsException ex) {
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testToolReadEntriesCommand() throws Exception {
-        ReadEntriesCommand cmd = new ReadEntriesCommand();
-        cmd.setUri(defaultUri);
-        cmd.setLedgerId(99999999);
-        try {
-            cmd.runCmd();
-        } catch (BKNoSuchLedgerExistsException ex) {
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testToolTruncateStream() throws Exception {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(conf);
-        confLocal.setLogSegmentCacheEnabled(false);
-        DistributedLogManager dlm = DLMTestUtil.createNewDLM("testToolTruncateStream", confLocal, defaultUri);
-        DLMTestUtil.generateCompletedLogSegments(dlm, confLocal, 3, 1000);
-
-        DLSN dlsn = new DLSN(2,1,0);
-        TruncateStreamCommand cmd = new TruncateStreamCommand();
-        cmd.setDlsn(dlsn);
-        cmd.setUri(defaultUri);
-        cmd.setStreamName("testToolTruncateStream");
-        cmd.setForce(true);
-
-        assertEquals(0, cmd.runCmd());
-
-        LogReader reader = dlm.getInputStream(0);
-        LogRecordWithDLSN record = reader.readNext(false);
-        assertEquals(dlsn, record.getDlsn());
-
-        reader.close();
-        dlm.close();
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestConfUtils.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestConfUtils.java
deleted file mode 100644
index 7061423..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestConfUtils.java
+++ /dev/null
@@ -1,53 +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 com.twitter.distributedlog.util;
-
-import org.apache.commons.configuration.CompositeConfiguration;
-import org.apache.commons.configuration.Configuration;
-import org.junit.Test;
-
-import static org.junit.Assert.*;
-
-public class TestConfUtils {
-
-    @Test(timeout = 60000)
-    public void testLoadConfiguration() {
-        Configuration conf1 = new CompositeConfiguration();
-        conf1.setProperty("key1", "value1");
-        conf1.setProperty("key2", "value2");
-        conf1.setProperty("key3", "value3");
-
-        Configuration conf2 = new CompositeConfiguration();
-        conf2.setProperty("bkc.key1", "bkc.value1");
-        conf2.setProperty("bkc.key4", "bkc.value4");
-
-        assertEquals("value1", conf1.getString("key1"));
-        assertEquals("value2", conf1.getString("key2"));
-        assertEquals("value3", conf1.getString("key3"));
-        assertEquals(null, conf1.getString("key4"));
-
-        ConfUtils.loadConfiguration(conf1, conf2, "bkc.");
-
-        assertEquals("bkc.value1", conf1.getString("key1"));
-        assertEquals("value2", conf1.getString("key2"));
-        assertEquals("value3", conf1.getString("key3"));
-        assertEquals("bkc.value4", conf1.getString("key4"));
-        assertEquals(null, conf1.getString("bkc.key1"));
-        assertEquals(null, conf1.getString("bkc.key4"));
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestDLUtils.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestDLUtils.java
deleted file mode 100644
index 8869a57..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestDLUtils.java
+++ /dev/null
@@ -1,273 +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 com.twitter.distributedlog.util;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import org.junit.Test;
-
-import java.util.List;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
-
-/**
- * Test Case for {@link DLUtils}
- */
-public class TestDLUtils {
-
-    private static LogSegmentMetadata completedLogSegment(
-            long logSegmentSequenceNumber,
-            long fromTxnId,
-            long toTxnId) {
-        return completedLogSegment(
-                logSegmentSequenceNumber,
-                fromTxnId,
-                toTxnId,
-                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
-    }
-
-    private static LogSegmentMetadata completedLogSegment(
-        long logSegmentSequenceNumber,
-        long fromTxnId,
-        long toTxnId,
-        int version) {
-        return DLMTestUtil.completedLogSegment(
-                "/logsegment/" + fromTxnId,
-                fromTxnId,
-                fromTxnId,
-                toTxnId,
-                100,
-                logSegmentSequenceNumber,
-                999L,
-                0L,
-                version);
-    }
-
-    private static LogSegmentMetadata inprogressLogSegment(
-            long logSegmentSequenceNumber, long firstTxId) {
-        return inprogressLogSegment(
-                logSegmentSequenceNumber,
-                firstTxId,
-                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
-    }
-
-    private static LogSegmentMetadata inprogressLogSegment(
-            long logSegmentSequenceNumber, long firstTxId, int version) {
-        return DLMTestUtil.inprogressLogSegment(
-                "/logsegment/" + firstTxId,
-                firstTxId,
-                firstTxId,
-                logSegmentSequenceNumber,
-                version);
-    }
-
-    @Test(timeout = 60000)
-    public void testFindLogSegmentNotLessThanTxnId() throws Exception {
-        long txnId = 999L;
-        // empty list
-        List<LogSegmentMetadata> emptyList = Lists.newArrayList();
-        assertEquals(-1, DLUtils.findLogSegmentNotLessThanTxnId(emptyList, txnId));
-
-        // list that all segment's txn id is larger than txn-id-to-search
-        List<LogSegmentMetadata> list1 = Lists.newArrayList(
-                completedLogSegment(1L, 1000L, 2000L));
-        assertEquals(-1, DLUtils.findLogSegmentNotLessThanTxnId(list1, txnId));
-
-        List<LogSegmentMetadata> list2 = Lists.newArrayList(
-                inprogressLogSegment(1L, 1000L));
-        assertEquals(-1, DLUtils.findLogSegmentNotLessThanTxnId(list2, txnId));
-
-        // the first log segment whose first txn id is less than txn-id-to-search
-        List<LogSegmentMetadata> list3 = Lists.newArrayList(
-                completedLogSegment(1L, 0L, 99L),
-                completedLogSegment(2L, 1000L, 2000L)
-        );
-        assertEquals(1, DLUtils.findLogSegmentNotLessThanTxnId(list3, txnId));
-
-        List<LogSegmentMetadata> list4 = Lists.newArrayList(
-                completedLogSegment(1L, 0L, 990L),
-                completedLogSegment(2L, 1000L, 2000L)
-        );
-        assertEquals(1, DLUtils.findLogSegmentNotLessThanTxnId(list4, txnId));
-
-        List<LogSegmentMetadata> list5 = Lists.newArrayList(
-                inprogressLogSegment(1L, 0L),
-                inprogressLogSegment(2L, 1000L)
-        );
-        assertEquals(0, DLUtils.findLogSegmentNotLessThanTxnId(list5, txnId));
-
-        // list that all segment's txn id is less than txn-id-to-search
-        List<LogSegmentMetadata> list6_0 = Lists.newArrayList(
-                completedLogSegment(1L, 100L, 200L));
-        assertEquals(0, DLUtils.findLogSegmentNotLessThanTxnId(list6_0, txnId));
-
-        List<LogSegmentMetadata> list6_1 = Lists.newArrayList(
-                completedLogSegment(1L, 100L, 199L),
-                completedLogSegment(2L, 200L, 299L));
-        assertEquals(1, DLUtils.findLogSegmentNotLessThanTxnId(list6_1, txnId));
-
-        List<LogSegmentMetadata> list7 = Lists.newArrayList(
-                inprogressLogSegment(1L, 100L));
-        assertEquals(0, DLUtils.findLogSegmentNotLessThanTxnId(list7, txnId));
-
-        // list that first segment's first txn id equals to txn-id-to-search
-        List<LogSegmentMetadata> list8 = Lists.newArrayList(
-                completedLogSegment(1L, 999L, 2000L));
-        assertEquals(0, DLUtils.findLogSegmentNotLessThanTxnId(list8, txnId));
-
-        List<LogSegmentMetadata> list9 = Lists.newArrayList(
-                inprogressLogSegment(1L, 999L));
-        assertEquals(0, DLUtils.findLogSegmentNotLessThanTxnId(list9, txnId));
-
-        List<LogSegmentMetadata> list10 = Lists.newArrayList(
-                completedLogSegment(1L, 0L, 999L),
-                completedLogSegment(2L, 999L, 2000L));
-        assertEquals(0, DLUtils.findLogSegmentNotLessThanTxnId(list10, txnId));
-
-        List<LogSegmentMetadata> list11 = Lists.newArrayList(
-                completedLogSegment(1L, 0L, 99L),
-                completedLogSegment(2L, 999L, 2000L));
-        assertEquals(1, DLUtils.findLogSegmentNotLessThanTxnId(list11, txnId));
-
-        List<LogSegmentMetadata> list12 = Lists.newArrayList(
-                inprogressLogSegment(1L, 0L),
-                inprogressLogSegment(2L, 999L));
-        assertEquals(1, DLUtils.findLogSegmentNotLessThanTxnId(list12, txnId));
-    }
-
-    @Test(timeout = 60000)
-    public void testNextLogSegmentSequenceNumber() throws Exception {
-        List<LogSegmentMetadata> v1List = Lists.newArrayList(
-                completedLogSegment(2L, 100L, 199L, LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value),
-                completedLogSegment(1L, 0L, 99L, LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value));
-        assertNull(DLUtils.nextLogSegmentSequenceNumber(v1List));
-
-        List<LogSegmentMetadata> afterV1List = Lists.newArrayList(
-                completedLogSegment(2L, 100L, 199L),
-                completedLogSegment(1L, 0L, 99L));
-        assertEquals((Long) 3L, DLUtils.nextLogSegmentSequenceNumber(afterV1List));
-
-        List<LogSegmentMetadata> mixList1 = Lists.newArrayList(
-                completedLogSegment(2L, 100L, 199L, LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value),
-                completedLogSegment(1L, 0L, 99L));
-        assertEquals((Long) 3L, DLUtils.nextLogSegmentSequenceNumber(mixList1));
-
-        List<LogSegmentMetadata> mixList2 = Lists.newArrayList(
-                completedLogSegment(2L, 100L, 199L),
-                completedLogSegment(1L, 0L, 99L, LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value));
-        assertEquals((Long) 3L, DLUtils.nextLogSegmentSequenceNumber(mixList2));
-    }
-
-    @Test(timeout = 60000, expected = UnexpectedException.class)
-    public void testUnexpectedExceptionOnComputeStartSequenceId() throws Exception {
-        List<LogSegmentMetadata> segments = Lists.newArrayList(
-                inprogressLogSegment(3L, 201L),
-                inprogressLogSegment(2L, 101L),
-                completedLogSegment(1L, 1L, 100L).mutator().setStartSequenceId(1L).build()
-        );
-        DLUtils.computeStartSequenceId(segments, segments.get(0));
-    }
-
-    @Test(timeout = 60000)
-    public void testComputeStartSequenceIdOnEmptyList() throws Exception {
-        List<LogSegmentMetadata> emptyList = Lists.newArrayList();
-        assertEquals(0L, DLUtils.computeStartSequenceId(emptyList, inprogressLogSegment(1L, 1L)));
-    }
-
-    @Test(timeout = 60000)
-    public void testComputeStartSequenceIdOnLowerSequenceNumberSegment() throws Exception {
-        List<LogSegmentMetadata> segments = Lists.newArrayList(
-                completedLogSegment(3L, 201L, 300L).mutator().setStartSequenceId(201L).build(),
-                completedLogSegment(2L, 101L, 200L).mutator().setStartSequenceId(101L).build()
-        );
-        assertEquals(0L, DLUtils.computeStartSequenceId(segments, inprogressLogSegment(1L, 1L)));
-    }
-
-    @Test(timeout = 60000)
-    public void testComputeStartSequenceIdOnHigherSequenceNumberSegment() throws Exception {
-        List<LogSegmentMetadata> segments = Lists.newArrayList(
-                completedLogSegment(3L, 201L, 300L).mutator().setStartSequenceId(201L).build(),
-                completedLogSegment(2L, 101L, 200L).mutator().setStartSequenceId(101L).build()
-        );
-        assertEquals(0L, DLUtils.computeStartSequenceId(segments, inprogressLogSegment(5L, 401L)));
-    }
-
-    @Test(timeout = 60000)
-    public void testComputeStartSequenceId() throws Exception {
-        List<LogSegmentMetadata> segments = Lists.newArrayList(
-                completedLogSegment(3L, 201L, 300L).mutator()
-                        .setStartSequenceId(201L).setRecordCount(100).build(),
-                completedLogSegment(2L, 101L, 200L).mutator()
-                        .setStartSequenceId(101L).setRecordCount(100).build()
-        );
-        assertEquals(301L, DLUtils.computeStartSequenceId(segments, inprogressLogSegment(4L, 301L)));
-    }
-
-    @Test(timeout = 60000)
-    public void testSerDeLogSegmentSequenceNumber() throws Exception {
-        long sn = 123456L;
-        byte[] snData = Long.toString(sn).getBytes(UTF_8);
-        assertEquals("Deserialization should succeed",
-                sn, DLUtils.deserializeLogSegmentSequenceNumber(snData));
-        assertArrayEquals("Serialization should succeed",
-                snData, DLUtils.serializeLogSegmentSequenceNumber(sn));
-    }
-
-    @Test(timeout = 60000, expected = NumberFormatException.class)
-    public void testDeserilizeInvalidLSSN() throws Exception {
-        byte[] corruptedData = "corrupted-lssn".getBytes(UTF_8);
-        DLUtils.deserializeLogSegmentSequenceNumber(corruptedData);
-    }
-
-    @Test(timeout = 60000)
-    public void testSerDeLogRecordTxnId() throws Exception {
-        long txnId = 123456L;
-        byte[] txnData = Long.toString(txnId).getBytes(UTF_8);
-        assertEquals("Deserialization should succeed",
-                txnId, DLUtils.deserializeTransactionId(txnData));
-        assertArrayEquals("Serialization should succeed",
-                txnData, DLUtils.serializeTransactionId(txnId));
-    }
-
-    @Test(timeout = 60000, expected = NumberFormatException.class)
-    public void testDeserilizeInvalidLogRecordTxnId() throws Exception {
-        byte[] corruptedData = "corrupted-txn-id".getBytes(UTF_8);
-        DLUtils.deserializeTransactionId(corruptedData);
-    }
-
-    @Test(timeout = 60000)
-    public void testSerDeLedgerId() throws Exception {
-        long ledgerId = 123456L;
-        byte[] ledgerIdData = Long.toString(ledgerId).getBytes(UTF_8);
-        assertEquals("Deserialization should succeed",
-                ledgerId, DLUtils.bytes2LogSegmentId(ledgerIdData));
-        assertArrayEquals("Serialization should succeed",
-                ledgerIdData, DLUtils.logSegmentId2Bytes(ledgerId));
-    }
-
-    @Test(timeout = 60000, expected = NumberFormatException.class)
-    public void testDeserializeInvalidLedgerId() throws Exception {
-        byte[] corruptedData = "corrupted-ledger-id".getBytes(UTF_8);
-        DLUtils.bytes2LogSegmentId(corruptedData);
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestFutureUtils.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestFutureUtils.java
deleted file mode 100644
index 12679e0..0000000
--- a/distributedlog-core/src/test/java/com/twitter/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 com.twitter.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/com/twitter/distributedlog/util/TestPermitManager.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestPermitManager.java
deleted file mode 100644
index db87a65..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestPermitManager.java
+++ /dev/null
@@ -1,92 +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 com.twitter.distributedlog.util;
-
-import com.twitter.distributedlog.zk.LimitedPermitManager;
-import org.junit.Test;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-
-import static org.junit.Assert.*;
-
-public class TestPermitManager {
-
-    @Test(timeout = 60000)
-    public void testUnlimitedPermitManager() {
-        PermitManager pm = PermitManager.UNLIMITED_PERMIT_MANAGER;
-        List<PermitManager.Permit> permits = new ArrayList<PermitManager.Permit>();
-        for (int i = 0; i < 10; i++) {
-            permits.add(pm.acquirePermit());
-        }
-        for (int i = 0; i < 10; i++) {
-            assertTrue(permits.get(i).isAllowed());
-            pm.releasePermit(permits.get(i));
-        }
-        PermitManager.Permit permit = pm.acquirePermit();
-        pm.disallowObtainPermits(permit);
-        pm.releasePermit(permit);
-
-        for (int i = 0; i < 10; i++) {
-            permits.add(pm.acquirePermit());
-        }
-        for (int i = 0; i < 10; i++) {
-            assertTrue(permits.get(i).isAllowed());
-            pm.releasePermit(permits.get(i));
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testLimitedPermitManager() {
-        ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor();
-        PermitManager pm = new LimitedPermitManager(1, 0, TimeUnit.SECONDS, executorService);
-        PermitManager.Permit permit1 = pm.acquirePermit();
-        PermitManager.Permit permit2 = pm.acquirePermit();
-        assertTrue(permit1.isAllowed());
-        assertFalse(permit2.isAllowed());
-        pm.releasePermit(permit2);
-        PermitManager.Permit permit3 = pm.acquirePermit();
-        assertFalse(permit3.isAllowed());
-        pm.releasePermit(permit3);
-        pm.releasePermit(permit1);
-        PermitManager.Permit permit4 = pm.acquirePermit();
-        assertTrue(permit4.isAllowed());
-        pm.releasePermit(permit4);
-
-        PermitManager pm2 = new LimitedPermitManager(2, 0, TimeUnit.SECONDS, executorService);
-
-        PermitManager.Permit permit5 = pm2.acquirePermit();
-        PermitManager.Permit permit6 = pm2.acquirePermit();
-        assertTrue(permit5.isAllowed());
-        assertTrue(permit6.isAllowed());
-        assertTrue(pm2.disallowObtainPermits(permit5));
-        assertFalse(pm2.disallowObtainPermits(permit6));
-        pm2.releasePermit(permit5);
-        pm2.releasePermit(permit6);
-        PermitManager.Permit permit7 = pm2.acquirePermit();
-        assertFalse(permit7.isAllowed());
-        pm2.releasePermit(permit7);
-        pm2.allowObtainPermits();
-        PermitManager.Permit permit8 = pm2.acquirePermit();
-        assertTrue(permit8.isAllowed());
-        pm2.releasePermit(permit2);
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestSafeQueueingFuturePool.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestSafeQueueingFuturePool.java
deleted file mode 100644
index bca3b28..0000000
--- a/distributedlog-core/src/test/java/com/twitter/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 com.twitter.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/com/twitter/distributedlog/util/TestTimeSequencer.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestTimeSequencer.java
deleted file mode 100644
index bdeda4f..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestTimeSequencer.java
+++ /dev/null
@@ -1,42 +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 com.twitter.distributedlog.util;
-
-import org.junit.Test;
-import static org.junit.Assert.*;
-
-public class TestTimeSequencer {
-
-    @Test(timeout = 60000)
-    public void testNonDecreasingId() {
-        TimeSequencer sequencer = new TimeSequencer();
-        long lastId = System.currentTimeMillis() + 3600000;
-        sequencer.setLastId(lastId);
-        for (int i = 0; i < 10; i++) {
-            assertEquals(lastId, sequencer.nextId());
-        }
-        sequencer.setLastId(15);
-        long prevId = 15;
-        for (int i = 0; i < 10; i++) {
-            long newId = sequencer.nextId();
-            assertTrue("id should not decrease",
-                    newId >= prevId);
-            prevId = newId;
-        }
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestUtils.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestUtils.java
deleted file mode 100644
index 081918d..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/util/TestUtils.java
+++ /dev/null
@@ -1,125 +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 com.twitter.distributedlog.util;
-
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.ZooKeeperClusterTestCase;
-import org.apache.bookkeeper.meta.ZkVersion;
-import org.apache.bookkeeper.versioning.Versioned;
-import org.apache.zookeeper.AsyncCallback;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.ZooDefs;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.concurrent.CountDownLatch;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.*;
-
-/**
- * Test Utils
- */
-public class TestUtils extends ZooKeeperClusterTestCase {
-
-    private final static int sessionTimeoutMs = 30000;
-
-    private ZooKeeperClient zkc;
-
-    @Before
-    public void setup() throws Exception {
-        zkc = TestZooKeeperClientBuilder.newBuilder()
-                .name("zkc")
-                .uri(DLMTestUtil.createDLMURI(zkPort, "/"))
-                .sessionTimeoutMs(sessionTimeoutMs)
-                .build();
-    }
-
-    @After
-    public void teardown() throws Exception {
-        zkc.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testZkAsyncCreateFulPathOptimisticRecursive() throws Exception {
-        String path1 = "/a/b/c/d";
-        Optional<String> parentPathShouldNotCreate = Optional.absent();
-        final CountDownLatch doneLatch1 = new CountDownLatch(1);
-        Utils.zkAsyncCreateFullPathOptimisticRecursive(zkc, path1, parentPathShouldNotCreate,
-                new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT,
-                new AsyncCallback.StringCallback() {
-                    @Override
-                    public void processResult(int rc, String path, Object ctx, String name) {
-                        doneLatch1.countDown();
-                    }
-                }, null);
-        doneLatch1.await();
-        assertNotNull(zkc.get().exists(path1, false));
-
-        String path2 = "/a/b/c/d/e/f/g";
-        parentPathShouldNotCreate = Optional.of("/a/b/c/d/e");
-        final CountDownLatch doneLatch2 = new CountDownLatch(1);
-        Utils.zkAsyncCreateFullPathOptimisticRecursive(zkc, path2, parentPathShouldNotCreate,
-                new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT,
-                new AsyncCallback.StringCallback() {
-                    @Override
-                    public void processResult(int rc, String path, Object ctx, String name) {
-                        doneLatch2.countDown();
-                    }
-                }, null);
-        doneLatch2.await();
-        assertNull(zkc.get().exists("/a/b/c/d/e", false));
-        assertNull(zkc.get().exists("/a/b/c/d/e/f", false));
-        assertNull(zkc.get().exists("/a/b/c/d/e/f/g", false));
-
-        parentPathShouldNotCreate = Optional.of("/a/b");
-        final CountDownLatch doneLatch3 = new CountDownLatch(1);
-        Utils.zkAsyncCreateFullPathOptimisticRecursive(zkc, path2, parentPathShouldNotCreate,
-                new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT,
-                new AsyncCallback.StringCallback() {
-                    @Override
-                    public void processResult(int rc, String path, Object ctx, String name) {
-                        doneLatch3.countDown();
-                    }
-                }, null);
-        doneLatch3.await();
-        assertNotNull(zkc.get().exists(path2, false));
-    }
-
-    @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));
-        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,
-                zkc.getDefaultACL(), CreateMode.PERSISTENT));
-        data = FutureUtils.result(Utils.zkGetData(zkc.get(), path2, false));
-        assertArrayEquals("Data should return as written",
-                rawData, data.getValue());
-        assertEquals("Version should be zero",
-                0, ((ZkVersion) data.getVersion()).getZnodeVersion());
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/zk/TestZKTransaction.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/zk/TestZKTransaction.java
deleted file mode 100644
index c797e91..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/zk/TestZKTransaction.java
+++ /dev/null
@@ -1,97 +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 com.twitter.distributedlog.zk;
-
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.exceptions.DLIllegalStateException;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
-import org.junit.Test;
-
-import java.util.concurrent.CountDownLatch;
-import javax.annotation.Nullable;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-/**
- * Test Case for zookeeper transaction
- */
-public class TestZKTransaction {
-
-    static class CountDownZKOp extends ZKOp {
-
-        final CountDownLatch commitLatch;
-        final CountDownLatch abortLatch;
-
-        CountDownZKOp(CountDownLatch commitLatch,
-                      CountDownLatch abortLatch) {
-            super(mock(Op.class));
-            this.commitLatch = commitLatch;
-            this.abortLatch = abortLatch;
-        }
-
-        @Override
-        protected void commitOpResult(OpResult opResult) {
-            this.commitLatch.countDown();
-        }
-
-        @Override
-        protected void abortOpResult(Throwable t, @Nullable OpResult opResult) {
-            this.abortLatch.countDown();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testProcessNullResults() throws Exception {
-        ZooKeeperClient zkc = mock(ZooKeeperClient.class);
-        ZKTransaction transaction = new ZKTransaction(zkc);
-        int numOps = 3;
-        final CountDownLatch commitLatch = new CountDownLatch(numOps);
-        final CountDownLatch abortLatch = new CountDownLatch(numOps);
-        for (int i = 0; i < numOps; i++) {
-            transaction.addOp(new CountDownZKOp(commitLatch, abortLatch));
-        }
-        transaction.processResult(
-                KeeperException.Code.CONNECTIONLOSS.intValue(),
-                "test-path",
-                null,
-                null);
-        abortLatch.await();
-        assertEquals(0, abortLatch.getCount());
-        assertEquals(numOps, commitLatch.getCount());
-    }
-
-    @Test(timeout = 60000)
-    public void testAbortTransaction() throws Exception {
-        ZooKeeperClient zkc = mock(ZooKeeperClient.class);
-        ZKTransaction transaction = new ZKTransaction(zkc);
-        int numOps = 3;
-        final CountDownLatch commitLatch = new CountDownLatch(numOps);
-        final CountDownLatch abortLatch = new CountDownLatch(numOps);
-        for (int i = 0; i < numOps; i++) {
-            transaction.addOp(new CountDownZKOp(commitLatch, abortLatch));
-        }
-        transaction.abort(new DLIllegalStateException("Illegal State"));
-        abortLatch.await();
-        assertEquals(0, abortLatch.getCount());
-        assertEquals(numOps, commitLatch.getCount());
-    }
-
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/zk/TestZKVersionedSetOp.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/zk/TestZKVersionedSetOp.java
deleted file mode 100644
index c8ed23c..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/zk/TestZKVersionedSetOp.java
+++ /dev/null
@@ -1,86 +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 com.twitter.distributedlog.zk;
-
-import com.twitter.distributedlog.util.Transaction;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Op;
-import org.apache.zookeeper.OpResult;
-import org.junit.Test;
-
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static org.junit.Assert.*;
-import static org.mockito.Mockito.*;
-
-/**
- * Test Case for versioned set operation
- */
-public class TestZKVersionedSetOp {
-
-    @Test(timeout = 60000)
-    public void testAbortNullOpResult() throws Exception {
-        final AtomicReference<Throwable> exception =
-                new AtomicReference<Throwable>();
-        final CountDownLatch latch = new CountDownLatch(1);
-        ZKVersionedSetOp versionedSetOp =
-                new ZKVersionedSetOp(mock(Op.class), new Transaction.OpListener<Version>() {
-                    @Override
-                    public void onCommit(Version r) {
-                        // no-op
-                    }
-
-                    @Override
-                    public void onAbort(Throwable t) {
-                        exception.set(t);
-                        latch.countDown();
-                    }
-                });
-        KeeperException ke = KeeperException.create(KeeperException.Code.SESSIONEXPIRED);
-        versionedSetOp.abortOpResult(ke, null);
-        latch.await();
-        assertTrue(ke == exception.get());
-    }
-
-    @Test(timeout = 60000)
-    public void testAbortOpResult() throws Exception {
-        final AtomicReference<Throwable> exception =
-                new AtomicReference<Throwable>();
-        final CountDownLatch latch = new CountDownLatch(1);
-        ZKVersionedSetOp versionedSetOp =
-                new ZKVersionedSetOp(mock(Op.class), new Transaction.OpListener<Version>() {
-                    @Override
-                    public void onCommit(Version r) {
-                        // no-op
-                    }
-
-                    @Override
-                    public void onAbort(Throwable t) {
-                        exception.set(t);
-                        latch.countDown();
-                    }
-                });
-        KeeperException ke = KeeperException.create(KeeperException.Code.SESSIONEXPIRED);
-        OpResult opResult = new OpResult.ErrorResult(KeeperException.Code.NONODE.intValue());
-        versionedSetOp.abortOpResult(ke, opResult);
-        latch.await();
-        assertTrue(exception.get() instanceof KeeperException.NoNodeException);
-    }
-}
diff --git a/distributedlog-core/src/test/java/com/twitter/distributedlog/zk/TestZKWatcherManager.java b/distributedlog-core/src/test/java/com/twitter/distributedlog/zk/TestZKWatcherManager.java
deleted file mode 100644
index b702d4c..0000000
--- a/distributedlog-core/src/test/java/com/twitter/distributedlog/zk/TestZKWatcherManager.java
+++ /dev/null
@@ -1,80 +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 com.twitter.distributedlog.zk;
-
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.junit.Test;
-
-import java.util.LinkedList;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-
-import static org.junit.Assert.*;
-
-public class TestZKWatcherManager {
-
-    @Test(timeout = 60000)
-    public void testRegisterUnregisterWatcher() throws Exception {
-        ZKWatcherManager watcherManager = ZKWatcherManager.newBuilder()
-                .name("test-register-unregister-watcher")
-                .zkc(null)
-                .statsLogger(NullStatsLogger.INSTANCE)
-                .build();
-        String path = "/test-register-unregister-watcher";
-        final List<WatchedEvent> events = new LinkedList<WatchedEvent>();
-        final CountDownLatch latch = new CountDownLatch(2);
-        Watcher watcher = new Watcher() {
-            @Override
-            public void process(WatchedEvent event) {
-                events.add(event);
-                latch.countDown();
-            }
-        };
-        watcherManager.registerChildWatcher(path, watcher);
-
-        // fire the event
-        WatchedEvent event0 = new WatchedEvent(
-                Watcher.Event.EventType.NodeCreated,
-                Watcher.Event.KeeperState.SyncConnected,
-                path);
-        WatchedEvent event1 = new WatchedEvent(
-                Watcher.Event.EventType.None,
-                Watcher.Event.KeeperState.SyncConnected,
-                path);
-        WatchedEvent event2 = new WatchedEvent(
-                Watcher.Event.EventType.NodeChildrenChanged,
-                Watcher.Event.KeeperState.SyncConnected,
-                path);
-        watcher.process(event1);
-        watcher.process(event2);
-
-        latch.await();
-
-        assertEquals(2, events.size());
-        assertEquals(event1, events.get(0));
-        assertEquals(event2, events.get(1));
-
-        // unregister watcher
-        watcherManager.unregisterChildWatcher(path, watcher, true);
-        // unregister gauges
-        watcherManager.unregisterGauges();
-        assertEquals(0, watcherManager.childWatches.size());
-    }
-}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/DLMTestUtil.java b/distributedlog-core/src/test/java/org/apache/distributedlog/DLMTestUtil.java
new file mode 100644
index 0000000..96d2d1c
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/DLMTestUtil.java
@@ -0,0 +1,489 @@
+/**
+ * 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 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.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.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;
+import org.apache.bookkeeper.feature.SettableFeatureProvider;
+import org.apache.bookkeeper.versioning.Version;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.net.URL;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertNotNull;
+
+/**
+ * Utility class for setting up bookkeeper ensembles
+ * and bringing individual bookies up and down
+ */
+public class DLMTestUtil {
+    protected static final Logger LOG = LoggerFactory.getLogger(DLMTestUtil.class);
+    private final static byte[] payloadStatic = repeatString("abc", 512).getBytes();
+
+    static String repeatString(String s, int n) {
+        String ret = s;
+        for(int i = 1; i < n; i++) {
+            ret += s;
+        }
+        return ret;
+    }
+
+    public static Map<Long, LogSegmentMetadata> readLogSegments(ZooKeeperClient zkc, String ledgerPath) throws Exception {
+        List<String> children = zkc.get().getChildren(ledgerPath, false);
+        LOG.info("Children under {} : {}", ledgerPath, children);
+        Map<Long, LogSegmentMetadata> segments =
+            new HashMap<Long, LogSegmentMetadata>(children.size());
+        for (String child : children) {
+            LogSegmentMetadata segment =
+                    FutureUtils.result(LogSegmentMetadata.read(zkc, ledgerPath + "/" + child));
+            LOG.info("Read segment {} : {}", child, segment);
+            segments.put(segment.getLogSegmentSequenceNumber(), segment);
+        }
+        return segments;
+    }
+
+    public static URI createDLMURI(int port, String path) throws Exception {
+        return LocalDLMEmulator.createDLMURI("127.0.0.1:" + port, path);
+    }
+
+    public static DistributedLogManager createNewDLM(String name,
+                                                     DistributedLogConfiguration conf,
+                                                     URI uri) throws Exception {
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf).uri(uri).build();
+        return namespace.openLog(name);
+    }
+
+    static MetadataAccessor createNewMetadataAccessor(DistributedLogConfiguration conf,
+                                                      String name,
+                                                      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()
+                .conf(conf).uri(uri).build();
+        return namespace.getNamespaceDriver().getMetadataAccessor(name);
+    }
+
+    public static void fenceStream(DistributedLogConfiguration conf, URI uri, String name) throws Exception {
+        DistributedLogManager dlm = createNewDLM(name, conf, uri);
+        try {
+            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)));
+        } finally {
+            dlm.close();
+        }
+    }
+
+    static long getNumberofLogRecords(DistributedLogManager bkdlm, long startTxId) throws IOException {
+        long numLogRecs = 0;
+        LogReader reader = bkdlm.getInputStream(startTxId);
+        LogRecord record = reader.readNext(false);
+        while (null != record) {
+            numLogRecs++;
+            verifyLogRecord(record);
+            record = reader.readNext(false);
+        }
+        reader.close();
+        return numLogRecs;
+    }
+
+    public static LogRecord getLogRecordInstance(long txId) {
+        return new LogRecord(txId, generatePayload(txId));
+    }
+
+    public static LogRecord getLogRecordInstance(long txId, int size) {
+        ByteBuffer buf = ByteBuffer.allocate(size);
+        return new LogRecord(txId, buf.array());
+    }
+
+    public static void verifyLogRecord(LogRecord record) {
+        assertEquals(generatePayload(record.getTransactionId()).length, record.getPayload().length);
+        assertArrayEquals(generatePayload(record.getTransactionId()), record.getPayload());
+        assertTrue(!record.isControl());
+        verifyPayload(record.getTransactionId(), record.getPayload());
+    }
+
+    static byte[] generatePayload(long txId) {
+        return String.format("%d;%d", txId, txId).getBytes();
+    }
+
+    static void verifyPayload(long txId, byte[] payload) {
+        String[] txIds = new String(payload).split(";");
+        assertEquals(Long.valueOf(txIds[0]), Long.valueOf(txIds[0]));
+    }
+
+    static LogRecord getLargeLogRecordInstance(long txId, boolean control) {
+        LogRecord record = new LogRecord(txId, payloadStatic);
+        if (control) {
+            record.setControl();
+        }
+        return record;
+    }
+
+    static LogRecord getLargeLogRecordInstance(long txId) {
+        return new LogRecord(txId, payloadStatic);
+    }
+
+    static List<LogRecord> getLargeLogRecordInstanceList(long firstTxId, int count) {
+        List<LogRecord> logrecs = new ArrayList<LogRecord>(count);
+        for (long i = 0; i < count; i++) {
+            logrecs.add(getLargeLogRecordInstance(firstTxId + i));
+        }
+        return logrecs;
+    }
+
+    static List<LogRecord> getLogRecordInstanceList(long firstTxId, int count, int size) {
+        List<LogRecord> logrecs = new ArrayList<LogRecord>(count);
+        for (long i = 0; i < count; i++) {
+            logrecs.add(getLogRecordInstance(firstTxId + i, size));
+        }
+        return logrecs;
+    }
+
+    static void verifyLargeLogRecord(LogRecord record) {
+        verifyLargeLogRecord(record.getPayload());
+    }
+
+    static void verifyLargeLogRecord(byte[] payload) {
+        assertArrayEquals(payloadStatic, payload);
+    }
+
+    static LogRecord getEmptyLogRecordInstance(long txId) {
+        return new LogRecord(txId, new byte[0]);
+    }
+
+    static void verifyEmptyLogRecord(LogRecord record) {
+        assertEquals(record.getPayload().length, 0);
+    }
+
+    public static LogRecordWithDLSN getLogRecordWithDLSNInstance(DLSN dlsn, long txId) {
+        return getLogRecordWithDLSNInstance(dlsn, txId, false);
+    }
+
+    public static LogRecordWithDLSN getLogRecordWithDLSNInstance(DLSN dlsn, long txId, boolean isControlRecord) {
+        LogRecordWithDLSN record = new LogRecordWithDLSN(dlsn, txId, generatePayload(txId), 1L);
+        record.setPositionWithinLogSegment((int)txId + 1);
+        if (isControlRecord) {
+            record.setControl();
+        }
+        return record;
+    }
+
+    public static String inprogressZNodeName(long logSegmentSeqNo) {
+        return String.format("%s_%018d", DistributedLogConstants.INPROGRESS_LOGSEGMENT_PREFIX, logSegmentSeqNo);
+    }
+
+    public static String completedLedgerZNodeNameWithVersion(long ledgerId, long firstTxId, long lastTxId, long logSegmentSeqNo) {
+        return String.format("%s_%018d_%018d_%018d_v%dl%d_%04d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX,
+                             firstTxId, lastTxId, logSegmentSeqNo, DistributedLogConstants.LOGSEGMENT_NAME_VERSION, ledgerId,
+                             DistributedLogConstants.LOCAL_REGION_ID);
+    }
+
+    public static String completedLedgerZNodeNameWithTxID(long firstTxId, long lastTxId) {
+        return String.format("%s_%018d_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX, firstTxId, lastTxId);
+    }
+
+    public static String completedLedgerZNodeNameWithLogSegmentSequenceNumber(long logSegmentSeqNo) {
+        return String.format("%s_%018d", DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX, logSegmentSeqNo);
+    }
+
+    public static LogSegmentMetadata inprogressLogSegment(String ledgerPath,
+                                                          long ledgerId,
+                                                          long firstTxId,
+                                                          long logSegmentSeqNo) {
+        return inprogressLogSegment(ledgerPath, ledgerId, firstTxId, logSegmentSeqNo,
+                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
+    }
+
+    public static LogSegmentMetadata inprogressLogSegment(String ledgerPath,
+                                                          long ledgerId,
+                                                          long firstTxId,
+                                                          long logSegmentSeqNo,
+                                                          int version) {
+        return new LogSegmentMetadata.LogSegmentMetadataBuilder(
+                    ledgerPath + "/" + inprogressZNodeName(logSegmentSeqNo),
+                    version,
+                    ledgerId,
+                    firstTxId)
+                .setLogSegmentSequenceNo(logSegmentSeqNo)
+                .build();
+    }
+
+    public static LogSegmentMetadata completedLogSegment(String ledgerPath,
+                                                         long ledgerId,
+                                                         long firstTxId,
+                                                         long lastTxId,
+                                                         int recordCount,
+                                                         long logSegmentSeqNo,
+                                                         long lastEntryId,
+                                                         long lastSlotId) {
+        return completedLogSegment(ledgerPath, ledgerId, firstTxId, lastTxId,
+                recordCount, logSegmentSeqNo, lastEntryId, lastSlotId,
+                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
+    }
+
+    public static LogSegmentMetadata completedLogSegment(String ledgerPath,
+                                                         long ledgerId,
+                                                         long firstTxId,
+                                                         long lastTxId,
+                                                         int recordCount,
+                                                         long logSegmentSeqNo,
+                                                         long lastEntryId,
+                                                         long lastSlotId,
+                                                         int version) {
+        LogSegmentMetadata metadata =
+                new LogSegmentMetadata.LogSegmentMetadataBuilder(
+                        ledgerPath + "/" + inprogressZNodeName(logSegmentSeqNo),
+                        version,
+                        ledgerId,
+                        firstTxId)
+                    .setInprogress(false)
+                    .setLogSegmentSequenceNo(logSegmentSeqNo)
+                    .build();
+        return metadata.completeLogSegment(ledgerPath + "/" + completedLedgerZNodeNameWithLogSegmentSequenceNumber(logSegmentSeqNo),
+                lastTxId, recordCount, lastEntryId, lastSlotId, firstTxId);
+    }
+
+    public static void generateCompletedLogSegments(DistributedLogManager manager, DistributedLogConfiguration conf,
+                                                    long numCompletedSegments, long segmentSize) throws Exception {
+        BKDistributedLogManager dlm = (BKDistributedLogManager) manager;
+        long txid = 1L;
+        for (long i = 0; i < numCompletedSegments; i++) {
+            BKSyncLogWriter writer = dlm.startLogSegmentNonPartitioned();
+            for (long j = 1; j <= segmentSize; j++) {
+                writer.write(DLMTestUtil.getLogRecordInstance(txid++));
+            }
+            writer.closeAndComplete();
+        }
+    }
+
+    public static long generateLogSegmentNonPartitioned(DistributedLogManager dlm, int controlEntries, int userEntries, long startTxid)
+            throws Exception {
+        return generateLogSegmentNonPartitioned(dlm, controlEntries, userEntries, startTxid, 1L);
+    }
+
+    public static long generateLogSegmentNonPartitioned(DistributedLogManager dlm, int controlEntries, int userEntries, long startTxid, long txidStep) throws Exception {
+        AsyncLogWriter out = dlm.startAsyncLogSegmentNonPartitioned();
+        long txid = startTxid;
+        for (int i = 0; i < controlEntries; ++i) {
+            LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
+            record.setControl();
+            Await.result(out.write(record));
+            txid += txidStep;
+        }
+        for (int i = 0; i < userEntries; ++i) {
+            LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
+            Await.result(out.write(record));
+            txid += txidStep;
+        }
+        Utils.close(out);
+        return txid - startTxid;
+    }
+
+    public static ZooKeeperClient getZooKeeperClient(BKDistributedLogManager dlm) {
+        return ((BKNamespaceDriver) dlm.getNamespaceDriver()).getWriterZKC();
+    }
+
+    public static BookKeeperClient getBookKeeperClient(BKDistributedLogManager dlm) {
+        return ((BKNamespaceDriver) dlm.getNamespaceDriver()).getReaderBKC();
+    }
+
+    public static void injectLogSegmentWithGivenLogSegmentSeqNo(DistributedLogManager manager, DistributedLogConfiguration conf,
+                                                                long logSegmentSeqNo, long startTxID, boolean writeEntries, long segmentSize,
+                                                                boolean completeLogSegment)
+            throws Exception {
+        BKDistributedLogManager dlm = (BKDistributedLogManager) manager;
+        BKLogWriteHandler writeHandler = dlm.createWriteHandler(false);
+        FutureUtils.result(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(),
+                conf.getAckQuorumSize(), BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes());
+        String inprogressZnodeName = writeHandler.inprogressZNodeName(lh.getId(), startTxID, logSegmentSeqNo);
+        String znodePath = writeHandler.inprogressZNode(lh.getId(), startTxID, logSegmentSeqNo);
+        int logSegmentMetadataVersion = conf.getDLLedgerMetadataLayoutVersion();
+        LogSegmentMetadata l =
+            new LogSegmentMetadata.LogSegmentMetadataBuilder(znodePath,
+                    logSegmentMetadataVersion, lh.getId(), startTxID)
+                .setLogSegmentSequenceNo(logSegmentSeqNo)
+                .setEnvelopeEntries(LogSegmentMetadata.supportsEnvelopedEntries(logSegmentMetadataVersion))
+                .build();
+        l.write(getZooKeeperClient(dlm));
+        writeHandler.maxTxId.update(Version.ANY, startTxID);
+        writeHandler.addLogSegmentToCache(inprogressZnodeName, l);
+        BKLogSegmentWriter writer = new BKLogSegmentWriter(
+                writeHandler.getFullyQualifiedName(),
+                inprogressZnodeName,
+                conf,
+                conf.getDLLedgerMetadataLayoutVersion(),
+                new BKLogSegmentEntryWriter(lh),
+                writeHandler.lock,
+                startTxID,
+                logSegmentSeqNo,
+                writeHandler.scheduler,
+                writeHandler.statsLogger,
+                writeHandler.statsLogger,
+                writeHandler.alertStatsLogger,
+                PermitLimiter.NULL_PERMIT_LIMITER,
+                new SettableFeatureProvider("", 0),
+                ConfUtils.getConstDynConf(conf));
+        if (writeEntries) {
+            long txid = startTxID;
+            for (long j = 1; j <= segmentSize; j++) {
+                writer.write(DLMTestUtil.getLogRecordInstance(txid++));
+            }
+            FutureUtils.result(writer.flushAndCommit());
+        }
+        if (completeLogSegment) {
+            FutureUtils.result(writeHandler.completeAndCloseLogSegment(writer));
+        }
+        FutureUtils.result(writeHandler.unlockHandler());
+    }
+
+    public static void injectLogSegmentWithLastDLSN(DistributedLogManager manager, DistributedLogConfiguration conf,
+                                                    long logSegmentSeqNo, long startTxID, long segmentSize,
+                                                    boolean recordWrongLastDLSN) throws Exception {
+        BKDistributedLogManager dlm = (BKDistributedLogManager) manager;
+        BKLogWriteHandler writeHandler = dlm.createWriteHandler(false);
+        FutureUtils.result(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(),
+                conf.getAckQuorumSize(), BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes());
+        String inprogressZnodeName = writeHandler.inprogressZNodeName(lh.getId(), startTxID, logSegmentSeqNo);
+        String znodePath = writeHandler.inprogressZNode(lh.getId(), startTxID, logSegmentSeqNo);
+        LogSegmentMetadata l =
+            new LogSegmentMetadata.LogSegmentMetadataBuilder(znodePath,
+                conf.getDLLedgerMetadataLayoutVersion(), lh.getId(), startTxID)
+            .setLogSegmentSequenceNo(logSegmentSeqNo)
+            .setInprogress(false)
+            .build();
+        l.write(getZooKeeperClient(dlm));
+        writeHandler.maxTxId.update(Version.ANY, startTxID);
+        writeHandler.addLogSegmentToCache(inprogressZnodeName, l);
+        BKLogSegmentWriter writer = new BKLogSegmentWriter(
+                writeHandler.getFullyQualifiedName(),
+                inprogressZnodeName,
+                conf,
+                conf.getDLLedgerMetadataLayoutVersion(),
+                new BKLogSegmentEntryWriter(lh),
+                writeHandler.lock,
+                startTxID,
+                logSegmentSeqNo,
+                writeHandler.scheduler,
+                writeHandler.statsLogger,
+                writeHandler.statsLogger,
+                writeHandler.alertStatsLogger,
+                PermitLimiter.NULL_PERMIT_LIMITER,
+                new SettableFeatureProvider("", 0),
+                ConfUtils.getConstDynConf(conf));
+        long txid = startTxID;
+        DLSN wrongDLSN = null;
+        for (long j = 1; j <= segmentSize; j++) {
+            DLSN dlsn = Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(txid++)));
+            if (j == (segmentSize - 1)) {
+                wrongDLSN = dlsn;
+            }
+        }
+        assertNotNull(wrongDLSN);
+        if (recordWrongLastDLSN) {
+            FutureUtils.result(writer.asyncClose());
+            writeHandler.completeAndCloseLogSegment(
+                    writeHandler.inprogressZNodeName(writer.getLogSegmentId(), writer.getStartTxId(), writer.getLogSegmentSequenceNumber()),
+                    writer.getLogSegmentSequenceNumber(),
+                    writer.getLogSegmentId(),
+                    writer.getStartTxId(),
+                    startTxID + segmentSize - 2,
+                    writer.getPositionWithinLogSegment() - 1,
+                    wrongDLSN.getEntryId(),
+                    wrongDLSN.getSlotId());
+        } else {
+            FutureUtils.result(writeHandler.completeAndCloseLogSegment(writer));
+        }
+        FutureUtils.result(writeHandler.unlockHandler());
+    }
+
+    public static void updateSegmentMetadata(ZooKeeperClient zkc, LogSegmentMetadata segment) throws Exception {
+        byte[] finalisedData = segment.getFinalisedData().getBytes(UTF_8);
+        zkc.get().setData(segment.getZkPath(), finalisedData, -1);
+    }
+
+    public static ServerConfiguration loadTestBkConf() {
+        ServerConfiguration conf = new ServerConfiguration();
+        ClassLoader classLoader = Thread.currentThread().getContextClassLoader();
+        URL confUrl = classLoader.getResource("bk_server.conf");
+        try {
+            if (null != confUrl) {
+                conf.loadConf(confUrl);
+                LOG.info("loaded bk_server.conf from resources");
+            }
+        } catch (org.apache.commons.configuration.ConfigurationException ex) {
+            LOG.warn("loading conf failed", ex);
+        }
+        conf.setAllowLoopback(true);
+        return conf;
+    }
+
+    public static <T> void validateFutureFailed(Future<T> future, Class exClass) {
+        try {
+            Await.result(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 {
+        try {
+            return Await.result(future, Duration.fromSeconds(10));
+        } 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
new file mode 100644
index 0000000..2dbef02
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/NonBlockingReadsTestUtil.java
@@ -0,0 +1,145 @@
+/**
+ * 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 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.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.*;
+
+/**
+ * Utils for non blocking reads tests
+ */
+class NonBlockingReadsTestUtil {
+
+    static final Logger LOG = LoggerFactory.getLogger(NonBlockingReadsTestUtil.class);
+
+    static final long DEFAULT_SEGMENT_SIZE = 1000;
+
+    static void readNonBlocking(DistributedLogManager dlm, boolean forceStall) throws Exception {
+        readNonBlocking(dlm, forceStall, DEFAULT_SEGMENT_SIZE, false);
+    }
+
+    static void readNonBlocking(DistributedLogManager dlm,
+                                boolean forceStall,
+                                long segmentSize,
+                                boolean waitForIdle) throws Exception {
+        BKSyncLogReader reader = null;
+        try {
+            reader = (BKSyncLogReader) dlm.getInputStream(1);
+        } catch (LogNotFoundException lnfe) {
+        }
+        while (null == reader) {
+            TimeUnit.MILLISECONDS.sleep(20);
+            try {
+                reader = (BKSyncLogReader) dlm.getInputStream(1);
+            } catch (LogNotFoundException lnfe) {
+            } catch (LogEmptyException lee) {
+            } catch (IOException ioe) {
+                LOG.error("Failed to open reader reading from {}", dlm.getStreamName());
+                throw ioe;
+            }
+        }
+        try {
+            LOG.info("Created reader reading from {}", dlm.getStreamName());
+            if (forceStall) {
+                reader.getReadHandler().disableReadAheadLogSegmentsNotification();
+            }
+
+            long numTrans = 0;
+            long lastTxId = -1;
+
+            boolean exceptionEncountered = false;
+            try {
+                while (true) {
+                    LogRecordWithDLSN record = reader.readNext(true);
+                    if (null != record) {
+                        DLMTestUtil.verifyLogRecord(record);
+                        assertTrue(lastTxId < record.getTransactionId());
+                        assertEquals(record.getTransactionId() - 1, record.getSequenceId());
+                        lastTxId = record.getTransactionId();
+                        numTrans++;
+                        continue;
+                    }
+
+                    if (numTrans >= (3 * segmentSize)) {
+                        if (waitForIdle) {
+                            while (true) {
+                                reader.readNext(true);
+                                TimeUnit.MILLISECONDS.sleep(10);
+                            }
+                        }
+                        break;
+                    }
+
+                    TimeUnit.MILLISECONDS.sleep(2);
+                }
+            } catch (LogReadException readexc) {
+                exceptionEncountered = true;
+            } catch (LogNotFoundException exc) {
+                exceptionEncountered = true;
+            }
+            assertFalse(exceptionEncountered);
+        } finally {
+            reader.close();
+        }
+    }
+
+    static void writeRecordsForNonBlockingReads(DistributedLogConfiguration conf,
+                                         DistributedLogManager dlm,
+                                         boolean recover)
+            throws Exception {
+        writeRecordsForNonBlockingReads(conf, dlm, recover, DEFAULT_SEGMENT_SIZE);
+    }
+
+    static void writeRecordsForNonBlockingReads(DistributedLogConfiguration conf,
+                                         DistributedLogManager dlm,
+                                         boolean recover,
+                                         long segmentSize)
+            throws Exception {
+        long txId = 1;
+        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++)));
+            }
+            if (recover) {
+                FutureUtils.result(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());
+                LOG.debug("Recovered Segments");
+            } else {
+                FutureUtils.result(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
new file mode 100644
index 0000000..922d89e
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamReader.java
@@ -0,0 +1,207 @@
+/**
+ * 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 java.io.ByteArrayInputStream;
+import java.net.URI;
+import java.util.Arrays;
+
+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;
+
+import static org.junit.Assert.*;
+
+public class TestAppendOnlyStreamReader extends TestDistributedLogBase {
+    static final Logger LOG = LoggerFactory.getLogger(TestAppendOnlyStreamReader.class);
+
+    @Rule
+    public TestName testNames = new TestName();
+
+    // Simple test subroutine writes some records, reads some back, skips ahead, skips back.
+    public void skipForwardThenSkipBack(String name, DistributedLogConfiguration conf) throws Exception {
+        DistributedLogManager dlmwrite = createNewDLM(conf, name);
+        DistributedLogManager dlmreader = createNewDLM(conf, name);
+
+        long txid = 1;
+        AppendOnlyStreamWriter writer = dlmwrite.getAppendOnlyStreamWriter();
+        writer.write(DLMTestUtil.repeatString("abc", 5).getBytes());
+        writer.write(DLMTestUtil.repeatString("abc", 5).getBytes());
+        writer.write(DLMTestUtil.repeatString("def", 5).getBytes());
+        writer.write(DLMTestUtil.repeatString("def", 5).getBytes());
+        writer.write(DLMTestUtil.repeatString("ghi", 5).getBytes());
+        writer.write(DLMTestUtil.repeatString("ghi", 5).getBytes());
+        writer.force(false);
+        writer.close();
+
+        AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
+        byte[] bytesIn = new byte[30];
+
+        byte[] bytes1 = DLMTestUtil.repeatString("abc", 10).getBytes();
+        byte[] bytes2 = DLMTestUtil.repeatString("def", 10).getBytes();
+        byte[] bytes3 = DLMTestUtil.repeatString("ghi", 10).getBytes();
+
+        int read = reader.read(bytesIn, 0, 30);
+        assertEquals(30, read);
+        assertTrue(Arrays.equals(bytes1, bytesIn));
+
+        reader.skipTo(60);
+        read = reader.read(bytesIn, 0, 30);
+        assertEquals(30, read);
+        assertTrue(Arrays.equals(bytes3, bytesIn));
+
+        reader.skipTo(30);
+        read = reader.read(bytesIn, 0, 30);
+        assertEquals(30, read);
+        assertTrue(Arrays.equals(bytes2, bytesIn));
+    }
+
+    @Test(timeout = 60000)
+    public void testSkipToSkipsBytesWithImmediateFlush() throws Exception {
+        String name = testNames.getMethodName();
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setOutputBufferSize(0);
+
+        skipForwardThenSkipBack(name, confLocal);
+    }
+
+    @Test(timeout = 60000)
+    public void testSkipToSkipsBytesWithLargerLogRecords() throws Exception {
+        String name = testNames.getMethodName();
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(1024*100);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(1000*60);
+
+        skipForwardThenSkipBack(name, confLocal);
+    }
+
+    @Test(timeout = 60000)
+    public void testSkipToSkipsBytesUntilEndOfStream() throws Exception {
+        String name = testNames.getMethodName();
+
+        DistributedLogManager dlmwrite = createNewDLM(conf, name);
+        DistributedLogManager dlmreader = createNewDLM(conf, name);
+
+        long txid = 1;
+        AppendOnlyStreamWriter writer = dlmwrite.getAppendOnlyStreamWriter();
+        writer.write(DLMTestUtil.repeatString("abc", 5).getBytes());
+        writer.markEndOfStream();
+        writer.force(false);
+        writer.close();
+
+        AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
+        byte[] bytesIn = new byte[9];
+
+        int read = reader.read(bytesIn, 0, 9);
+        assertEquals(9, read);
+        assertTrue(Arrays.equals(DLMTestUtil.repeatString("abc", 3).getBytes(), bytesIn));
+
+        assertTrue(reader.skipTo(15));
+
+        try {
+            read = reader.read(bytesIn, 0, 1);
+            fail("Should have thrown");
+        } catch (EndOfStreamException ex) {
+        }
+
+        assertTrue(reader.skipTo(0));
+
+        try {
+            reader.skipTo(16);
+            fail("Should have thrown");
+        } catch (EndOfStreamException ex) {
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testSkipToreturnsFalseIfPositionDoesNotExistYetForUnSealedStream() throws Exception {
+        String name = testNames.getMethodName();
+
+        DistributedLogManager dlmwrite = createNewDLM(conf, name);
+        DistributedLogManager dlmreader = createNewDLM(conf, name);
+
+        long txid = 1;
+        AppendOnlyStreamWriter writer = dlmwrite.getAppendOnlyStreamWriter();
+        writer.write(DLMTestUtil.repeatString("abc", 5).getBytes());
+        writer.close();
+
+        final AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
+        byte[] bytesIn = new byte[9];
+
+        int read = reader.read(bytesIn, 0, 9);
+        assertEquals(9, read);
+        assertTrue(Arrays.equals(DLMTestUtil.repeatString("abc", 3).getBytes(), bytesIn));
+
+        assertFalse(reader.skipTo(16));
+        assertFalse(reader.skipTo(16));
+
+        AppendOnlyStreamWriter writer2 = dlmwrite.getAppendOnlyStreamWriter();
+        writer2.write(DLMTestUtil.repeatString("abc", 5).getBytes());
+        writer2.close();
+
+        assertTrue(reader.skipTo(16));
+
+        byte[] bytesIn2 = new byte[5];
+        read = reader.read(bytesIn2, 0, 5);
+        assertEquals(5, read);
+        assertTrue(Arrays.equals("bcabc".getBytes(), bytesIn2));
+    }
+
+    @Test(timeout = 60000)
+    public void testSkipToForNoPositionChange() throws Exception {
+        String name = testNames.getMethodName();
+
+        DistributedLogManager dlmwrite = createNewDLM(conf, name);
+        DistributedLogManager dlmreader = createNewDLM(conf, name);
+
+        long txid = 1;
+        AppendOnlyStreamWriter writer = dlmwrite.getAppendOnlyStreamWriter();
+        writer.write(DLMTestUtil.repeatString("abc", 5).getBytes());
+        writer.close();
+
+        final AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
+
+        assertTrue(reader.skipTo(0));
+
+        byte[] bytesIn = new byte[4];
+        int read = reader.read(bytesIn, 0, 4);
+        assertEquals(4, read);
+        assertEquals(new String("abca"), new String(bytesIn));
+
+        assertTrue(reader.skipTo(reader.position()));
+
+        assertTrue(reader.skipTo(1));
+
+        read = reader.read(bytesIn, 0, 4);
+        assertEquals(4, read);
+        assertEquals(new String("bcab"), new String(bytesIn));
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java
new file mode 100644
index 0000000..d095af1
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAppendOnlyStreamWriter.java
@@ -0,0 +1,337 @@
+/**
+ * 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 java.io.ByteArrayInputStream;
+import java.net.URI;
+
+import org.apache.distributedlog.exceptions.BKTransmitException;
+import org.apache.distributedlog.util.FutureUtils;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+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;
+
+import static org.junit.Assert.*;
+
+public class TestAppendOnlyStreamWriter extends TestDistributedLogBase {
+    static final Logger LOG = LoggerFactory.getLogger(TestAppendOnlyStreamWriter.class);
+
+    @Rule
+    public TestName testNames = new TestName();
+
+    @Test(timeout = 60000)
+    public void testBasicReadAndWriteBehavior() throws Exception {
+        String name = testNames.getMethodName();
+        DistributedLogManager dlmwrite = createNewDLM(conf, name);
+        DistributedLogManager dlmreader = createNewDLM(conf, name);
+        byte[] byteStream = DLMTestUtil.repeatString("abc", 51).getBytes();
+
+        long txid = 1;
+        AppendOnlyStreamWriter writer = dlmwrite.getAppendOnlyStreamWriter();
+        writer.write(DLMTestUtil.repeatString("abc", 11).getBytes());
+        writer.write(DLMTestUtil.repeatString("abc", 40).getBytes());
+        writer.force(false);
+        writer.close();
+        AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
+
+        byte[] bytesIn = new byte[byteStream.length];
+        int read = reader.read(bytesIn, 0, 23);
+        assertEquals(23, read);
+        read = reader.read(bytesIn, 23, 31);
+        assertEquals(read, 31);
+        byte[] bytesInTemp = new byte[byteStream.length];
+        read = reader.read(bytesInTemp, 0, byteStream.length);
+        assertEquals(read, byteStream.length - 23 - 31);
+        read = new ByteArrayInputStream(bytesInTemp).read(bytesIn, 23 + 31, byteStream.length - 23 - 31);
+        assertEquals(read, byteStream.length - 23 - 31);
+        assertArrayEquals(bytesIn, byteStream);
+        reader.close();
+        dlmreader.close();
+        dlmwrite.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteFutureDoesNotCompleteUntilWritePersisted() throws Exception {
+        String name = testNames.getMethodName();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.setPeriodicFlushFrequencyMilliSeconds(Integer.MAX_VALUE);
+        conf.setImmediateFlushEnabled(false);
+
+        DistributedLogManager dlmwriter = createNewDLM(conf, name);
+        DistributedLogManager dlmreader = createNewDLM(conf, name);
+        byte[] byteStream = DLMTestUtil.repeatString("abc", 51).getBytes();
+
+        // Can't reliably test the future is not completed until fsync is called, since writer.force may just
+        // 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());
+
+        // The real problem is the fsync completes before writes are submitted, so it never takes effect.
+        Thread.sleep(1000);
+        assertFalse(dlsnFuture.isDefined());
+        writer.force(false);
+        // Must not throw.
+        Await.result(dlsnFuture, Duration.fromSeconds(5));
+        writer.close();
+        dlmwriter.close();
+
+        AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
+        byte[] bytesIn = new byte[byteStream.length];
+        int read = reader.read(bytesIn, 0, 31);
+        assertEquals(31, read);
+        reader.close();
+        dlmreader.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testPositionUpdatesOnlyAfterWriteCompletion() throws Exception {
+        String name = testNames.getMethodName();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.setPeriodicFlushFrequencyMilliSeconds(10*1000);
+        conf.setImmediateFlushEnabled(false);
+
+        DistributedLogManager dlmwriter = createNewDLM(conf, name);
+        DistributedLogManager dlmreader = createNewDLM(conf, name);
+        byte[] byteStream = DLMTestUtil.repeatString("abc", 11).getBytes();
+
+        // Can't reliably test the future is not completed until fsync is called, since writer.force may just
+        // 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);
+        Thread.sleep(100);
+
+        // Write hasn't been persisted, position better not be updated.
+        assertFalse(dlsnFuture.isDefined());
+        assertEquals(0, writer.position());
+        writer.force(false);
+        // Position guaranteed to be accurate after writer.force().
+        assertEquals(byteStream.length, writer.position());
+
+        // Close writer.
+        writer.close();
+        dlmwriter.close();
+
+        // Make sure we can read it.
+        AppendOnlyStreamReader reader = dlmreader.getAppendOnlyStreamReader();
+        byte[] bytesIn = new byte[byteStream.length];
+        int read = reader.read(bytesIn, 0, byteStream.length);
+        assertEquals(byteStream.length, read);
+        assertEquals(byteStream.length, reader.position());
+        reader.close();
+        dlmreader.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testPositionDoesntUpdateBeforeWriteCompletion() throws Exception {
+        String name = testNames.getMethodName();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+
+        // Long flush time, but we don't wait for it.
+        conf.setPeriodicFlushFrequencyMilliSeconds(100*1000);
+        conf.setImmediateFlushEnabled(false);
+        conf.setOutputBufferSize(1024*1024);
+
+        DistributedLogManager dlmwriter = createNewDLM(conf, name);
+        byte[] byteStream = DLMTestUtil.repeatString("abc", 11).getBytes();
+
+        AppendOnlyStreamWriter writer = dlmwriter.getAppendOnlyStreamWriter();
+        assertEquals(0, writer.position());
+
+        // 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);
+        assertEquals(0, writer.position());
+
+        writer.close();
+        dlmwriter.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testPositionUpdatesOnlyAfterWriteCompletionWithoutFsync() throws Exception {
+        String name = testNames.getMethodName();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.setPeriodicFlushFrequencyMilliSeconds(1*1000);
+        conf.setImmediateFlushEnabled(false);
+        conf.setOutputBufferSize(1024*1024);
+
+        DistributedLogManager dlmwriter = createNewDLM(conf, name);
+        byte[] byteStream = DLMTestUtil.repeatString("abc", 11).getBytes();
+
+        AppendOnlyStreamWriter writer = dlmwriter.getAppendOnlyStreamWriter();
+        assertEquals(0, writer.position());
+
+        Await.result(writer.write(byteStream));
+        assertEquals(33, writer.position());
+
+        writer.close();
+        dlmwriter.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testWriterStartsAtTxidZeroForEmptyStream() throws Exception {
+        String name = testNames.getMethodName();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.setImmediateFlushEnabled(true);
+        conf.setOutputBufferSize(1024);
+        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
+
+        URI uri = createDLMURI("/" + name);
+        FutureUtils.result(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));
+
+        writer.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testOffsetGapAfterSegmentWriterFailure() throws Exception {
+        String name = testNames.getMethodName();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.setImmediateFlushEnabled(false);
+        conf.setPeriodicFlushFrequencyMilliSeconds(60*1000);
+        conf.setOutputBufferSize(1024*1024);
+        conf.setLogSegmentSequenceNumberValidationEnabled(false);
+
+        final int WRITE_LEN = 5;
+        final int SECTION_WRITES = 10;
+        long read = writeRecordsAndReadThemBackAfterInjectingAFailedTransmit(conf, name, WRITE_LEN, SECTION_WRITES);
+        assertEquals((2*SECTION_WRITES + 1)*WRITE_LEN, read);
+    }
+
+    @Test(timeout = 60000)
+    public void testNoOffsetGapAfterSegmentWriterFailure() throws Exception {
+        String name = testNames.getMethodName();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.setImmediateFlushEnabled(false);
+        conf.setPeriodicFlushFrequencyMilliSeconds(60*1000);
+        conf.setOutputBufferSize(1024*1024);
+        conf.setDisableRollingOnLogSegmentError(true);
+
+        final int WRITE_LEN = 5;
+        final int SECTION_WRITES = 10;
+
+        try {
+            writeRecordsAndReadThemBackAfterInjectingAFailedTransmit(conf, name, WRITE_LEN, SECTION_WRITES);
+            fail("should have thrown");
+        } catch (BKTransmitException ex) {
+            ;
+        }
+
+        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
+        long length = dlm.getLastTxId();
+        long read = read(dlm, length);
+        assertEquals(length, read);
+    }
+
+    long writeRecordsAndReadThemBackAfterInjectingAFailedTransmit(
+            DistributedLogConfiguration conf,
+            String name,
+            int writeLen,
+            int sectionWrites)
+            throws Exception {
+
+        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
+
+        URI uri = createDLMURI("/" + name);
+        FutureUtils.result(dlm.getWriterMetadataStore().getLog(uri, name, true, true));
+
+        // Log exists but is empty, better not throw.
+        AppendOnlyStreamWriter writer = dlm.getAppendOnlyStreamWriter();
+        byte[] byteStream = DLMTestUtil.repeatString("A", writeLen).getBytes();
+
+        // Log a hundred entries. Offset is advanced accordingly.
+        for (int i = 0; i < sectionWrites; i++) {
+            writer.write(byteStream);
+        }
+        writer.force(false);
+
+        long read = read(dlm, 1*sectionWrites*writeLen);
+        assertEquals(1*sectionWrites*writeLen, read);
+
+        // Now write another 100, but trigger failure during transmit.
+        for (int i = 0; i < sectionWrites; i++) {
+            writer.write(byteStream);
+        }
+
+        try {
+            FailpointUtils.setFailpoint(
+                FailpointUtils.FailPointName.FP_TransmitFailGetBuffer,
+                FailpointUtils.FailPointActions.FailPointAction_Throw);
+
+            writer.force(false);
+            fail("should have thown ⊙﹏⊙");
+        } catch (WriteException we) {
+            ;
+        } finally {
+            FailpointUtils.removeFailpoint(
+                FailpointUtils.FailPointName.FP_TransmitFailGetBuffer);
+        }
+
+        // This actually fails because we try to close an errored out stream.
+        writer.write(byteStream);
+
+        // Writing another 100 triggers offset gap.
+        for (int i = 0; i < sectionWrites; i++) {
+            writer.write(byteStream);
+        }
+
+        writer.force(false);
+        writer.markEndOfStream();
+        writer.close();
+
+        long length = dlm.getLastTxId();
+        assertEquals(3*sectionWrites*writeLen+5, length);
+        read = read(dlm, length);
+        dlm.close();
+        return read;
+    }
+
+    long read(DistributedLogManager dlm, long n) throws Exception {
+        AppendOnlyStreamReader reader = dlm.getAppendOnlyStreamReader();
+        byte[] bytesIn = new byte[1];
+        long offset = 0;
+        try {
+            while (offset < n) {
+                int read = reader.read(bytesIn, 0, 1);
+                offset += read;
+            }
+        } catch (EndOfStreamException ex) {
+            LOG.info("Caught ex", ex);
+        } finally {
+            reader.close();
+        }
+        return offset;
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.java
new file mode 100644
index 0000000..139d935
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncBulkWrite.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 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.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.apache.distributedlog.DLMTestUtil.validateFutureFailed;
+import static org.apache.distributedlog.DLMTestUtil.validateFutureSucceededAndGetResult;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORDSET_SIZE;
+import static org.junit.Assert.*;
+
+/**
+ * Test cases for bulk writes.
+ */
+public class TestAsyncBulkWrite extends TestDistributedLogBase {
+
+    static final Logger LOG = LoggerFactory.getLogger(TestAsyncBulkWrite.class);
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    protected final DistributedLogConfiguration testConf;
+
+    public TestAsyncBulkWrite() {
+        this.testConf = new DistributedLogConfiguration();
+        this.testConf.addConfiguration(conf);
+        this.testConf.setReaderIdleErrorThresholdMillis(1200000);
+    }
+
+    /**
+     * Test Case for partial failure in a bulk write.
+     *
+     * Write a batch: 10 good records + 1 too large record + 10 good records.
+     *
+     * Expected: first 10 good records succeed, the too-large-record will be rejected, while
+     *           the last 10 good records will be cancelled because their previous write is rejected.
+     */
+    @Test(timeout = 60000)
+    public void testAsyncBulkWritePartialFailureBufferFailure() throws Exception {
+        String name = "distrlog-testAsyncBulkWritePartialFailure";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(1024);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+
+        final int goodRecs = 10;
+
+        // Generate records: 10 good records, 1 too large record, 10 good records
+        final List<LogRecord> records = DLMTestUtil.getLargeLogRecordInstanceList(1, goodRecs);
+        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);
+
+        // One future returned for each write.
+        assertEquals(2*goodRecs + 1, results.size());
+
+        // First goodRecs are good.
+        for (int i = 0; i < goodRecs; i++) {
+            DLSN dlsn = validateFutureSucceededAndGetResult(results.get(i));
+        }
+
+        // First failure is log rec too big.
+        validateFutureFailed(results.get(goodRecs), LogRecordTooLongException.class);
+
+        // Rest are WriteCancelledException.
+        for (int i = goodRecs+1; i < 2*goodRecs+1; i++) {
+            validateFutureFailed(results.get(i), WriteCancelledException.class);
+        }
+
+        writer.closeAndComplete();
+        dlm.close();
+    }
+
+    /**
+     * Test Case for a total failure in a bulk write.
+     *
+     * Write 100 records as a batch. Inject failure on transmit and all records should be failed.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testAsyncBulkWriteTotalFailureTransmitFailure() throws Exception {
+        String name = "distrlog-testAsyncBulkWriteTotalFailureDueToTransmitFailure";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(1024);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+
+        final int batchSize = 100;
+        FailpointUtils.setFailpoint(
+                FailpointUtils.FailPointName.FP_TransmitComplete,
+                FailpointUtils.FailPointActions.FailPointAction_Default
+        );
+        try {
+            // Since we don't hit MAX_TRANSMISSION_SIZE, the failure is triggered on final flush, which
+            // will enqueue cancel promises task to the ordered future pool.
+            checkAllSubmittedButFailed(writer, batchSize, 1024, 1);
+        } finally {
+            FailpointUtils.removeFailpoint(
+                FailpointUtils.FailPointName.FP_TransmitComplete
+            );
+        }
+
+        writer.abort();
+        dlm.close();
+    }
+
+    /**
+     * Test Case: There is no log segment rolling when there is partial failure in async bulk write.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testAsyncBulkWriteNoLedgerRollWithPartialFailures() throws Exception {
+        String name = "distrlog-testAsyncBulkWriteNoLedgerRollWithPartialFailures";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(1024);
+        confLocal.setMaxLogSegmentBytes(1024);
+        confLocal.setLogSegmentRollingIntervalMinutes(0);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+
+        // 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 = 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;
+        records = new ArrayList<LogRecord>(2);
+        records.add(DLMTestUtil.getLogRecordInstance(txid++, 2048));
+        records.add(DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1));
+        futureResults = writer.writeBulk(records);
+        results = validateFutureSucceededAndGetResult(futureResults);
+        result = results.get(0);
+        dlsn = validateFutureSucceededAndGetResult(result);
+        assertEquals(1, dlsn.getLogSegmentSequenceNo());
+
+        // Now writer is in a bad state.
+        records = new ArrayList<LogRecord>(1);
+        records.add(DLMTestUtil.getLogRecordInstance(txid++, 2048));
+        futureResults = writer.writeBulk(records);
+        validateFutureFailed(futureResults, WriteException.class);
+
+        writer.closeAndComplete();
+        dlm.close();
+    }
+
+    /**
+     * Test Case: A large write batch will span records into multiple entries and ledgers.
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testSimpleAsyncBulkWriteSpanningEntryAndLedger() throws Exception {
+        String name = "distrlog-testSimpleAsyncBulkWriteSpanningEntryAndLedger";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(1024);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+
+        int batchSize = 100;
+        int recSize = 1024;
+
+        // First entry.
+        long ledgerIndex = 1;
+        long entryIndex = 0;
+        long slotIndex = 0;
+        long txIndex = 1;
+        checkAllSucceeded(writer, batchSize, recSize, ledgerIndex, entryIndex, slotIndex, txIndex);
+
+        // New entry.
+        entryIndex++;
+        slotIndex = 0;
+        txIndex += batchSize;
+        checkAllSucceeded(writer, batchSize, recSize, ledgerIndex, entryIndex, slotIndex, txIndex);
+
+        // Roll ledger.
+        ledgerIndex++;
+        entryIndex = 0;
+        slotIndex = 0;
+        txIndex += batchSize;
+        writer.closeAndComplete();
+        writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+        checkAllSucceeded(writer, batchSize, recSize, ledgerIndex, entryIndex, slotIndex, txIndex);
+
+        writer.closeAndComplete();
+        dlm.close();
+    }
+
+    /**
+     * Test Case: A large write batch will span multiple packets.
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testAsyncBulkWriteSpanningPackets() throws Exception {
+        String name = "distrlog-testAsyncBulkWriteSpanningPackets";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(1024);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+
+        // First entry.
+        int numTransmissions = 4;
+        int recSize = 10*1024;
+        int batchSize = (numTransmissions*MAX_LOGRECORDSET_SIZE+1)/recSize;
+        long ledgerIndex = 1;
+        long entryIndex = 0;
+        long slotIndex = 0;
+        long txIndex = 1;
+        DLSN dlsn = checkAllSucceeded(writer, batchSize, recSize, ledgerIndex, entryIndex, slotIndex, txIndex);
+        assertEquals(4, dlsn.getEntryId());
+        assertEquals(1, dlsn.getLogSegmentSequenceNo());
+
+        writer.closeAndComplete();
+        dlm.close();
+    }
+
+    /**
+     * Test Case: Test Transmit Failures when a large write batch spans multiple packets.
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testAsyncBulkWriteSpanningPacketsWithTransmitFailure() throws Exception {
+        String name = "distrlog-testAsyncBulkWriteSpanningPacketsWithTransmitFailure";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(1024);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+
+        // First entry.
+        int numTransmissions = 4;
+        int recSize = 10*1024;
+        int batchSize = (numTransmissions*MAX_LOGRECORDSET_SIZE+1)/recSize;
+        long ledgerIndex = 1;
+        long entryIndex = 0;
+        long slotIndex = 0;
+        long txIndex = 1;
+
+        DLSN dlsn = checkAllSucceeded(writer, batchSize, recSize, ledgerIndex, entryIndex, slotIndex, txIndex);
+        assertEquals(4, dlsn.getEntryId());
+        assertEquals(1, dlsn.getLogSegmentSequenceNo());
+
+        FailpointUtils.setFailpoint(
+            FailpointUtils.FailPointName.FP_TransmitComplete,
+            FailpointUtils.FailPointActions.FailPointAction_Default
+        );
+
+        try {
+            checkAllSubmittedButFailed(writer, batchSize, recSize, 1);
+        } finally {
+            FailpointUtils.removeFailpoint(
+                FailpointUtils.FailPointName.FP_TransmitComplete
+            );
+        }
+        writer.abort();
+        dlm.close();
+    }
+
+    private DLSN checkAllSucceeded(BKAsyncLogWriter writer,
+                                   int batchSize,
+                                   int recSize,
+                                   long ledgerIndex,
+                                   long entryIndex,
+                                   long slotIndex,
+                                   long txIndex) throws Exception {
+
+        List<LogRecord> records = DLMTestUtil.getLogRecordInstanceList(txIndex, batchSize, recSize);
+        Future<List<Future<DLSN>>> futureResults = writer.writeBulk(records);
+        assertNotNull(futureResults);
+        List<Future<DLSN>> results = Await.result(futureResults, Duration.fromSeconds(10));
+        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));
+            lastDlsn = dlsn;
+
+            // If we cross a transmission boundary, slot id gets reset.
+            if (dlsn.getEntryId() > prevEntryId) {
+                slotIndex = 0;
+            }
+            assertEquals(ledgerIndex, dlsn.getLogSegmentSequenceNo());
+            assertEquals(slotIndex, dlsn.getSlotId());
+            slotIndex++;
+            prevEntryId = dlsn.getEntryId();
+        }
+        return lastDlsn;
+    }
+
+    private void checkAllSubmittedButFailed(BKAsyncLogWriter writer,
+                                            int batchSize,
+                                            int recSize,
+                                            long txIndex) throws Exception {
+
+        List<LogRecord> records = DLMTestUtil.getLogRecordInstanceList(txIndex, batchSize, recSize);
+        Future<List<Future<DLSN>>> futureResults = writer.writeBulk(records);
+        assertNotNull(futureResults);
+        List<Future<DLSN>> results = Await.result(futureResults, Duration.fromSeconds(10));
+        assertNotNull(results);
+        assertEquals(results.size(), records.size());
+        for (Future<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
new file mode 100644
index 0000000..adceaf9
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderLock.java
@@ -0,0 +1,607 @@
+/**
+ * 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 java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+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.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.namespace.NamespaceDriver;
+import org.apache.distributedlog.subscription.SubscriptionsStore;
+import org.apache.distributedlog.util.FutureUtils;
+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;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+
+public class TestAsyncReaderLock extends TestDistributedLogBase {
+    static final Logger LOG = LoggerFactory.getLogger(TestAsyncReaderLock.class);
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    void assertAcquiredFlagsSet(boolean[] acquiredFlags, int endIndex) {
+        for (int i = 0; i < endIndex; i++) {
+            assertTrue("reader " + i + " should have acquired lock", acquiredFlags[i]);
+        }
+        for (int i = endIndex; i < acquiredFlags.length; i++) {
+            assertFalse("reader " + i + " should not have acquired lock", acquiredFlags[i]);
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testReaderLockIfLockPathDoesntExist() throws Exception {
+        final String name = runtime.getMethodName();
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+        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());
+        assertEquals(1L, record.getTransactionId());
+        assertEquals(0L, record.getSequenceId());
+        DLMTestUtil.verifyLogRecord(record);
+
+        String readLockPath = reader1.readHandler.getReadLockPath();
+        Utils.close(reader1);
+
+        // 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());
+        assertEquals(1L, record.getTransactionId());
+        assertEquals(0L, record.getSequenceId());
+        DLMTestUtil.verifyLogRecord(record);
+    }
+
+    @Test(timeout = 60000)
+    public void testReaderLockCloseInAcquireCallback() throws Exception {
+        final String name = runtime.getMethodName();
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+        writer.write(DLMTestUtil.getLogRecordInstance(1L));
+        writer.closeAndComplete();
+
+        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) {
+                        latch.countDown();
+                        return null;
+                    }
+                });
+            }
+        });
+
+        latch.await();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReaderLockBackgroundReaderLockAcquire() throws Exception {
+        final String name = runtime.getMethodName();
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+        writer.write(DLMTestUtil.getLogRecordInstance(1L));
+        writer.closeAndComplete();
+
+        Future<AsyncLogReader> futureReader1 = dlm.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        AsyncLogReader reader1 = Await.result(futureReader1);
+        reader1.readNext();
+
+        final CountDownLatch acquiredLatch = new CountDownLatch(1);
+        final AtomicBoolean acquired = new AtomicBoolean(false);
+        Thread acquireThread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                Future<AsyncLogReader> futureReader2 = null;
+                DistributedLogManager dlm2 = null;
+                try {
+                    dlm2 = createNewDLM(conf, name);
+                    futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+                    AsyncLogReader reader2 = Await.result(futureReader2);
+                    acquired.set(true);
+                    acquiredLatch.countDown();
+                } catch (Exception ex) {
+                    fail("shouldn't reach here");
+                } finally {
+                    try {
+                        dlm2.close();
+                    } catch (Exception ex) {
+                        fail("shouldn't reach here");
+                    }
+                }
+            }
+        }, "acquire-thread");
+        acquireThread.start();
+
+        Thread.sleep(1000);
+        assertEquals(false, acquired.get());
+        Utils.close(reader1);
+
+        acquiredLatch.await();
+        assertEquals(true, acquired.get());
+        dlm.close();
+    }
+
+    int countDefined(ArrayList<Future<AsyncLogReader>> readers) {
+        int done = 0;
+        for (Future<AsyncLogReader> futureReader : readers) {
+            if (futureReader.isDefined()) {
+                done++;
+            }
+        }
+        return done;
+    }
+
+    @Test(timeout = 60000)
+    public void testReaderLockManyLocks() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+        writer.write(DLMTestUtil.getLogRecordInstance(1L));
+        writer.write(DLMTestUtil.getLogRecordInstance(2L));
+        writer.closeAndComplete();
+
+        int count = 5;
+        final CountDownLatch acquiredLatch = new CountDownLatch(count);
+        final ArrayList<Future<AsyncLogReader>> readers = new ArrayList<Future<AsyncLogReader>>(count);
+        for (int i = 0; i < count; i++) {
+            readers.add(null);
+        }
+        final DistributedLogManager[] dlms = new DistributedLogManager[count];
+        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>() {
+                @Override
+                public void onSuccess(AsyncLogReader reader) {
+                    acquiredLatch.countDown();
+                    reader.asyncClose();
+                }
+                @Override
+                public void onFailure(Throwable cause) {
+                    fail("acquire shouldnt have failed");
+                }
+            });
+        }
+
+        acquiredLatch.await();
+        for (int i = 0; i < count; i++) {
+            dlms[i].close();
+        }
+
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReaderLockDlmClosed() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogManager dlm0 = createNewDLM(conf, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
+        writer.write(DLMTestUtil.getLogRecordInstance(1L));
+        writer.write(DLMTestUtil.getLogRecordInstance(2L));
+        writer.closeAndComplete();
+
+        DistributedLogManager dlm1 = createNewDLM(conf, name);
+        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        AsyncLogReader reader1 = Await.result(futureReader1);
+
+        BKDistributedLogManager dlm2 = (BKDistributedLogManager) createNewDLM(conf, name);
+        Future<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+
+        dlm2.close();
+        try {
+            Await.result(futureReader2);
+            fail("should have thrown exception!");
+        } catch (LockClosedException ex) {
+        } catch (LockCancelledException ex) {
+        }
+
+        Utils.close(reader1);
+        dlm0.close();
+        dlm1.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReaderLockSessionExpires() throws Exception {
+        String name = runtime.getMethodName();
+        URI uri = createDLMURI("/" + name);
+        ensureURICreated(uri);
+        DistributedLogNamespace ns0 = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build();
+        DistributedLogManager dlm0 = ns0.openLog(name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
+        writer.write(DLMTestUtil.getLogRecordInstance(1L));
+        writer.write(DLMTestUtil.getLogRecordInstance(2L));
+        writer.closeAndComplete();
+
+        DistributedLogNamespace ns1 = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build();
+        DistributedLogManager dlm1 = ns1.openLog(name);
+        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        AsyncLogReader reader1 = Await.result(futureReader1);
+        ZooKeeperClientUtils.expireSession(((BKNamespaceDriver) ns1.getNamespaceDriver()).getWriterZKC(), zkServers, 1000);
+
+        // The result of expireSession is somewhat non-deterministic with this lock.
+        // It may fail with LockingException or it may succesfully reacquire, so for
+        // the moment rather than make it deterministic we accept either result.
+        boolean success = false;
+        try {
+            Await.result(reader1.readNext());
+            success = true;
+        } catch (LockingException ex) {
+        }
+        if (success) {
+            Await.result(reader1.readNext());
+        }
+
+        Utils.close(reader1);
+        dlm0.close();
+        ns0.close();
+        dlm1.close();
+        ns1.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReaderLockFutureCancelledWhileWaiting() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogManager dlm0 = createNewDLM(conf, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
+        writer.write(DLMTestUtil.getLogRecordInstance(1L));
+        writer.write(DLMTestUtil.getLogRecordInstance(2L));
+        writer.closeAndComplete();
+
+        DistributedLogManager dlm1 = createNewDLM(conf, name);
+        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        AsyncLogReader reader1 = Await.result(futureReader1);
+
+        DistributedLogManager dlm2 = createNewDLM(conf, name);
+        Future<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        try {
+            FutureUtils.cancel(futureReader2);
+            Await.result(futureReader2);
+            fail("Should fail getting log reader as it is cancelled");
+        } catch (LockClosedException ex) {
+        } catch (LockCancelledException ex) {
+        } catch (OwnershipAcquireFailedException oafe) {
+        }
+
+        futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        Utils.close(reader1);
+
+        Await.result(futureReader2);
+
+        dlm0.close();
+        dlm1.close();
+        dlm2.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReaderLockFutureCancelledWhileLocked() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogManager dlm0 = createNewDLM(conf, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
+        writer.write(DLMTestUtil.getLogRecordInstance(1L));
+        writer.write(DLMTestUtil.getLogRecordInstance(2L));
+        writer.closeAndComplete();
+
+        DistributedLogManager dlm1 = createNewDLM(conf, name);
+        Future<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());
+
+        dlm0.close();
+        dlm1.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReaderLockSharedDlmDoesNotConflict() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogManager dlm0 = createNewDLM(conf, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm0.startAsyncLogSegmentNonPartitioned());
+        writer.write(DLMTestUtil.getLogRecordInstance(1L));
+        writer.write(DLMTestUtil.getLogRecordInstance(2L));
+        writer.closeAndComplete();
+
+        DistributedLogManager dlm1 = createNewDLM(conf, name);
+        Future<AsyncLogReader> futureReader1 = dlm1.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        Future<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);
+
+        dlm0.close();
+        dlm1.close();
+    }
+
+    static class ReadRecordsListener implements FutureEventListener<AsyncLogReader> {
+
+        final AtomicReference<DLSN> currentDLSN;
+        final String name;
+        final ExecutorService executorService;
+
+        final CountDownLatch latch = new CountDownLatch(1);
+        boolean failed = false;
+
+        public ReadRecordsListener(AtomicReference<DLSN> currentDLSN,
+                                   String name,
+                                   ExecutorService executorService) {
+            this.currentDLSN = currentDLSN;
+            this.name = name;
+            this.executorService = executorService;
+        }
+        public CountDownLatch getLatch() {
+            return latch;
+        }
+        public boolean failed() {
+            return failed;
+        }
+        public boolean done() {
+            return latch.getCount() == 0;
+        }
+
+        @Override
+        public void onSuccess(final AsyncLogReader reader) {
+            LOG.info("Reader {} is ready to read entries", name);
+            executorService.submit(new Runnable() {
+                @Override
+                public void run() {
+                    readEntries(reader);
+                }
+            });
+        }
+
+        private void readEntries(AsyncLogReader reader) {
+            try {
+                for (int i = 0; i < 300; i++) {
+                    LogRecordWithDLSN record = Await.result(reader.readNext());
+                    currentDLSN.set(record.getDlsn());
+                }
+            } catch (Exception ex) {
+                failed = true;
+            } finally {
+                latch.countDown();
+            }
+        }
+
+        @Override
+        public void onFailure(Throwable cause) {
+            LOG.error("{} failed to open reader", name, cause);
+            failed = true;
+            latch.countDown();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testReaderLockMultiReadersScenario() throws Exception {
+        final String name = runtime.getMethodName();
+        URI uri = createDLMURI("/" + name);
+        ensureURICreated(uri);
+
+        DistributedLogConfiguration localConf = new DistributedLogConfiguration();
+        localConf.addConfiguration(conf);
+        localConf.setImmediateFlushEnabled(false);
+        localConf.setPeriodicFlushFrequencyMilliSeconds(60 * 1000);
+        localConf.setOutputBufferSize(0);
+        // Otherwise, we won't be able to run scheduled threads for readahead when we're in a callback.
+        localConf.setNumWorkerThreads(2);
+        localConf.setLockTimeout(Long.MAX_VALUE);
+
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(localConf).uri(uri).clientId("main").build();
+
+        DistributedLogManager dlm0 = namespace.openLog(name);
+        DLMTestUtil.generateCompletedLogSegments(dlm0, localConf, 9, 100);
+        dlm0.close();
+
+        int recordCount = 0;
+        AtomicReference<DLSN> currentDLSN = new AtomicReference<DLSN>(DLSN.InitialDLSN);
+
+        String clientId1 = "reader1";
+        DistributedLogNamespace namespace1 = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(localConf).uri(uri).clientId(clientId1).build();
+        DistributedLogManager dlm1 = namespace1.openLog(name);
+        String clientId2 = "reader2";
+        DistributedLogNamespace namespace2 = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(localConf).uri(uri).clientId(clientId2).build();
+        DistributedLogManager dlm2 = namespace2.openLog(name);
+        String clientId3 = "reader3";
+        DistributedLogNamespace namespace3 = DistributedLogNamespaceBuilder.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);
+        LOG.info("{} opened reader on stream {}", clientId1, name);
+
+        LOG.info("{} is opening reader on stream {}", clientId2, name);
+        Future<AsyncLogReader> futureReader2 = dlm2.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+        LOG.info("{} is opening reader on stream {}", clientId3, name);
+        Future<AsyncLogReader> futureReader3 = dlm3.getAsyncLogReaderWithLock(DLSN.InitialDLSN);
+
+        ExecutorService executorService = Executors.newCachedThreadPool();
+
+        ReadRecordsListener listener2 =
+                new ReadRecordsListener(currentDLSN, clientId2, executorService);
+        ReadRecordsListener listener3 =
+                new ReadRecordsListener(currentDLSN, clientId3, executorService);
+        futureReader2.addEventListener(listener2);
+        futureReader3.addEventListener(listener3);
+
+        // Get reader1 and start reading.
+        for ( ; recordCount < 200; recordCount++) {
+            LogRecordWithDLSN record = Await.result(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);
+        listener2.getLatch().await();
+        assertTrue(listener2.done());
+        assertTrue(listener2.failed());
+
+        // Reader1 starts reading again.
+        for (; recordCount < 300; recordCount++) {
+            LogRecordWithDLSN record = Await.result(reader1.readNext());
+            currentDLSN.set(record.getDlsn());
+        }
+
+        // Reader1 is done, someone else can take over. Since reader2 was
+        // aborted, reader3 should take its place.
+        assertFalse(listener3.done());
+        Utils.close(reader1);
+        listener3.getLatch().await();
+        assertTrue(listener3.done());
+        assertFalse(listener3.failed());
+
+        assertEquals(new DLSN(3, 99, 0), currentDLSN.get());
+
+        try {
+            Await.result(futureReader2);
+        } catch (Exception ex) {
+            // Can't get this one to close it--the dlm will take care of it.
+        }
+
+        Utils.close(Await.result(futureReader3));
+
+        dlm1.close();
+        namespace1.close();
+        dlm2.close();
+        namespace2.close();
+        dlm3.close();
+        namespace3.close();
+
+        executorService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testAsyncReadWithSubscriberId() throws Exception {
+        String name = "distrlog-asyncread-with-sbuscriber-id";
+        String subscriberId = "asyncreader";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        DLSN readDLSN = DLSN.InitialDLSN;
+
+        int txid = 1;
+        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++)));
+                if (i == 1 && j == 1L) {
+                    readDLSN = dlsn;
+                }
+            }
+            writer.closeAndComplete();
+        }
+
+        BKAsyncLogReader reader0 = (BKAsyncLogReader) Await.result(dlm.getAsyncLogReaderWithLock(subscriberId));
+        assertEquals(DLSN.NonInclusiveLowerBound, reader0.getStartDLSN());
+        long numTxns = 0;
+        LogRecordWithDLSN record = Await.result(reader0.readNext());
+        while (null != record) {
+            DLMTestUtil.verifyEmptyLogRecord(record);
+            ++numTxns;
+            assertEquals(numTxns, record.getTransactionId());
+            assertEquals(record.getTransactionId() - 1, record.getSequenceId());
+
+            if (txid - 1 == numTxns) {
+                break;
+            }
+            record = Await.result(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));
+        assertEquals(readDLSN, reader1.getStartDLSN());
+        numTxns = 0;
+        long startTxID =  10L;
+        record = Await.result(reader1.readNext());
+        while (null != record) {
+            DLMTestUtil.verifyEmptyLogRecord(record);
+            ++numTxns;
+            ++startTxID;
+            assertEquals(startTxID, record.getTransactionId());
+            assertEquals(record.getTransactionId() - 1L, record.getSequenceId());
+
+            if (startTxID == txid - 1) {
+                break;
+            }
+            record = Await.result(reader1.readNext());
+        }
+        assertEquals(txid - 1, startTxID);
+        assertEquals(20, numTxns);
+        Utils.close(reader1);
+
+        dlm.close();
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
new file mode 100644
index 0000000..2f5766d
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestAsyncReaderWriter.java
@@ -0,0 +1,2206 @@
+/**
+ * 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 java.io.IOException;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicReference;
+
+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.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.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;
+import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.feature.FixedValueFeature;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.IdleReaderException;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+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.util.FailpointUtils;
+import org.apache.distributedlog.util.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;
+import static org.apache.distributedlog.DLMTestUtil.validateFutureFailed;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+import static org.junit.Assert.*;
+
+public class TestAsyncReaderWriter extends TestDistributedLogBase {
+    static final Logger LOG = LoggerFactory.getLogger(TestAsyncReaderWriter.class);
+
+    protected DistributedLogConfiguration testConf;
+
+    public TestAsyncReaderWriter() {
+        this.testConf = new DistributedLogConfiguration();
+        this.testConf.loadConf(conf);
+        this.testConf.setReaderIdleErrorThresholdMillis(1200000);
+        this.testConf.setReadAheadWaitTimeOnEndOfStream(20);
+    }
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    /**
+     * Test writing control records to writers: writers should be able to write control records, and
+     * the readers should skip control records while reading.
+     */
+    @Test(timeout = 60000)
+    public void testWriteControlRecord() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(1024);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        // Write 3 log segments. For each log segments, write one control record and nine user records.
+        int txid = 1;
+        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))));
+            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));
+            }
+            writer.closeAndComplete();
+        }
+        dlm.close();
+
+        // Read all the written data: It should skip control records and only return user records.
+        DistributedLogManager readDlm = createNewDLM(confLocal, name);
+        LogReader reader = readDlm.getInputStream(1);
+
+        long numTrans = 0;
+        long expectedTxId = 2;
+        LogRecord record = reader.readNext(false);
+        while (null != record) {
+            DLMTestUtil.verifyLargeLogRecord(record);
+            numTrans++;
+            assertEquals(expectedTxId, record.getTransactionId());
+            if (expectedTxId % 10 == 0) {
+                expectedTxId += 2;
+            } else {
+                ++expectedTxId;
+            }
+            record = reader.readNext(false);
+        }
+        reader.close();
+        assertEquals(3 * 9, numTrans);
+        assertEquals(3 * 9, readDlm.getLogRecordCount());
+        readDlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testAsyncWritePendingWritesAbortedWhenLedgerRollTriggerFails() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(1024);
+        confLocal.setMaxLogSegmentBytes(1024);
+        confLocal.setLogSegmentRollingIntervalMinutes(0);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+
+        // 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));
+        assertEquals(1, dlsn.getLogSegmentSequenceNo());
+
+        record = DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1);
+        result = writer.write(record);
+        validateFutureFailed(result, LogRecordTooLongException.class);
+
+        record = DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1);
+        result = writer.write(record);
+        validateFutureFailed(result, WriteException.class);
+
+        record = DLMTestUtil.getLogRecordInstance(txid++, MAX_LOGRECORD_SIZE + 1);
+        result = writer.write(record);
+        validateFutureFailed(result, WriteException.class);
+
+        writer.closeAndComplete();
+        dlm.close();
+    }
+
+    /**
+     * Test Case: Simple Async Writes. Writes 30 records. They should be written correctly.
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testSimpleAsyncWrite() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(1024);
+
+        int numLogSegments = 3;
+        int numRecordsPerLogSegment = 10;
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        final CountDownLatch syncLatch = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
+        final AtomicBoolean errorsFound = new AtomicBoolean(false);
+        final AtomicReference<DLSN> maxDLSN = new AtomicReference<DLSN>(DLSN.InvalidDLSN);
+        int txid = 1;
+        for (long i = 0; i < numLogSegments; i++) {
+            final long currentLogSegmentSeqNo = i + 1;
+            BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+            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>() {
+                    @Override
+                    public void onSuccess(DLSN value) {
+                        if(value.getLogSegmentSequenceNo() != currentLogSegmentSeqNo) {
+                            LOG.debug("LogSegmentSequenceNumber: {}, Expected {}", value.getLogSegmentSequenceNo(), currentLogSegmentSeqNo);
+                            errorsFound.set(true);
+                        }
+
+                        if(value.getEntryId() != currentEntryId) {
+                            LOG.debug("EntryId: {}, Expected {}", value.getEntryId(), currentEntryId);
+                            errorsFound.set(true);
+                        }
+
+                        if (value.compareTo(maxDLSN.get()) > 0) {
+                            maxDLSN.set(value);
+                        }
+
+                        syncLatch.countDown();
+                        LOG.debug("SyncLatch: {}", syncLatch.getCount());
+                    }
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        LOG.error("Encountered exception on writing record {} in log segment {}", currentEntryId, currentLogSegmentSeqNo);
+                        errorsFound.set(true);
+                    }
+                });
+            }
+            writer.closeAndComplete();
+        }
+
+        syncLatch.await();
+        assertFalse("Should not encounter any errors for async writes", errorsFound.get());
+
+        LogRecordWithDLSN last = dlm.getLastLogRecord();
+        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()));
+        DLMTestUtil.verifyLargeLogRecord(last);
+
+        dlm.close();
+    }
+
+    /**
+     * Write records into <i>numLogSegments</i> log segments. Each log segment has <i>numRecordsPerLogSegment</i> records.
+     *
+     * @param dlm
+     *          distributedlog manager
+     * @param numLogSegments
+     *          number of log segments
+     * @param numRecordsPerLogSegment
+     *          number records per log segment
+     * @param startTxId
+     *          start tx id
+     * @return next tx id
+     */
+    private static long writeRecords(DistributedLogManager dlm,
+                                     int numLogSegments,
+                                     int numRecordsPerLogSegment,
+                                     long startTxId,
+                                     boolean emptyRecord) throws IOException {
+        long txid = startTxId;
+        for (long i = 0; i < numLogSegments; i++) {
+            BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+            for (long j = 1; j <= numRecordsPerLogSegment; j++) {
+                if (emptyRecord) {
+                    writer.write(DLMTestUtil.getEmptyLogRecordInstance(txid++));
+                } else {
+                    writer.write(DLMTestUtil.getLargeLogRecordInstance(txid++));
+                }
+            }
+            writer.closeAndComplete();
+        }
+        return txid;
+    }
+
+    /**
+     * Write <code>numRecords</code> records to the log, starting with <code>startTxId</code>.
+     * It flushes every <code>flushPerNumRecords</code> records.
+     *
+     * @param dlm
+     *          distributedlog manager
+     * @param numRecords
+     *          num records to write
+     * @param startTxId
+     *          start tx id
+     * @param flushPerNumRecords
+     *          number records to flush
+     * @return next tx id
+     * @throws IOException
+     */
+    private static long writeLogSegment(DistributedLogManager dlm,
+                                        int numRecords,
+                                        long startTxId,
+                                        int flushPerNumRecords,
+                                        boolean emptyRecord) throws IOException {
+        long txid = startTxId;
+        LogWriter writer = dlm.startLogSegmentNonPartitioned();
+        for (long j = 1; j <= numRecords; j++) {
+            if (emptyRecord) {
+                writer.write(DLMTestUtil.getEmptyLogRecordInstance(txid++));
+            } else {
+                writer.write(DLMTestUtil.getLargeLogRecordInstance(txid++));
+            }
+            if (j % flushPerNumRecords == 0 ) {
+                writer.setReadyToFlush();
+                writer.flushAndSync();
+            }
+        }
+        writer.setReadyToFlush();
+        writer.flushAndSync();
+        writer.close();
+        return txid;
+    }
+
+    private static void readNext(final AsyncLogReader reader,
+                                 final DLSN startPosition,
+                                 final long startSequenceId,
+                                 final boolean monotonic,
+                                 final CountDownLatch syncLatch,
+                                 final CountDownLatch completionLatch,
+                                 final AtomicBoolean errorsFound) {
+        Future<LogRecordWithDLSN> record = reader.readNext();
+        record.addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+            @Override
+            public void onSuccess(LogRecordWithDLSN value) {
+                try {
+                    if (monotonic) {
+                        assertEquals(startSequenceId, value.getSequenceId());
+                    } else {
+                        assertTrue(value.getSequenceId() < 0);
+                        assertTrue(value.getSequenceId() > startSequenceId);
+                    }
+                    LOG.info("Received record {} from {}", value, reader.getStreamName());
+                    assertTrue(!value.isControl());
+                    assertTrue(value.getDlsn().getSlotId() == 0);
+                    assertTrue(value.getDlsn().compareTo(startPosition) >= 0);
+                    DLMTestUtil.verifyLargeLogRecord(value);
+                } catch (Exception exc) {
+                    LOG.debug("Exception Encountered when verifying log record {} : ", value.getDlsn(), exc);
+                    errorsFound.set(true);
+                    completionLatch.countDown();
+                    return;
+                }
+                syncLatch.countDown();
+                if (syncLatch.getCount() <= 0) {
+                    completionLatch.countDown();
+                } else {
+                    TestAsyncReaderWriter.readNext(
+                            reader,
+                            value.getDlsn().getNextDLSN(),
+                            monotonic ? value.getSequenceId() + 1 : value.getSequenceId(),
+                            monotonic,
+                            syncLatch,
+                            completionLatch,
+                            errorsFound);
+                }
+            }
+            @Override
+            public void onFailure(Throwable cause) {
+                LOG.error("Encountered Exception on reading {}", reader.getStreamName(), cause);
+                errorsFound.set(true);
+                completionLatch.countDown();
+            }
+        });
+    }
+
+    void simpleAsyncReadTest(String name, DistributedLogConfiguration confLocal) throws Exception {
+        confLocal.setOutputBufferSize(1024);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadBatchSize(10);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        int numLogSegments = 3;
+        int numRecordsPerLogSegment = 10;
+
+        // Write 30 records: 3 log segments, 10 records per log segment
+        long txid = 1L;
+        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false);
+        // Write another log segment with 5 records and flush every 2 records
+        txid = writeLogSegment(dlm, 5, txid, 2, false);
+
+        final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
+        final CountDownLatch syncLatch = new CountDownLatch((int) (txid - 1));
+        final CountDownLatch completionLatch = new CountDownLatch(1);
+        final AtomicBoolean errorsFound = new AtomicBoolean(false);
+
+        boolean monotonic = LogSegmentMetadata.supportsSequenceId(confLocal.getDLLedgerMetadataLayoutVersion());
+        TestAsyncReaderWriter.readNext(
+                reader,
+                DLSN.InvalidDLSN,
+                monotonic ? 0L : Long.MIN_VALUE,
+                monotonic,
+                syncLatch,
+                completionLatch,
+                errorsFound);
+
+        completionLatch.await();
+        assertFalse("Errors encountered on reading records", errorsFound.get());
+        syncLatch.await();
+
+        Utils.close(reader);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testSimpleAsyncRead() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        simpleAsyncReadTest(name, confLocal);
+    }
+
+    @Test(timeout = 60000)
+    public void testSimpleAsyncReadWriteWithMonitoredFuturePool() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setTaskExecutionWarnTimeMicros(1000);
+        confLocal.setEnableTaskExecutionStats(true);
+        simpleAsyncReadTest(name, confLocal);
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkAsyncRead() throws Exception {
+        String name = "distrlog-bulkasyncread";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadMaxRecords(10000);
+        confLocal.setReadAheadBatchSize(10);
+
+        int numLogSegments = 3;
+        int numRecordsPerLogSegment = 20;
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, 1L, false);
+
+        final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN);
+        int expectedTxID = 1;
+        int numReads = 0;
+        while (expectedTxID <= numLogSegments * numRecordsPerLogSegment) {
+            if (expectedTxID == numLogSegments * numRecordsPerLogSegment) {
+                break;
+            }
+            List<LogRecordWithDLSN> records = Await.result(reader.readBulk(20));
+            LOG.info("Bulk read {} entries.", records.size());
+
+            assertTrue(records.size() >= 1);
+            for (LogRecordWithDLSN record : records) {
+                assertEquals(expectedTxID, record.getTransactionId());
+                ++expectedTxID;
+            }
+            ++numReads;
+        }
+
+        // we expect bulk read works
+        assertTrue(numReads < 60);
+
+        Utils.close(reader);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkAsyncReadWithWriteBatch() throws Exception {
+        String name = "distrlog-bulkasyncread-with-writebatch";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setOutputBufferSize(1024000);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadMaxRecords(10000);
+        confLocal.setReadAheadBatchSize(10);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        int numLogSegments = 3;
+        int numRecordsPerLogSegment = 20;
+
+        writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, 1L, false);
+
+        final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN);
+        int expectedTxID = 1;
+        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));
+            assertEquals(20, records.size());
+            for (LogRecordWithDLSN record : records) {
+                assertEquals(expectedTxID, record.getTransactionId());
+                ++expectedTxID;
+            }
+        }
+
+        Utils.close(reader);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testAsyncReadEmptyRecords() throws Exception {
+        String name = "distrlog-simpleasyncreadempty";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadBatchSize(10);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        int numLogSegments = 3;
+        int numRecordsPerLogSegment = 10;
+
+        long txid = 1L;
+        // write 3 log segments, 10 records per log segment
+        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, true);
+        // write another log segment with 5 records and flush every 2 records
+        txid = writeLogSegment(dlm, 5, txid, 2, true);
+
+        AsyncLogReader asyncReader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
+        assertEquals("Expected stream name = " + name + " but " + asyncReader.getStreamName() + " found",
+                name, asyncReader.getStreamName());
+        long numTrans = 0;
+        DLSN lastDLSN = DLSN.InvalidDLSN;
+        LogRecordWithDLSN record = Await.result(asyncReader.readNext());
+        while (null != record) {
+            DLMTestUtil.verifyEmptyLogRecord(record);
+            assertEquals(0, record.getDlsn().getSlotId());
+            assertTrue(record.getDlsn().compareTo(lastDLSN) > 0);
+            lastDLSN = record.getDlsn();
+            numTrans++;
+            if (numTrans >= (txid - 1)) {
+                break;
+            }
+            record = Await.result(asyncReader.readNext());
+        }
+        assertEquals((txid - 1), numTrans);
+        Utils.close(asyncReader);
+        dlm.close();
+    }
+
+    /**
+     * Test Async Read by positioning to a given position in the log
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testSimpleAsyncReadPosition() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(1024);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadBatchSize(10);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        int numLogSegments = 3;
+        int numRecordsPerLogSegment = 10;
+
+        long txid = 1L;
+        // write 3 log segments, 10 records per log segment
+        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false);
+        // write another log segment with 5 records
+        txid = writeLogSegment(dlm, 5, txid, Integer.MAX_VALUE, false);
+
+        final CountDownLatch syncLatch = new CountDownLatch((int)(txid - 14));
+        final CountDownLatch doneLatch = new CountDownLatch(1);
+        final AtomicBoolean errorsFound = new AtomicBoolean(false);
+        final AsyncLogReader reader = dlm.getAsyncLogReader(new DLSN(2, 2, 4));
+        assertEquals(name, reader.getStreamName());
+
+        boolean monotonic = LogSegmentMetadata.supportsSequenceId(confLocal.getDLLedgerMetadataLayoutVersion());
+        TestAsyncReaderWriter.readNext(
+                reader,
+                new DLSN(2, 3, 0),
+                monotonic ? 13L : Long.MIN_VALUE,
+                monotonic,
+                syncLatch,
+                doneLatch,
+                errorsFound);
+
+        doneLatch.await();
+        assertFalse("Errors found on reading records", errorsFound.get());
+        syncLatch.await();
+
+        Utils.close(reader);
+        dlm.close();
+    }
+
+    /**
+     * Test write/read entries when immediate flush is disabled.
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testSimpleAsyncReadWrite() throws Exception {
+        testSimpleAsyncReadWriteInternal(runtime.getMethodName(), false);
+    }
+
+    /**
+     * Test write/read entries when immediate flush is enabled.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testSimpleAsyncReadWriteImmediateFlush() throws Exception {
+        testSimpleAsyncReadWriteInternal(runtime.getMethodName(), true);
+    }
+
+    /**
+     * Test if entries written using log segment metadata that doesn't support enveloping
+     * can be read correctly by a reader supporting both.
+     *
+     * NOTE: An older reader cannot read enveloped entry, so we don't have a test case covering
+     *       the other scenario.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testNoEnvelopeWriterEnvelopeReader() throws Exception {
+        testSimpleAsyncReadWriteInternal(runtime.getMethodName(), true,
+                LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value - 1);
+    }
+
+    static class WriteFutureEventListener implements FutureEventListener<DLSN> {
+        private final LogRecord record;
+        private final long currentLogSegmentSeqNo;
+        private final long currentEntryId;
+        private final CountDownLatch syncLatch;
+        private final AtomicBoolean errorsFound;
+        private final boolean verifyEntryId;
+
+        WriteFutureEventListener(LogRecord record,
+                                 long currentLogSegmentSeqNo,
+                                 long currentEntryId,
+                                 CountDownLatch syncLatch,
+                                 AtomicBoolean errorsFound,
+                                 boolean verifyEntryId) {
+            this.record = record;
+            this.currentLogSegmentSeqNo = currentLogSegmentSeqNo;
+            this.currentEntryId = currentEntryId;
+            this.syncLatch = syncLatch;
+            this.errorsFound = errorsFound;
+            this.verifyEntryId = verifyEntryId;
+        }
+
+        /**
+         * Invoked if the computation completes successfully
+         */
+        @Override
+        public void onSuccess(DLSN value) {
+            if(value.getLogSegmentSequenceNo() != currentLogSegmentSeqNo) {
+                LOG.error("Ledger Seq No: {}, Expected: {}", value.getLogSegmentSequenceNo(), currentLogSegmentSeqNo);
+                errorsFound.set(true);
+            }
+
+            if(verifyEntryId && value.getEntryId() != currentEntryId) {
+                LOG.error("EntryId: {}, Expected: {}", value.getEntryId(), currentEntryId);
+                errorsFound.set(true);
+            }
+            syncLatch.countDown();
+        }
+
+        /**
+         * Invoked if the computation completes unsuccessfully
+         */
+        @Override
+        public void onFailure(Throwable cause) {
+            LOG.error("Encountered failures on writing record as (lid = {}, eid = {}) :",
+                    new Object[]{currentLogSegmentSeqNo, currentEntryId, cause});
+            errorsFound.set(true);
+            syncLatch.countDown();
+        }
+    }
+
+    void testSimpleAsyncReadWriteInternal(String name, boolean immediateFlush)
+            throws Exception {
+        testSimpleAsyncReadWriteInternal(name, immediateFlush,
+                                         LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
+    }
+
+    void testSimpleAsyncReadWriteInternal(String name, boolean immediateFlush,
+                                          int logSegmentVersion) throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadBatchSize(10);
+        confLocal.setOutputBufferSize(1024);
+        confLocal.setDLLedgerMetadataLayoutVersion(logSegmentVersion);
+        confLocal.setImmediateFlushEnabled(immediateFlush);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        int numLogSegments = 3;
+        int numRecordsPerLogSegment = 10;
+
+        final CountDownLatch readLatch = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
+        final CountDownLatch readDoneLatch = new CountDownLatch(1);
+        final AtomicBoolean readErrors = new AtomicBoolean(false);
+        final CountDownLatch writeLatch = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
+        final AtomicBoolean writeErrors = new AtomicBoolean(false);
+        final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
+        assertEquals(name, reader.getStreamName());
+
+        int txid = 1;
+        for (long i = 0; i < 3; i++) {
+            final long currentLogSegmentSeqNo = i + 1;
+            BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+            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(
+                        record, currentLogSegmentSeqNo, currentEntryId, writeLatch, writeErrors, true));
+                if (i == 0 && j == 0) {
+                    boolean monotonic = LogSegmentMetadata.supportsSequenceId(logSegmentVersion);
+                    TestAsyncReaderWriter.readNext(
+                            reader,
+                            DLSN.InvalidDLSN,
+                            monotonic ? 0L : Long.MIN_VALUE,
+                            monotonic,
+                            readLatch,
+                            readDoneLatch,
+                            readErrors);
+                }
+            }
+            writer.closeAndComplete();
+        }
+
+        writeLatch.await();
+        assertFalse("All writes should succeed", writeErrors.get());
+
+        readDoneLatch.await();
+        assertFalse("All reads should succeed", readErrors.get());
+        readLatch.await();
+
+        Utils.close(reader);
+        dlm.close();
+    }
+
+
+    /**
+     * Test Case: starting reading when the streams don't exist.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testSimpleAsyncReadWriteStartEmpty() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadBatchSize(10);
+        confLocal.setOutputBufferSize(1024);
+
+        int numLogSegments = 3;
+        int numRecordsPerLogSegment = 10;
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        final CountDownLatch readerReadyLatch = new CountDownLatch(1);
+        final CountDownLatch readerDoneLatch = new CountDownLatch(1);
+        final CountDownLatch readerSyncLatch = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
+
+        final TestReader reader = new TestReader(
+                "test-reader",
+                dlm,
+                DLSN.InitialDLSN,
+                false,
+                0,
+                readerReadyLatch,
+                readerSyncLatch,
+                readerDoneLatch);
+
+        reader.start();
+
+        // Increase the probability of reader failure and retry
+        Thread.sleep(500);
+
+        final AtomicBoolean writeErrors = new AtomicBoolean(false);
+        final CountDownLatch writeLatch = new CountDownLatch(30);
+
+        int txid = 1;
+        for (long i = 0; i < 3; i++) {
+            final long currentLogSegmentSeqNo = i + 1;
+            BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+            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(
+                        record, currentLogSegmentSeqNo, currentEntryId, writeLatch, writeErrors, true));
+            }
+            writer.closeAndComplete();
+        }
+
+        writeLatch.await();
+        assertFalse("All writes should succeed", writeErrors.get());
+
+        readerDoneLatch.await();
+        assertFalse("Should not encounter errors during reading", reader.areErrorsFound());
+        readerSyncLatch.await();
+
+        assertTrue("Should position reader at least once", reader.getNumReaderPositions().get() > 1);
+        reader.stop();
+        dlm.close();
+    }
+
+
+    /**
+     * Test Case: starting reading when the streams don't exist.
+     * {@link https://issues.apache.org/jira/browse/DL-42}
+     */
+    @DistributedLogAnnotations.FlakyTest
+    @Ignore
+    @Test(timeout = 120000)
+    public void testSimpleAsyncReadWriteStartEmptyFactory() throws Exception {
+        // int count = 50;
+        int count = 1;
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadBatchSize(10);
+        confLocal.setOutputBufferSize(1024);
+
+        int numLogSegments = 3;
+        int numRecordsPerLogSegment = 1;
+
+        URI uri = createDLMURI("/" + name);
+        ensureURICreated(uri);
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(confLocal).uri(uri).build();
+        final DistributedLogManager[] dlms = new DistributedLogManager[count];
+        final TestReader[] readers = new TestReader[count];
+        final CountDownLatch readyLatch = new CountDownLatch(count);
+        final CountDownLatch[] syncLatches = new CountDownLatch[count];
+        final CountDownLatch[] readerDoneLatches = new CountDownLatch[count];
+        for (int s = 0; s < count; s++) {
+            dlms[s] = namespace.openLog(name + String.format("%d", s));
+            readerDoneLatches[s] = new CountDownLatch(1);
+            syncLatches[s] = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
+            readers[s] = new TestReader("reader-" + s,
+                    dlms[s], DLSN.InitialDLSN, false, 0, readyLatch, syncLatches[s], readerDoneLatches[s]);
+            readers[s].start();
+        }
+
+        // wait all readers were positioned at least once
+        readyLatch.await();
+
+        final CountDownLatch writeLatch = new CountDownLatch(3 * count);
+        final AtomicBoolean writeErrors = new AtomicBoolean(false);
+
+        int txid = 1;
+        for (long i = 0; i < 3; i++) {
+            final long currentLogSegmentSeqNo = i + 1;
+            BKAsyncLogWriter[] writers = new BKAsyncLogWriter[count];
+            for (int s = 0; s < count; s++) {
+                writers[s] = (BKAsyncLogWriter)(dlms[s].startAsyncLogSegmentNonPartitioned());
+            }
+            for (long j = 0; j < 1; j++) {
+                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(
+                            record, currentLogSegmentSeqNo, currentEntryId, writeLatch, writeErrors, true));
+                }
+            }
+            for (int s = 0; s < count; s++) {
+                writers[s].closeAndComplete();
+            }
+        }
+
+        writeLatch.await();
+        assertFalse("All writes should succeed", writeErrors.get());
+
+        for (int s = 0; s < count; s++) {
+            readerDoneLatches[s].await();
+            assertFalse("Reader " + s + " should not encounter errors", readers[s].areErrorsFound());
+            syncLatches[s].await();
+            assertEquals(numLogSegments * numRecordsPerLogSegment, readers[s].getNumReads().get());
+            assertTrue("Reader " + s + " should position at least once", readers[s].getNumReaderPositions().get() > 0);
+        }
+
+        for (int s = 0; s < count; s++) {
+            readers[s].stop();
+            dlms[s].close();
+        }
+    }
+
+    /**
+     * Flaky test fixed: readers need to be added to the pendingReaders
+     * @throws Exception
+     */
+    @Test(timeout = 300000)
+    public void testSimpleAsyncReadWriteSimulateErrors() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadBatchSize(10);
+        confLocal.setOutputBufferSize(1024);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        int numLogSegments = 5;
+        int numRecordsPerLogSegment = 10;
+
+        final CountDownLatch doneLatch = new CountDownLatch(1);
+        final CountDownLatch syncLatch = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
+
+        TestReader reader = new TestReader(
+                "test-reader",
+                dlm,
+                DLSN.InitialDLSN,
+                true,
+                0,
+                new CountDownLatch(1),
+                syncLatch,
+                doneLatch);
+
+        reader.start();
+
+        final CountDownLatch writeLatch = new CountDownLatch(numLogSegments * numRecordsPerLogSegment);
+        final AtomicBoolean writeErrors = new AtomicBoolean(false);
+
+        int txid = 1;
+        for (long i = 0; i < numLogSegments; i++) {
+            final long currentLogSegmentSeqNo = i + 1;
+            BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+            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(
+                        record, currentLogSegmentSeqNo, currentEntryId, writeLatch, writeErrors, true));
+            }
+            writer.closeAndComplete();
+        }
+
+        writeLatch.await();
+        assertFalse("All writes should succeed", writeErrors.get());
+
+        doneLatch.await();
+        assertFalse("Should not encounter errors during reading", reader.areErrorsFound());
+        syncLatch.await();
+
+        assertTrue("Should position reader at least once", reader.getNumReaderPositions().get() > 1);
+        reader.stop();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testSimpleAsyncReadWritePiggyBack() throws Exception {
+        String name = runtime.getMethodName();
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setEnableReadAhead(true);
+        confLocal.setReadAheadWaitTime(500);
+        confLocal.setReadAheadBatchSize(10);
+        confLocal.setReadAheadMaxRecords(100);
+        confLocal.setOutputBufferSize(1024);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(100);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
+
+        int numLogSegments = 3;
+        int numRecordsPerLogSegment = 10;
+
+        final CountDownLatch readLatch = new CountDownLatch(30);
+        final CountDownLatch readDoneLatch = new CountDownLatch(1);
+        final AtomicBoolean readErrors = new AtomicBoolean(false);
+        final CountDownLatch writeLatch = new CountDownLatch(30);
+        final AtomicBoolean writeErrors = new AtomicBoolean(false);
+
+        int txid = 1;
+        for (long i = 0; i < numLogSegments; i++) {
+            final long currentLogSegmentSeqNo = i + 1;
+            BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+            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(
+                        record, currentLogSegmentSeqNo, j, writeLatch, writeErrors, false));
+                if (i == 0 && j == 0) {
+                    boolean monotonic = LogSegmentMetadata.supportsSequenceId(confLocal.getDLLedgerMetadataLayoutVersion());
+                    TestAsyncReaderWriter.readNext(
+                            reader,
+                            DLSN.InvalidDLSN,
+                            monotonic ? 0L : Long.MIN_VALUE,
+                            monotonic,
+                            readLatch,
+                            readDoneLatch,
+                            readErrors);
+                }
+            }
+            writer.closeAndComplete();
+        }
+
+        writeLatch.await();
+        assertFalse("All writes should succeed", writeErrors.get());
+
+        readDoneLatch.await();
+        assertFalse("All reads should succeed", readErrors.get());
+        readLatch.await();
+
+        Utils.close(reader);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testCancelReadRequestOnReaderClosed() throws Exception {
+        final String name = "distrlog-cancel-read-requests-on-reader-closed";
+
+        DistributedLogManager dlm = createNewDLM(testConf, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+        writer.write(DLMTestUtil.getLogRecordInstance(1L));
+        writer.closeAndComplete();
+
+        final AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN);
+        LogRecordWithDLSN record = Await.result(reader.readNext());
+        assertEquals(1L, record.getTransactionId());
+        DLMTestUtil.verifyLogRecord(record);
+
+        final CountDownLatch readLatch = new CountDownLatch(1);
+        final AtomicBoolean receiveExpectedException = new AtomicBoolean(false);
+        Thread readThread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    Await.result(reader.readNext());
+                } catch (ReadCancelledException rce) {
+                    receiveExpectedException.set(true);
+                } catch (Throwable t) {
+                    LOG.error("Receive unexpected exception on reading stream {} : ", name, t);
+                }
+                readLatch.countDown();
+            }
+        }, "read-thread");
+        readThread.start();
+
+        Thread.sleep(1000);
+
+        // close reader should cancel the pending read next
+        Utils.close(reader);
+
+        readLatch.await();
+        readThread.join();
+
+        assertTrue("Read request should be cancelled.", receiveExpectedException.get());
+
+        // closed reader should reject any readNext
+        try {
+            Await.result(reader.readNext());
+            fail("Reader should reject readNext if it is closed.");
+        } catch (ReadCancelledException rce) {
+            // expected
+        }
+
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testAsyncWriteWithMinDelayBetweenFlushes() throws Exception {
+        String name = "distrlog-asyncwrite-mindelay";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setMinDelayBetweenImmediateFlushMs(100);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        final Thread currentThread = Thread.currentThread();
+        final int COUNT = 5000;
+        final CountDownLatch syncLatch = new CountDownLatch(COUNT);
+        int txid = 1;
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+        Stopwatch executionTime = Stopwatch.createStarted();
+        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>() {
+                @Override
+                public void onSuccess(DLSN value) {
+                    syncLatch.countDown();
+                    LOG.debug("SyncLatch: {} ; DLSN: {} ", syncLatch.getCount(), value);
+                }
+                @Override
+                public void onFailure(Throwable cause) {
+                    currentThread.interrupt();
+                }
+            });
+        }
+
+        boolean success = false;
+        if (!(Thread.interrupted())) {
+            try {
+                success = syncLatch.await(10, TimeUnit.SECONDS);
+            } catch (InterruptedException exc) {
+                Thread.currentThread().interrupt();
+            }
+        }
+
+        // Abort, not graceful close, since the latter will
+        // flush as well, and may add an entry.
+        writer.abort();
+
+        executionTime.stop();
+        assertTrue(!(Thread.interrupted()));
+        assertTrue(success);
+
+        LogRecordWithDLSN last = dlm.getLastLogRecord();
+        LOG.info("Last Entry {}; elapsed time {}", last.getDlsn().getEntryId(), executionTime.elapsed(TimeUnit.MILLISECONDS));
+
+        // Regardless of how many records we wrote; the number of BK entries should always be bounded by the min delay.
+        // Since there are two flush processes--data flush and control flush, and since control flush may also end up flushing
+        // data if data is available, the upper bound is 2*(time/min_delay + 1)
+        assertTrue(last.getDlsn().getEntryId() <= ((executionTime.elapsed(TimeUnit.MILLISECONDS) / confLocal.getMinDelayBetweenImmediateFlushMs() + 1))*2);
+        DLMTestUtil.verifyLogRecord(last);
+
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testAsyncWriteWithMinDelayBetweenFlushesFlushFailure() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setMinDelayBetweenImmediateFlushMs(1);
+
+        URI uri = createDLMURI("/" + name);
+        ensureURICreated(uri);
+
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(confLocal).uri(uri).clientId("gabbagoo").build();
+        DistributedLogManager dlm = namespace.openLog(name);
+        DistributedLogNamespace namespace1 = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(confLocal).uri(uri).clientId("tortellini").build();
+        DistributedLogManager dlm1 = namespace1.openLog(name);
+
+        int txid = 1;
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+
+        // First write succeeds since lock isnt checked until transmit, which is scheduled
+        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
+        writer.flushAndCommit();
+
+        BKLogSegmentWriter perStreamWriter = writer.getCachedLogWriter();
+        DistributedLock lock = perStreamWriter.getLock();
+        FutureUtils.result(lock.asyncClose());
+
+        // Get second writer, steal lock
+        BKAsyncLogWriter writer2 = (BKAsyncLogWriter)(dlm1.startAsyncLogSegmentNonPartitioned());
+
+        try {
+            // Succeeds, kicks off scheduked flush
+            writer.write(DLMTestUtil.getLogRecordInstance(txid++));
+
+            // Succeeds, kicks off scheduled flush
+            Thread.sleep(100);
+            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
+            fail("should have thrown");
+        } catch (LockingException ex) {
+            LOG.debug("caught exception ", ex);
+        }
+
+        writer.close();
+        dlm.close();
+    }
+
+    public void writeRecordsWithOutstandingWriteLimit(int stream, int global, boolean shouldFail) throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setPerWriterOutstandingWriteLimit(stream);
+        confLocal.setOutstandingWriteLimitDarkmode(false);
+        DistributedLogManager dlm;
+        if (global > -1) {
+            dlm = createNewDLM(confLocal, runtime.getMethodName(),
+                    new SimplePermitLimiter(false, global, new NullStatsLogger(), true, new FixedValueFeature("", 0)));
+        } else {
+            dlm = createNewDLM(confLocal, runtime.getMethodName());
+        }
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+        ArrayList<Future<DLSN>> results = new ArrayList<Future<DLSN>>(1000);
+        for (int i = 0; i < 1000; i++) {
+            results.add(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+        }
+        for (Future<DLSN> result : results) {
+            try {
+                Await.result(result);
+                if (shouldFail) {
+                    fail("should fail due to no outstanding writes permitted");
+                }
+            } catch (OverCapacityException ex) {
+                assertTrue(shouldFail);
+            }
+        }
+        writer.closeAndComplete();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testOutstandingWriteLimitNoLimit() throws Exception {
+        writeRecordsWithOutstandingWriteLimit(-1, -1, false);
+    }
+
+    @Test(timeout = 60000)
+    public void testOutstandingWriteLimitVeryHighLimit() throws Exception {
+        writeRecordsWithOutstandingWriteLimit(Integer.MAX_VALUE, Integer.MAX_VALUE, false);
+    }
+
+    @Test(timeout = 60000)
+    public void testOutstandingWriteLimitBlockAllStreamLimit() throws Exception {
+        writeRecordsWithOutstandingWriteLimit(0, Integer.MAX_VALUE, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testOutstandingWriteLimitBlockAllGlobalLimit() throws Exception {
+        writeRecordsWithOutstandingWriteLimit(Integer.MAX_VALUE, 0, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testOutstandingWriteLimitBlockAllLimitWithDarkmode() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setPerWriterOutstandingWriteLimit(0);
+        confLocal.setOutstandingWriteLimitDarkmode(true);
+        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+        ArrayList<Future<DLSN>> results = new ArrayList<Future<DLSN>>(1000);
+        for (int i = 0; i < 1000; i++) {
+            results.add(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+        }
+        for (Future<DLSN> result : results) {
+            Await.result(result);
+        }
+        writer.closeAndComplete();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseAndCompleteLogSegmentWhenStreamIsInError() throws Exception {
+        String name = "distrlog-close-and-complete-logsegment-when-stream-is-in-error";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+
+        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+
+        long txId = 1L;
+        for (int i = 0; i < 5; i++) {
+            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
+        }
+
+        BKLogSegmentWriter logWriter = writer.getCachedLogWriter();
+
+        BKNamespaceDriver driver = (BKNamespaceDriver) dlm.getNamespaceDriver();
+        // fence the ledger
+        driver.getReaderBKC().get().openLedger(logWriter.getLogSegmentId(),
+                BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
+
+        try {
+            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
+            fail("Should fail write to a fenced ledger with BKTransmitException");
+        } catch (BKTransmitException bkte) {
+            // expected
+        }
+
+        try {
+            writer.closeAndComplete();
+            fail("Should fail to complete a log segment when its ledger is fenced");
+        } catch (BKTransmitException bkte) {
+            // expected
+        }
+
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(1, segments.size());
+        assertTrue(segments.get(0).isInProgress());
+
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseAndCompleteLogSegmentWhenCloseFailed() throws Exception {
+        String name = "distrlog-close-and-complete-logsegment-when-close-failed";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+
+        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+
+        long txId = 1L;
+        for (int i = 0; i < 5; i++) {
+            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
+        }
+
+        BKLogSegmentWriter logWriter = writer.getCachedLogWriter();
+
+        BKNamespaceDriver driver = (BKNamespaceDriver) dlm.getNamespaceDriver();
+        // fence the ledger
+        driver.getReaderBKC().get().openLedger(logWriter.getLogSegmentId(),
+                BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
+
+        try {
+            // insert a write to detect the fencing state, to make test more robust.
+            writer.write(DLMTestUtil.getLogRecordInstance(txId++));
+            writer.closeAndComplete();
+            fail("Should fail to complete a log segment when its ledger is fenced");
+        } catch (IOException ioe) {
+            // expected
+            LOG.error("Failed to close and complete log segment {} : ", logWriter.getFullyQualifiedLogSegment(), ioe);
+        }
+
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(1, segments.size());
+        assertTrue(segments.get(0).isInProgress());
+
+        dlm.close();
+    }
+
+    private void testAsyncReadIdleErrorInternal(String name,
+                                                final int idleReaderErrorThreshold,
+                                                final boolean heartBeatUsingControlRecs,
+                                                final boolean simulateReaderStall) throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setReadAheadBatchSize(1);
+        confLocal.setReadAheadMaxRecords(1);
+        confLocal.setReaderIdleWarnThresholdMillis(0);
+        confLocal.setReaderIdleErrorThresholdMillis(idleReaderErrorThreshold);
+        final DistributedLogManager dlm = createNewDLM(confLocal, name);
+        final Thread currentThread = Thread.currentThread();
+        final int segmentSize = 3;
+        final int numSegments = 3;
+        final CountDownLatch latch = new CountDownLatch(1);
+        final ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
+        executor.schedule(
+            new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        int txid = 1;
+                        for (long i = 0; i < numSegments; i++) {
+                            long start = txid;
+                            BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+                            for (long j = 1; j <= segmentSize; j++) {
+                                writer.write(DLMTestUtil.getLargeLogRecordInstance(txid++));
+                                if ((i == 0) && (j == 1)) {
+                                    latch.countDown();
+                                }
+                            }
+
+                            if (heartBeatUsingControlRecs) {
+                                // There should be a control record such that
+                                // wait time + commit time (BK) < Idle Reader Threshold
+                                int threadSleepTime = idleReaderErrorThreshold
+                                    - 200 // BK commitTime
+                                    - 100; //safety margin
+
+                                for (int iter = 1; iter <= (2 * idleReaderErrorThreshold / threadSleepTime) ; iter++) {
+                                    Thread.sleep(threadSleepTime);
+                                    writer.write(DLMTestUtil.getLargeLogRecordInstance(txid, true));
+                                    writer.setReadyToFlush();
+                                }
+                                Thread.sleep(threadSleepTime);
+                            }
+
+                            writer.closeAndComplete();
+                            if (!heartBeatUsingControlRecs) {
+                                Thread.sleep(2 * idleReaderErrorThreshold);
+                            }
+                        }
+                    } catch (Exception exc) {
+                        if (!executor.isShutdown()) {
+                            currentThread.interrupt();
+                        }
+                    }
+                }
+            }, 0, TimeUnit.MILLISECONDS);
+
+        latch.await();
+        BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
+        if (simulateReaderStall) {
+            reader.disableProcessingReadRequests();
+        }
+        boolean exceptionEncountered = false;
+        int recordCount = 0;
+        try {
+            while (true) {
+                Future<LogRecordWithDLSN> record = reader.readNext();
+                Await.result(record);
+                recordCount++;
+
+                if (recordCount >= segmentSize * numSegments) {
+                    break;
+                }
+            }
+        } catch (IdleReaderException exc) {
+            exceptionEncountered = true;
+        }
+
+        if (simulateReaderStall) {
+            assertTrue(exceptionEncountered);
+        } else if (heartBeatUsingControlRecs) {
+            assertFalse(exceptionEncountered);
+            Assert.assertEquals(segmentSize * numSegments, recordCount);
+        } else {
+            assertTrue(exceptionEncountered);
+            Assert.assertEquals(segmentSize, recordCount);
+        }
+        assertFalse(currentThread.isInterrupted());
+        Utils.close(reader);
+        executor.shutdown();
+    }
+
+    @Test(timeout = 10000)
+    public void testAsyncReadIdleControlRecord() throws Exception {
+        String name = "distrlog-async-reader-idle-error-control";
+        testAsyncReadIdleErrorInternal(name, 500, true, false);
+    }
+
+    @Test(timeout = 10000)
+    public void testAsyncReadIdleError() throws Exception {
+        String name = "distrlog-async-reader-idle-error";
+        testAsyncReadIdleErrorInternal(name, 1000, false, false);
+    }
+
+    @Test(timeout = 10000)
+    public void testAsyncReadIdleError2() throws Exception {
+        String name = "distrlog-async-reader-idle-error-2";
+        testAsyncReadIdleErrorInternal(name, 1000, true, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testReleaseLockAfterFailedToRecover() throws Exception {
+        String name = "release-lock-after-failed-to-recover";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setLockTimeout(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setOutputBufferSize(0);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer =
+                (BKAsyncLogWriter)(dlm.startAsyncLogSegmentNonPartitioned());
+
+        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+        writer.abort();
+
+        for (int i = 0; i < 2; i++) {
+            FailpointUtils.setFailpoint(
+                    FailpointUtils.FailPointName.FP_RecoverIncompleteLogSegments,
+                    FailpointUtils.FailPointActions.FailPointAction_Throw);
+
+            try {
+                dlm.startAsyncLogSegmentNonPartitioned();
+                fail("Should fail during recovering incomplete log segments");
+            } catch (IOException ioe) {
+                // expected;
+            } finally {
+                FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_RecoverIncompleteLogSegments);
+            }
+        }
+
+        writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
+
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(1, segments.size());
+        assertFalse(segments.get(0).isInProgress());
+
+        writer.close();
+        dlm.close();
+    }
+
+    @DistributedLogAnnotations.FlakyTest
+    @Test(timeout = 60000)
+    public void testAsyncReadMissingLogSegmentsNotification() throws Exception {
+        String name = "distrlog-async-reader-missing-zk-notification";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setReadAheadBatchSize(1);
+        confLocal.setReadAheadMaxRecords(1);
+        confLocal.setReadLACLongPollTimeout(49);
+        confLocal.setReaderIdleWarnThresholdMillis(100);
+        confLocal.setReaderIdleErrorThresholdMillis(20000);
+        final DistributedLogManager dlm = createNewDLM(confLocal, name);
+        final Thread currentThread = Thread.currentThread();
+        final int segmentSize = 10;
+        final int numSegments = 3;
+        final CountDownLatch latch = new CountDownLatch(1);
+        final CountDownLatch readLatch = new CountDownLatch(1);
+        final ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
+        executor.schedule(
+                new Runnable() {
+                    @Override
+                    public void run() {
+                        try {
+                            int txid = 1;
+                            for (long i = 0; i < numSegments; i++) {
+                                BKSyncLogWriter writer = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+                                for (long j = 1; j <= segmentSize; j++) {
+                                    writer.write(DLMTestUtil.getLargeLogRecordInstance(txid++));
+                                    if ((i == 0) && (j == 1)) {
+                                        latch.countDown();
+                                    } else {
+                                        // wait for reader to start
+                                        readLatch.await();
+                                    }
+                                }
+                                writer.closeAndComplete();
+                                Thread.sleep(100);
+                            }
+                        } catch (Exception exc) {
+                            if (!executor.isShutdown()) {
+                                currentThread.interrupt();
+                            }
+                        }
+                    }
+                }, 0, TimeUnit.MILLISECONDS);
+
+        latch.await();
+        BKAsyncLogReader reader = (BKAsyncLogReader)dlm.getAsyncLogReader(DLSN.InitialDLSN);
+        reader.disableReadAheadLogSegmentsNotification();
+        boolean exceptionEncountered = false;
+        int recordCount = 0;
+        try {
+            while (true) {
+                Future<LogRecordWithDLSN> record = reader.readNext();
+                Await.result(record);
+                if (recordCount == 0) {
+                    readLatch.countDown();
+                }
+                recordCount++;
+
+                if (recordCount >= segmentSize * numSegments) {
+                    break;
+                }
+            }
+        } catch (IdleReaderException exc) {
+            exceptionEncountered = true;
+        }
+        assertTrue(!exceptionEncountered);
+        Assert.assertEquals(recordCount, segmentSize * numSegments);
+        assertTrue(!currentThread.isInterrupted());
+        Utils.close(reader);
+        executor.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLastTxId() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+
+        int numRecords = 10;
+        for (int i = 0; i < numRecords; i++) {
+            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
+            assertEquals("last tx id should become " + i,
+                    i, writer.getLastTxId());
+        }
+        // open a writer to recover the inprogress log segment
+        AsyncLogWriter recoverWriter = dlm.startAsyncLogSegmentNonPartitioned();
+        assertEquals("recovered last tx id should be " + (numRecords - 1),
+                numRecords - 1, recoverWriter.getLastTxId());
+    }
+
+    @Test(timeout = 60000)
+    public void testMaxReadAheadRecords() throws Exception {
+        int maxRecords = 1;
+        int batchSize = 8;
+        int maxAllowedCachedRecords = maxRecords + batchSize - 1;
+
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(Integer.MAX_VALUE);
+        confLocal.setReadAheadMaxRecords(maxRecords);
+        confLocal.setReadAheadBatchSize(batchSize);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+
+        int numRecords = 40;
+        for (int i = 1; i <= numRecords; i++) {
+            Await.result(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));
+
+        BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
+        record = Await.result(reader.readNext());
+        LOG.info("Read record {}", record);
+        assertEquals(1L, record.getTransactionId());
+
+        assertNotNull(reader.getReadAheadReader());
+        assertTrue(reader.getReadAheadReader().getNumCachedEntries() <= maxAllowedCachedRecords);
+
+        for (int i = 2; i <= numRecords; i++) {
+            record = Await.result(reader.readNext());
+            LOG.info("Read record {}", record);
+            assertEquals((long) i, record.getTransactionId());
+            TimeUnit.MILLISECONDS.sleep(20);
+            int numCachedEntries = reader.getReadAheadReader().getNumCachedEntries();
+            assertTrue("Should cache less than " + batchSize + " records but already found "
+                    + numCachedEntries + " records when reading " + i + "th record",
+                    numCachedEntries <= maxAllowedCachedRecords);
+        }
+        Utils.close(reader);
+    }
+
+    @Test(timeout = 60000)
+    public void testMarkEndOfStream() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
+
+        final int NUM_RECORDS = 10;
+        int i = 1;
+        for (; i <= NUM_RECORDS; i++) {
+            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
+            assertEquals("last tx id should become " + i,
+                    i, writer.getLastTxId());
+        }
+
+        Await.result(writer.markEndOfStream());
+
+        // Multiple end of streams are ok.
+        Await.result(writer.markEndOfStream());
+
+        try {
+            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
+            fail("Should have thrown");
+        } catch (EndOfStreamException ex) {
+        }
+
+        BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
+        LogRecord record = null;
+        for (int j = 0; j < NUM_RECORDS; j++) {
+            record = Await.result(reader.readNext());
+            assertEquals(j+1, record.getTransactionId());
+        }
+
+        try {
+            record = Await.result(reader.readNext());
+            fail("Should have thrown");
+        } catch (EndOfStreamException ex) {
+        }
+        Utils.close(reader);
+    }
+
+    @Test(timeout = 60000)
+    public void testMarkEndOfStreamAtBeginningOfSegment() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
+        Await.result(writer.markEndOfStream());
+        try {
+            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(1)));
+            fail("Should have thrown");
+        } catch (EndOfStreamException ex) {
+        }
+        writer.close();
+
+        BKAsyncLogReader reader = (BKAsyncLogReader) dlm.getAsyncLogReader(DLSN.InitialDLSN);
+        try {
+            LogRecord record = Await.result(reader.readNext());
+            fail("Should have thrown");
+        } catch (EndOfStreamException ex) {
+        }
+        Utils.close(reader);
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkReadWaitingMoreRecords() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
+        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+        LogRecord controlRecord = DLMTestUtil.getLogRecordInstance(1L);
+        controlRecord.setControl();
+        FutureUtils.result(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();
+
+        // write another records
+        for (int i = 0; i < 5; i++) {
+            long txid = 2L + i;
+            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txid)));
+            controlRecord = DLMTestUtil.getLogRecordInstance(txid);
+            controlRecord.setControl();
+            FutureUtils.result(writer.write(controlRecord));
+        }
+
+        List<LogRecordWithDLSN> bulkReadRecords = FutureUtils.result(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);
+        assertEquals(3L, record.getTransactionId());
+        DLMTestUtil.verifyLogRecord(record);
+
+        Utils.close(reader);
+        writer.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkReadNotWaitingMoreRecords() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
+        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+        LogRecord controlRecord = DLMTestUtil.getLogRecordInstance(1L);
+        controlRecord.setControl();
+        FutureUtils.result(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();
+
+        List<LogRecordWithDLSN> bulkReadRecords = FutureUtils.result(bulkReadFuture);
+        assertEquals(1, bulkReadRecords.size());
+        assertEquals(1L, bulkReadRecords.get(0).getTransactionId());
+        for (LogRecordWithDLSN record : bulkReadRecords) {
+            DLMTestUtil.verifyLogRecord(record);
+        }
+
+        // write another records
+        for (int i = 0; i < 5; i++) {
+            long txid = 2L + i;
+            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txid)));
+            controlRecord = DLMTestUtil.getLogRecordInstance(txid);
+            controlRecord.setControl();
+            FutureUtils.result(writer.write(controlRecord));
+        }
+
+        LogRecordWithDLSN record = FutureUtils.result(readFuture);
+        assertEquals(2L, record.getTransactionId());
+        DLMTestUtil.verifyLogRecord(record);
+
+        Utils.close(reader);
+        writer.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReadBrokenEntries() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadBatchSize(1);
+        confLocal.setPositionGapDetectionEnabled(false);
+        confLocal.setReadAheadSkipBrokenEntries(true);
+        confLocal.setEIInjectReadAheadBrokenEntries(true);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        int numLogSegments = 3;
+        int numRecordsPerLogSegment = 10;
+
+        long txid = 1L;
+        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false);
+
+        AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
+
+        // 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());
+            assertFalse(record.getDlsn().getEntryId() % 10 == 0);
+        }
+
+        Utils.close(reader);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReadBrokenEntriesWithGapDetection() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadBatchSize(1);
+        confLocal.setPositionGapDetectionEnabled(true);
+        confLocal.setReadAheadSkipBrokenEntries(true);
+        confLocal.setEIInjectReadAheadBrokenEntries(true);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        int numLogSegments = 1;
+        int numRecordsPerLogSegment = 100;
+
+        long txid = 1L;
+        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false);
+
+        AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
+
+        try {
+            // 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());
+                assertFalse(record.getDlsn().getEntryId() % 10 == 0);
+            }
+            fail("should have thrown");
+        } catch (DLIllegalStateException e) {
+        }
+
+        Utils.close(reader);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReadBrokenEntriesAndLargeBatchSize() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadBatchSize(5);
+        confLocal.setPositionGapDetectionEnabled(false);
+        confLocal.setReadAheadSkipBrokenEntries(true);
+        confLocal.setEIInjectReadAheadBrokenEntries(true);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        int numLogSegments = 1;
+        int numRecordsPerLogSegment = 100;
+
+        long txid = 1L;
+        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false);
+
+        AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
+
+        // Every 10th record broken. Reading 5 at once, beginning from 0:
+        // 1. range 0-4 will be corrupted and discarded
+        // 2. ranges 1-5, 2-6, 3-7, 4-8, 5-9 will be ok
+        // 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());
+            assertFalse(record.getDlsn().getEntryId() % 10 == 0);
+        }
+
+        Utils.close(reader);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReadBrokenEntriesAndLargeBatchSizeCrossSegment() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(testConf);
+
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setReadAheadWaitTime(10);
+        confLocal.setReadAheadBatchSize(8);
+        confLocal.setPositionGapDetectionEnabled(false);
+        confLocal.setReadAheadSkipBrokenEntries(true);
+        confLocal.setEIInjectReadAheadBrokenEntries(true);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+
+        int numLogSegments = 3;
+        int numRecordsPerLogSegment = 5;
+
+        long txid = 1L;
+        txid = writeRecords(dlm, numLogSegments, numRecordsPerLogSegment, txid, false);
+
+        AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
+
+        // Every 10th record broken. Reading 8 at once, beginning from 0:
+        // 1. range 0-7 will be corrupted and discarded
+        // 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());
+            assertFalse(record.getDlsn().getEntryId() % 10 == 0);
+        }
+
+        Utils.close(reader);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogStreamWithDifferentReplicationFactor() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+
+        ConcurrentBaseConfiguration baseConf = new ConcurrentConstConfiguration(confLocal);
+        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(baseConf);
+        dynConf.setProperty(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE,
+                DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT - 1);
+
+        URI uri = createDLMURI("/" + name);
+        ensureURICreated(uri);
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.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)));
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(1, segments.size());
+        long ledgerId = segments.get(0).getLogSegmentId();
+        LedgerHandle lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC()
+                .get().openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
+        LedgerMetadata metadata = BookKeeperAccessor.getLedgerMetadata(lh);
+        assertEquals(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT, metadata.getEnsembleSize());
+        lh.close();
+        Utils.close(writer);
+        dlm.close();
+
+        // use customized configuration
+        dlm = namespace.openLog(
+                name + "-custom",
+                Optional.<DistributedLogConfiguration>absent(),
+                Optional.of(dynConf),
+                Optional.<StatsLogger>absent());
+        writer = dlm.startAsyncLogSegmentNonPartitioned();
+        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+        segments = dlm.getLogSegments();
+        assertEquals(1, segments.size());
+        ledgerId = segments.get(0).getLogSegmentId();
+        lh = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC()
+                .get().openLedgerNoRecovery(ledgerId, BookKeeper.DigestType.CRC32, confLocal.getBKDigestPW().getBytes(UTF_8));
+        metadata = BookKeeperAccessor.getLedgerMetadata(lh);
+        assertEquals(DistributedLogConfiguration.BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT - 1, metadata.getEnsembleSize());
+        lh.close();
+        Utils.close(writer);
+        dlm.close();
+        namespace.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteRecordSet() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+
+        URI uri = createDLMURI("/" + name);
+        ensureURICreated(uri);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
+        List<Future<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();
+        // 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>();
+            recordSetWriter.writeRecord(ByteBuffer.wrap(record.getPayload()), writePromise);
+            recordSetFutures.add(writePromise);
+        }
+        final ByteBuffer recordSetBuffer = recordSetWriter.getBuffer();
+        byte[] data = new byte[recordSetBuffer.remaining()];
+        recordSetBuffer.get(data);
+        LogRecord setRecord = new LogRecord(6L, data);
+        setRecord.setRecordSet();
+        Future<DLSN> writeRecordSetFuture = writer.write(setRecord);
+        writeRecordSetFuture.addEventListener(new FutureEventListener<DLSN>() {
+            @Override
+            public void onSuccess(DLSN dlsn) {
+                recordSetWriter.completeTransmit(
+                        dlsn.getLogSegmentSequenceNo(),
+                        dlsn.getEntryId(),
+                        dlsn.getSlotId());
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                recordSetWriter.abortTransmit(cause);
+            }
+        });
+        writeFutures.add(writeRecordSetFuture);
+        FutureUtils.result(writeRecordSetFuture);
+        // write last 5 records
+        for (int i = 0; i < 5; i++) {
+            LogRecord record = DLMTestUtil.getLogRecordInstance(11L + i);
+            Future<DLSN> writeFuture = writer.write(record);
+            writeFutures.add(writeFuture);
+            // make sure get log record count returns the right count
+            if (i == 0) {
+                FutureUtils.result(writeFuture);
+                assertEquals(10, dlm.getLogRecordCount());
+            }
+        }
+
+        List<DLSN> writeResults = FutureUtils.result(Future.collect(writeFutures));
+
+        for (int i = 0; i < 5; i++) {
+            Assert.assertEquals(new DLSN(1L, i, 0L), writeResults.get(i));
+        }
+        Assert.assertEquals(new DLSN(1L, 5L, 0L), writeResults.get(5));
+        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));
+        for (int i = 0; i < 5; i++) {
+            Assert.assertEquals(new DLSN(1L, 5L, i), recordSetWriteResults.get(i));
+        }
+
+        FutureUtils.result(writer.flushAndCommit());
+
+        DistributedLogConfiguration readConf1 = new DistributedLogConfiguration();
+        readConf1.addConfiguration(confLocal);
+        readConf1.setDeserializeRecordSetOnReads(true);
+
+        DistributedLogManager readDLM1 = createNewDLM(readConf1, name);
+        AsyncLogReader reader1 = readDLM1.getAsyncLogReader(DLSN.InitialDLSN);
+        for (int i = 0; i < 15; i++) {
+            LogRecordWithDLSN record = FutureUtils.result(reader1.readNext());
+            if (i < 5) {
+                assertEquals(new DLSN(1L, i, 0L), record.getDlsn());
+                assertEquals(1L + i, record.getTransactionId());
+            } else if (i >= 10) {
+                assertEquals(new DLSN(1L, 6L + i - 10, 0L), record.getDlsn());
+                assertEquals(11L + i - 10, record.getTransactionId());
+            } else {
+                assertEquals(new DLSN(1L, 5L, i - 5), record.getDlsn());
+                assertEquals(6L, record.getTransactionId());
+            }
+            assertEquals(i+1, record.getPositionWithinLogSegment());
+            assertArrayEquals(DLMTestUtil.generatePayload(i+1), record.getPayload());
+        }
+        Utils.close(reader1);
+        readDLM1.close();
+
+        DistributedLogConfiguration readConf2 = new DistributedLogConfiguration();
+        readConf2.addConfiguration(confLocal);
+        readConf2.setDeserializeRecordSetOnReads(false);
+
+        DistributedLogManager readDLM2 = createNewDLM(readConf2, name);
+        AsyncLogReader reader2 = readDLM2.getAsyncLogReader(DLSN.InitialDLSN);
+        for (int i = 0; i < 11; i++) {
+            LogRecordWithDLSN record = FutureUtils.result(reader2.readNext());
+            LOG.info("Read record {}", record);
+            if (i < 5) {
+                assertEquals(new DLSN(1L, i, 0L), record.getDlsn());
+                assertEquals(1L + i, record.getTransactionId());
+                assertEquals(i + 1, record.getPositionWithinLogSegment());
+                assertArrayEquals(DLMTestUtil.generatePayload(i+1), record.getPayload());
+            } else if (i >= 6L) {
+                assertEquals(new DLSN(1L, 6L + i - 6, 0L), record.getDlsn());
+                assertEquals(11L + i - 6, record.getTransactionId());
+                assertEquals(11 + i - 6, record.getPositionWithinLogSegment());
+                assertArrayEquals(DLMTestUtil.generatePayload(11L + i - 6), record.getPayload());
+            } else {
+                assertEquals(new DLSN(1L, 5L, 0), record.getDlsn());
+                assertEquals(6L, record.getTransactionId());
+                assertEquals(6, record.getPositionWithinLogSegment());
+                assertTrue(record.isRecordSet());
+                assertEquals(5, LogRecordSet.numRecords(record));
+            }
+        }
+        Utils.close(reader2);
+        readDLM2.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testIdleReaderExceptionWhenKeepAliveIsDisabled() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setPeriodicKeepAliveMilliSeconds(0);
+        confLocal.setReadLACLongPollTimeout(9);
+        confLocal.setReaderIdleWarnThresholdMillis(20);
+        confLocal.setReaderIdleErrorThresholdMillis(40);
+
+        URI uri = createDLMURI("/" + name);
+        ensureURICreated(uri);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) FutureUtils.result(dlm.openAsyncLogWriter());
+        writer.write(DLMTestUtil.getLogRecordInstance(1L));
+
+        AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(DLSN.InitialDLSN));
+        try {
+            FutureUtils.result(reader.readNext());
+            fail("Should fail when stream is idle");
+        } catch (IdleReaderException ire) {
+            // expected
+        }
+        Utils.close(reader);
+        writer.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testIdleReaderExceptionWhenKeepAliveIsEnabled() throws Exception {
+        String name = runtime.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(testConf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setPeriodicKeepAliveMilliSeconds(1000);
+        confLocal.setReadLACLongPollTimeout(999);
+        confLocal.setReaderIdleWarnThresholdMillis(2000);
+        confLocal.setReaderIdleErrorThresholdMillis(4000);
+
+        URI uri = createDLMURI("/" + name);
+        ensureURICreated(uri);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) FutureUtils.result(dlm.openAsyncLogWriter());
+        writer.write(DLMTestUtil.getLogRecordInstance(1L));
+
+        AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(DLSN.InitialDLSN));
+        LogRecordWithDLSN record = FutureUtils.result(reader.readNext());
+        assertEquals(1L, record.getTransactionId());
+        DLMTestUtil.verifyLogRecord(record);
+
+        Utils.close(reader);
+        writer.close();
+        dlm.close();
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java
new file mode 100644
index 0000000..dff0133
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogManager.java
@@ -0,0 +1,1201 @@
+/**
+ * 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 java.net.URI;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import org.apache.distributedlog.exceptions.AlreadyTruncatedTransactionException;
+import org.apache.distributedlog.exceptions.BKTransmitException;
+import org.apache.distributedlog.exceptions.LogEmptyException;
+import org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.apache.distributedlog.exceptions.LogReadException;
+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;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.distributedlog.callback.LogSegmentListener;
+import org.apache.distributedlog.exceptions.EndOfStreamException;
+import org.apache.distributedlog.exceptions.InvalidStreamNameException;
+import org.apache.distributedlog.exceptions.LogRecordTooLongException;
+import org.apache.distributedlog.exceptions.TransactionIdOutOfOrderException;
+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 static org.junit.Assert.*;
+import static org.junit.Assert.assertEquals;
+
+public class TestBKDistributedLogManager extends TestDistributedLogBase {
+    static final Logger LOG = LoggerFactory.getLogger(TestBKDistributedLogManager.class);
+
+    private static final Random RAND = new Random(System.currentTimeMillis());
+
+    @Rule
+    public TestName testNames = new TestName();
+
+    private static final long DEFAULT_SEGMENT_SIZE = 1000;
+
+    private void testNonPartitionedWritesInternal(String name, DistributedLogConfiguration conf) throws Exception {
+        BKDistributedLogManager dlm = createNewDLM(conf, name);
+
+        long txid = 1;
+        for (long i = 0; i < 3; i++) {
+            long start = txid;
+            BKSyncLogWriter writer = dlm.startLogSegmentNonPartitioned();
+            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
+                writer.write(DLMTestUtil.getLogRecordInstance(txid++));
+            }
+            BKLogSegmentWriter perStreamLogWriter = writer.getCachedLogWriter();
+            writer.closeAndComplete();
+            BKLogWriteHandler blplm = dlm.createWriteHandler(true);
+            assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
+                                                                perStreamLogWriter.getLogSegmentSequenceNumber()), false));
+            FutureUtils.result(blplm.asyncClose());
+        }
+
+        LogWriter writer = dlm.startLogSegmentNonPartitioned();
+        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
+            writer.write(DLMTestUtil.getLogRecordInstance(txid++));
+        }
+        writer.setReadyToFlush();
+        writer.flushAndSync();
+        writer.close();
+
+        LogReader reader = dlm.getInputStream(1);
+        long numTrans = 0;
+        LogRecord record = reader.readNext(false);
+        long lastTxId = -1;
+        while (null != record) {
+            DLMTestUtil.verifyLogRecord(record);
+            assert (lastTxId < record.getTransactionId());
+            lastTxId = record.getTransactionId();
+            numTrans++;
+            record = reader.readNext(false);
+        }
+        reader.close();
+        assertEquals((txid - 1), numTrans);
+    }
+
+    @Test(timeout = 60000)
+    public void testSimpleWrite() throws Exception {
+        BKDistributedLogManager dlm = createNewDLM(conf, "distrlog-simplewrite");
+        BKSyncLogWriter out = dlm.startLogSegmentNonPartitioned();
+        for (long i = 1; i <= 100; i++) {
+            LogRecord op = DLMTestUtil.getLogRecordInstance(i);
+            out.write(op);
+        }
+        BKLogSegmentWriter perStreamLogWriter = out.getCachedLogWriter();
+        out.closeAndComplete();
+
+        BKLogWriteHandler blplm = dlm.createWriteHandler(true);
+        assertNotNull(zkc.exists(blplm.completedLedgerZNode(1, 100,
+                perStreamLogWriter.getLogSegmentSequenceNumber()), false));
+        FutureUtils.result(blplm.asyncClose());
+    }
+
+    @Test(timeout = 60000)
+    public void testNumberOfTransactions() throws Exception {
+        String name = "distrlog-txncount";
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+        for (long i = 1; i <= 100; i++) {
+            LogRecord op = DLMTestUtil.getLogRecordInstance(i);
+            out.write(op);
+        }
+        out.closeAndComplete();
+
+        long numTrans = DLMTestUtil.getNumberofLogRecords(createNewDLM(conf, name), 1);
+        assertEquals(100, numTrans);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testContinuousReaders() throws Exception {
+        String name = "distrlog-continuous";
+        BKDistributedLogManager dlm = createNewDLM(conf, name);
+        long txid = 1;
+        for (long i = 0; i < 3; i++) {
+            long start = txid;
+            BKSyncLogWriter out = dlm.startLogSegmentNonPartitioned();
+            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
+                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+                out.write(op);
+            }
+            BKLogSegmentWriter perStreamLogWriter = out.getCachedLogWriter();
+            out.closeAndComplete();
+            BKLogWriteHandler blplm = dlm.createWriteHandler(true);
+
+            assertNotNull(
+                zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
+                                                      perStreamLogWriter.getLogSegmentSequenceNumber()), false));
+            FutureUtils.result(blplm.asyncClose());
+        }
+
+        BKSyncLogWriter out = dlm.startLogSegmentNonPartitioned();
+        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
+            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+            out.write(op);
+        }
+        out.setReadyToFlush();
+        out.flushAndSync();
+        out.close();
+        dlm.close();
+
+        dlm = createNewDLM(conf, name);
+
+        LogReader reader = dlm.getInputStream(1);
+        long numTrans = 0;
+        LogRecord record = reader.readNext(false);
+        while (null != record) {
+            DLMTestUtil.verifyLogRecord(record);
+            numTrans++;
+            record = reader.readNext(false);
+        }
+        assertEquals((txid - 1), numTrans);
+        assertEquals(txid - 1, dlm.getLogRecordCount());
+        reader.close();
+        dlm.close();
+    }
+
+    /**
+     * Create a bkdlm namespace, write a journal from txid 1, close stream.
+     * Try to create a new journal from txid 1. Should throw an exception.
+     */
+    @Test(timeout = 60000)
+    public void testWriteRestartFrom1() throws Exception {
+        DistributedLogManager dlm = createNewDLM(conf, "distrlog-restartFrom1");
+        long txid = 1;
+        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
+            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+            out.write(op);
+        }
+        out.closeAndComplete();
+
+        txid = 1;
+        try {
+            out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+            out.write(DLMTestUtil.getLogRecordInstance(txid));
+            fail("Shouldn't be able to start another journal from " + txid
+                + " when one already exists");
+        } catch (Exception ioe) {
+            LOG.info("Caught exception as expected", ioe);
+        } finally {
+            out.close();
+        }
+
+        // test border case
+        txid = DEFAULT_SEGMENT_SIZE - 1;
+        try {
+            out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+            out.write(DLMTestUtil.getLogRecordInstance(txid));
+            fail("Shouldn't be able to start another journal from " + txid
+                + " when one already exists");
+        } catch (TransactionIdOutOfOrderException rste) {
+            LOG.info("Caught exception as expected", rste);
+        } finally {
+            out.close();
+        }
+
+        // open journal continuing from before
+        txid = DEFAULT_SEGMENT_SIZE + 1;
+        out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+        assertNotNull(out);
+
+        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
+            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+            out.write(op);
+        }
+        out.closeAndComplete();
+
+        // open journal arbitarily far in the future
+        txid = DEFAULT_SEGMENT_SIZE * 4;
+        out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+        out.write(DLMTestUtil.getLogRecordInstance(txid));
+        out.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testTwoWritersOnLockDisabled() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        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)));
+
+        // write a record to writer 1 again
+        try {
+            FutureUtils.result(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());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testSimpleRead() throws Exception {
+        String name = "distrlog-simpleread";
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        final long numTransactions = 10000;
+        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+        for (long i = 1; i <= numTransactions; i++) {
+            LogRecord op = DLMTestUtil.getLogRecordInstance(i);
+            out.write(op);
+        }
+        out.closeAndComplete();
+
+        assertEquals(numTransactions, DLMTestUtil.getNumberofLogRecords(createNewDLM(conf, name), 1));
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testNumberOfTransactionsWithInprogressAtEnd() throws Exception {
+        String name = "distrlog-inprogressAtEnd";
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        long txid = 1;
+        for (long i = 0; i < 3; i++) {
+            long start = txid;
+            BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
+                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+                out.write(op);
+            }
+            BKLogSegmentWriter perStreamLogWriter = out.getCachedLogWriter();
+            out.closeAndComplete();
+            BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
+            assertNotNull(
+                zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
+                                                      perStreamLogWriter.getLogSegmentSequenceNumber()), false));
+            FutureUtils.result(blplm.asyncClose());
+        }
+        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
+            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+            out.write(op);
+        }
+        out.setReadyToFlush();
+        out.flushAndSync();
+        out.close();
+
+        long numTrans = DLMTestUtil.getNumberofLogRecords(createNewDLM(conf, name), 1);
+        assertEquals((txid - 1), numTrans);
+    }
+
+    @Test(timeout = 60000)
+    public void testContinuousReaderBulk() throws Exception {
+        String name = "distrlog-continuous-bulk";
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        long txid = 1;
+        for (long i = 0; i < 3; i++) {
+            BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
+                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+                out.write(op);
+            }
+            out.closeAndComplete();
+        }
+
+        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
+            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+            out.write(op);
+        }
+        out.setReadyToFlush();
+        out.flushAndSync();
+        out.close();
+        dlm.close();
+
+        dlm = createNewDLM(conf, name);
+
+        LogReader reader = dlm.getInputStream(1);
+        long numTrans = 0;
+        List<LogRecordWithDLSN> recordList = reader.readBulk(false, 13);
+        long lastTxId = -1;
+        while (!recordList.isEmpty()) {
+            for (LogRecord record : recordList) {
+                assert (lastTxId < record.getTransactionId());
+                lastTxId = record.getTransactionId();
+                DLMTestUtil.verifyLogRecord(record);
+                numTrans++;
+            }
+            recordList = reader.readBulk(false, 13);
+        }
+        reader.close();
+        assertEquals((txid - 1), numTrans);
+    }
+
+    @Test(timeout = 60000)
+    public void testContinuousReadersWithEmptyLedgers() throws Exception {
+        String name = "distrlog-continuous-emptyledgers";
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        long txid = 1;
+        for (long i = 0; i < 3; i++) {
+            long start = txid;
+            BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
+                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+                out.write(op);
+            }
+            BKLogSegmentWriter writer = out.getCachedLogWriter();
+            out.closeAndComplete();
+            BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
+
+            assertNotNull(
+                zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
+                                                      writer.getLogSegmentSequenceNumber()), false));
+            BKLogSegmentWriter perStreamLogWriter = blplm.startLogSegment(txid - 1);
+            blplm.completeAndCloseLogSegment(perStreamLogWriter.getLogSegmentSequenceNumber(),
+                    perStreamLogWriter.getLogSegmentId(), txid - 1, txid - 1, 0);
+            assertNotNull(
+                zkc.exists(blplm.completedLedgerZNode(txid - 1, txid - 1,
+                                                      perStreamLogWriter.getLogSegmentSequenceNumber()), false));
+            FutureUtils.result(blplm.asyncClose());
+        }
+
+        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
+            LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+            out.write(op);
+        }
+        out.setReadyToFlush();
+        out.flushAndSync();
+        out.close();
+        dlm.close();
+
+        dlm = createNewDLM(conf, name);
+
+        AsyncLogReader asyncreader = dlm.getAsyncLogReader(DLSN.InvalidDLSN);
+        long numTrans = 0;
+        LogRecordWithDLSN record = Await.result(asyncreader.readNext());
+        while (null != record) {
+            DLMTestUtil.verifyLogRecord(record);
+            numTrans++;
+            if (numTrans >= (txid - 1)) {
+                break;
+            }
+            record = Await.result(asyncreader.readNext());
+        }
+        assertEquals((txid - 1), numTrans);
+        Utils.close(asyncreader);
+
+        LogReader reader = dlm.getInputStream(1);
+        numTrans = 0;
+        record = reader.readNext(false);
+        while (null != record) {
+            DLMTestUtil.verifyLogRecord(record);
+            numTrans++;
+            record = reader.readNext(false);
+        }
+        assertEquals((txid - 1), numTrans);
+        reader.close();
+        assertEquals(txid - 1, dlm.getLogRecordCount());
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testNonPartitionedWrites() throws Exception {
+        String name = "distrlog-non-partitioned-bulk";
+        testNonPartitionedWritesInternal(name, conf);
+    }
+
+    @Test(timeout = 60000)
+    public void testCheckLogExists() throws Exception {
+        String name = "distrlog-check-log-exists";
+        DistributedLogManager dlm = createNewDLM(conf, name);
+
+        long txid = 1;
+        LogWriter writer = dlm.startLogSegmentNonPartitioned();
+        for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
+            writer.write(DLMTestUtil.getLogRecordInstance(txid++));
+        }
+        writer.setReadyToFlush();
+        writer.flushAndSync();
+        writer.close();
+        dlm.close();
+
+        URI uri = createDLMURI("/" + name);
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.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()
+                .conf(conf).uri(nonExistentUri).build();
+        assertFalse(nonExistentNS.logExists(name));
+
+        int logCount = 0;
+        Iterator<String> logIter = namespace.getLogs();
+        while(logIter.hasNext()) {
+            String log = logIter.next();
+            logCount++;
+            assertEquals(name, log);
+        }
+        assertEquals(1, logCount);
+
+        namespace.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testMetadataAccessor() throws Exception {
+        String name = "distrlog-metadata-accessor";
+        MetadataAccessor metadata = DLMTestUtil.createNewMetadataAccessor(conf, name, createDLMURI("/" + name));
+        assertEquals(name, metadata.getStreamName());
+        metadata.createOrUpdateMetadata(name.getBytes());
+        assertEquals(name, new String(metadata.getMetadata()));
+        metadata.deleteMetadata();
+        assertEquals(null, metadata.getMetadata());
+    }
+
+    @Test(timeout = 60000)
+    public void testSubscriptionsStore() throws Exception {
+        String name = "distrlog-subscriptions-store";
+        String subscriber0 = "subscriber-0";
+        String subscriber1 = "subscriber-1";
+        String subscriber2 = "subscriber-2";
+
+        DLSN commitPosition0 = new DLSN(4, 33, 5);
+        DLSN commitPosition1 = new DLSN(4, 34, 5);
+        DLSN commitPosition2 = new DLSN(5, 34, 5);
+        DLSN commitPosition3 = new DLSN(6, 35, 6);
+
+        DistributedLogManager dlm = createNewDLM(conf, name);
+
+        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);
+        // empty
+        assertTrue(Await.result(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());
+        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());
+        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());
+        assertEquals(3, committedPositions.size());
+        assertEquals(commitPosition0, committedPositions.get(subscriber0));
+        assertEquals(commitPosition1, committedPositions.get(subscriber1));
+        assertEquals(commitPosition2, committedPositions.get(subscriber2));
+
+        // subscriber 2 advance again
+        DistributedLogManager newDLM = createNewDLM(conf, name);
+        SubscriptionsStore newStore = newDLM.getSubscriptionsStore();
+        Await.result(newStore.advanceCommitPosition(subscriber2, commitPosition3));
+        newStore.close();
+        newDLM.close();
+
+        committedPositions = Await.result(store.getLastCommitPositions());
+        assertEquals(3, committedPositions.size());
+        assertEquals(commitPosition0, committedPositions.get(subscriber0));
+        assertEquals(commitPosition1, committedPositions.get(subscriber1));
+        assertEquals(commitPosition3, committedPositions.get(subscriber2));
+
+        dlm.close();
+
+    }
+
+    private long writeAndMarkEndOfStream(DistributedLogManager dlm, long txid) throws Exception {
+        for (long i = 0; i < 3; i++) {
+            long start = txid;
+            BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
+                writer.write(DLMTestUtil.getLogRecordInstance(txid++));
+            }
+
+            BKLogSegmentWriter perStreamLogWriter = writer.getCachedLogWriter();
+
+            if (i < 2) {
+                writer.closeAndComplete();
+                BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
+                assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
+                                                                    perStreamLogWriter.getLogSegmentSequenceNumber()), false));
+                FutureUtils.result(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());
+            }
+        }
+        return txid;
+    }
+
+    @Test(timeout = 60000)
+    public void testMarkEndOfStream() throws Exception {
+        String name = "distrlog-mark-end-of-stream";
+        DistributedLogManager dlm = createNewDLM(conf, name);
+
+        long txid = 1;
+        txid = writeAndMarkEndOfStream(dlm, txid);
+
+        LogReader reader = dlm.getInputStream(1);
+        long numTrans = 0;
+        boolean exceptionEncountered = false;
+        LogRecord record = null;
+        try {
+            record = reader.readNext(false);
+            long expectedTxId = 1;
+            while (null != record) {
+                DLMTestUtil.verifyLogRecord(record);
+                assertEquals(expectedTxId, record.getTransactionId());
+                expectedTxId++;
+                numTrans++;
+                record = reader.readNext(false);
+            }
+        } catch (EndOfStreamException exc) {
+            LOG.info("Encountered EndOfStream on reading records after {}", record);
+            exceptionEncountered = true;
+        }
+        assertEquals((txid - 1), numTrans);
+        assertTrue(exceptionEncountered);
+        exceptionEncountered = false;
+        try {
+            reader.readNext(false);
+        } catch (EndOfStreamException exc) {
+            exceptionEncountered = true;
+        }
+        assertTrue(exceptionEncountered);
+        reader.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteFailsAfterMarkEndOfStream() throws Exception {
+        String name = "distrlog-mark-end-failure";
+        DistributedLogManager dlm = createNewDLM(conf, name);
+
+        long txid = 1;
+        txid = writeAndMarkEndOfStream(dlm, txid);
+
+        assertEquals(txid - 1, dlm.getLastTxId());
+        LogRecord last = dlm.getLastLogRecord();
+        assertEquals(txid - 1, last.getTransactionId());
+        DLMTestUtil.verifyLogRecord(last);
+        assertTrue(dlm.isEndOfStreamMarked());
+
+        LogWriter writer = null;
+        boolean exceptionEncountered = false;
+        try {
+            writer = dlm.startLogSegmentNonPartitioned();
+            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE / 2; j++) {
+                writer.write(DLMTestUtil.getLogRecordInstance(txid++));
+            }
+        } catch (EndOfStreamException exc) {
+            exceptionEncountered = true;
+        }
+        writer.close();
+        assertTrue(exceptionEncountered);
+    }
+
+    @Test(timeout = 60000)
+    public void testMarkEndOfStreamOnEmptyStream() throws Exception {
+        markEndOfStreamOnEmptyLogSegment(0);
+    }
+
+    @Test(timeout = 60000)
+    public void testMarkEndOfStreamOnClosedStream() throws Exception {
+        markEndOfStreamOnEmptyLogSegment(3);
+    }
+
+    private void markEndOfStreamOnEmptyLogSegment(int numCompletedSegments) throws Exception {
+        String name = "distrlog-mark-end-empty-" + numCompletedSegments;
+
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        DLMTestUtil.generateCompletedLogSegments(dlm, conf, numCompletedSegments, DEFAULT_SEGMENT_SIZE);
+
+        BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+        writer.markEndOfStream();
+
+        LogReader reader = dlm.getInputStream(1);
+        long numTrans = 0;
+        boolean exceptionEncountered = false;
+        try {
+            LogRecord record = reader.readNext(false);
+            long lastTxId = -1;
+            while (null != record) {
+                DLMTestUtil.verifyLogRecord(record);
+                assert (lastTxId < record.getTransactionId());
+                lastTxId = record.getTransactionId();
+                numTrans++;
+                record = reader.readNext(false);
+            }
+        } catch (EndOfStreamException exc) {
+            exceptionEncountered = true;
+        }
+        assertEquals(numCompletedSegments * DEFAULT_SEGMENT_SIZE, numTrans);
+        assertTrue(exceptionEncountered);
+        exceptionEncountered = false;
+        try {
+            reader.readNext(false);
+        } catch (EndOfStreamException exc) {
+            exceptionEncountered = true;
+        }
+        assertTrue(exceptionEncountered);
+        reader.close();
+    }
+
+    @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(
+                                DLMTestUtil.repeatString("abcdefgh", 256), 512).getBytes())));
+    }
+
+    @Test(timeout = 60000)
+    public void testMaxTransmissionSize() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setOutputBufferSize(1024 * 1024);
+        BKDistributedLogManager dlm =
+                createNewDLM(confLocal, "distrlog-transmissionSize");
+        AsyncLogWriter out = FutureUtils.result(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);
+            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);
+        } catch (LogRecordTooLongException exc) {
+            exceptionEncountered = true;
+        } finally {
+            FutureUtils.result(out.asyncClose());
+        }
+        assertFalse(exceptionEncountered);
+        Abortables.abortQuietly(out);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void deleteDuringRead() throws Exception {
+        String name = "distrlog-delete-with-reader";
+        DistributedLogManager dlm = createNewDLM(conf, name);
+
+        long txid = 1;
+        for (long i = 0; i < 3; i++) {
+            long start = txid;
+            BKSyncLogWriter writer = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
+                writer.write(DLMTestUtil.getLogRecordInstance(txid++));
+            }
+
+            BKLogSegmentWriter perStreamLogWriter = writer.getCachedLogWriter();
+
+            writer.closeAndComplete();
+            BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
+            assertNotNull(zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
+                                                                perStreamLogWriter.getLogSegmentSequenceNumber()), false));
+            FutureUtils.result(blplm.asyncClose());
+        }
+
+        LogReader reader = dlm.getInputStream(1);
+        long numTrans = 1;
+        LogRecord record = reader.readNext(false);
+        assert (null != record);
+        DLMTestUtil.verifyLogRecord(record);
+        long lastTxId = record.getTransactionId();
+
+        dlm.delete();
+
+        boolean exceptionEncountered = false;
+        try {
+            record = reader.readNext(false);
+            while (null != record) {
+                DLMTestUtil.verifyLogRecord(record);
+                assert (lastTxId < record.getTransactionId());
+                lastTxId = record.getTransactionId();
+                numTrans++;
+                record = reader.readNext(false);
+            }
+            // make sure the exception is thrown from readahead
+            while (true) {
+                reader.readNext(false);
+            }
+        } catch (LogReadException readexc) {
+            exceptionEncountered = true;
+        } catch (LogNotFoundException exc) {
+            exceptionEncountered = true;
+        }
+        assertTrue(exceptionEncountered);
+        reader.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testImmediateFlush() throws Exception {
+        String name = "distrlog-immediate-flush";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setOutputBufferSize(0);
+        testNonPartitionedWritesInternal(name, confLocal);
+    }
+
+    @Test(timeout = 60000)
+    public void testLastLogRecordWithEmptyLedgers() throws Exception {
+        String name = "distrlog-lastLogRec-emptyledgers";
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        long txid = 1;
+        for (long i = 0; i < 3; i++) {
+            long start = txid;
+            BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
+                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+                out.write(op);
+            }
+            BKLogSegmentWriter perStreamLogWriter = out.getCachedLogWriter();
+            out.closeAndComplete();
+            BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
+
+            assertNotNull(
+                zkc.exists(blplm.completedLedgerZNode(start, txid - 1,
+                                                      perStreamLogWriter.getLogSegmentSequenceNumber()), false));
+            BKLogSegmentWriter writer = blplm.startLogSegment(txid - 1);
+            blplm.completeAndCloseLogSegment(writer.getLogSegmentSequenceNumber(),
+                    writer.getLogSegmentId(), txid - 1, txid - 1, 0);
+            assertNotNull(
+                zkc.exists(blplm.completedLedgerZNode(txid - 1, txid - 1,
+                                                      writer.getLogSegmentSequenceNumber()), false));
+            FutureUtils.result(blplm.asyncClose());
+        }
+
+        BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+        LogRecord op = DLMTestUtil.getLogRecordInstance(txid);
+        op.setControl();
+        out.write(op);
+        out.setReadyToFlush();
+        out.flushAndSync();
+        out.abort();
+        dlm.close();
+
+        dlm = createNewDLM(conf, name);
+
+        assertEquals(txid - 1, dlm.getLastTxId());
+        LogRecord last = dlm.getLastLogRecord();
+        assertEquals(txid - 1, last.getTransactionId());
+        DLMTestUtil.verifyLogRecord(last);
+        assertEquals(txid - 1, dlm.getLogRecordCount());
+
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testLogSegmentListener() throws Exception {
+        String name = "distrlog-logsegment-listener";
+        int numSegments = 3;
+        final CountDownLatch[] latches = new CountDownLatch[numSegments + 1];
+        for (int i = 0; i < numSegments + 1; i++) {
+            latches[i] = new CountDownLatch(1);
+        }
+
+        final AtomicInteger numFailures = new AtomicInteger(0);
+        final AtomicReference<Collection<LogSegmentMetadata>> receivedStreams =
+                new AtomicReference<Collection<LogSegmentMetadata>>();
+
+        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(conf, name);
+
+        FutureUtils.result(dlm.getWriterMetadataStore().getLog(dlm.getUri(), name, true, true));
+        dlm.registerListener(new LogSegmentListener() {
+            @Override
+            public void onSegmentsUpdated(List<LogSegmentMetadata> segments) {
+                int updates = segments.size();
+                boolean hasIncompletedLogSegments = false;
+                for (LogSegmentMetadata l : segments) {
+                    if (l.isInProgress()) {
+                        hasIncompletedLogSegments = true;
+                        break;
+                    }
+                }
+                if (hasIncompletedLogSegments) {
+                    return;
+                }
+                if (updates >= 1) {
+                    if (segments.get(segments.size() - 1).getLogSegmentSequenceNumber() != updates) {
+                        numFailures.incrementAndGet();
+                    }
+                }
+                receivedStreams.set(segments);
+                latches[updates].countDown();
+            }
+
+            @Override
+            public void onLogStreamDeleted() {
+                // no-op
+            }
+        });
+        long txid = 1;
+        for (int i = 0; i < numSegments; i++) {
+            LOG.info("Waiting for creating log segment {}.", i);
+            latches[i].await();
+            LOG.info("Creating log segment {}.", i);
+            BKSyncLogWriter out = (BKSyncLogWriter)dlm.startLogSegmentNonPartitioned();
+            LOG.info("Created log segment {}.", i);
+            for (long j = 1; j <= DEFAULT_SEGMENT_SIZE; j++) {
+                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+                out.write(op);
+            }
+            out.closeAndComplete();
+            LOG.info("Completed log segment {}.", i);
+        }
+        latches[numSegments].await();
+        assertEquals(0, numFailures.get());
+        assertNotNull(receivedStreams.get());
+        assertEquals(numSegments, receivedStreams.get().size());
+        int seqno = 1;
+        for (LogSegmentMetadata m : receivedStreams.get()) {
+            assertEquals(seqno, m.getLogSegmentSequenceNumber());
+            assertEquals((seqno - 1) * DEFAULT_SEGMENT_SIZE + 1, m.getFirstTxId());
+            assertEquals(seqno * DEFAULT_SEGMENT_SIZE, m.getLastTxId());
+            ++seqno;
+        }
+
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLastDLSN() throws Exception {
+        String name = "distrlog-get-last-dlsn";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setFirstNumEntriesPerReadLastRecordScan(2);
+        confLocal.setMaxNumEntriesPerReadLastRecordScan(4);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setOutputBufferSize(0);
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
+        long txid = 1;
+        LOG.info("Writing 10 control records");
+        for (int i = 0; i < 10; i++) {
+            LogRecord record = DLMTestUtil.getLogRecordInstance(txid++);
+            record.setControl();
+            Await.result(writer.writeControlRecord(record));
+        }
+        LOG.info("10 control records are written");
+
+        try {
+            dlm.getLastDLSN();
+            fail("Should fail on getting last dlsn from an empty log.");
+        } catch (LogEmptyException lee) {
+            // expected
+        }
+
+        writer.closeAndComplete();
+        LOG.info("Completed first log segment");
+
+        writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
+        Await.result(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));
+        }
+
+        assertEquals(new DLSN(2, 0, 0), dlm.getLastDLSN());
+
+        writer.closeAndComplete();
+        LOG.info("Completed third log segment");
+
+        assertEquals(new DLSN(2, 0, 0), dlm.getLastDLSN());
+
+        writer.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordCountAsync() throws Exception {
+        DistributedLogManager dlm = createNewDLM(conf, testNames.getMethodName());
+        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));
+        assertEquals(20, count.longValue());
+
+        writer.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testInvalidStreamFromInvalidZkPath() throws Exception {
+        String baseName = testNames.getMethodName();
+        String streamName = "\0blah";
+        URI uri = createDLMURI("/" + baseName);
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf).uri(uri).build();
+
+        DistributedLogManager dlm = null;
+        AsyncLogWriter writer = null;
+        try {
+            dlm = namespace.openLog(streamName);
+            writer = dlm.startAsyncLogSegmentNonPartitioned();
+            fail("should have thrown");
+        } catch (InvalidStreamNameException e) {
+        } finally {
+            if (null != writer) {
+                Utils.close(writer);
+            }
+            if (null != dlm) {
+                dlm.close();
+            }
+            namespace.close();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testTruncationValidation() throws Exception {
+        String name = "distrlog-truncation-validation";
+        URI uri = createDLMURI("/" + name);
+        ZooKeeperClient zookeeperClient = TestZooKeeperClientBuilder.newBuilder()
+            .uri(uri)
+            .build();
+        OrderedScheduler scheduler = OrderedScheduler.newBuilder()
+                .name("test-truncation-validation")
+                .corePoolSize(1)
+                .build();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setDLLedgerMetadataLayoutVersion(LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setLogSegmentCacheEnabled(false);
+
+        LogSegmentMetadataStore metadataStore = new ZKLogSegmentMetadataStore(confLocal, zookeeperClient, scheduler);
+
+        BKDistributedLogManager dlm = createNewDLM(confLocal, name);
+        DLSN truncDLSN = DLSN.InitialDLSN;
+        DLSN beyondTruncDLSN = DLSN.InitialDLSN;
+        long beyondTruncTxId = 1;
+        long txid = 1;
+        for (long i = 0; i < 3; i++) {
+            long start = txid;
+            BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+            for (long j = 1; j <= 10; j++) {
+                LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid++);
+                Future<DLSN> dlsn = writer.write(record);
+
+                if (i == 1 && j == 2) {
+                    truncDLSN = Await.result(dlsn);
+                } else if (i == 2 && j == 3) {
+                    beyondTruncDLSN = Await.result(dlsn);
+                    beyondTruncTxId = record.getTransactionId();
+                } else if (j == 10) {
+                    Await.ready(dlsn);
+                }
+            }
+
+            writer.close();
+        }
+
+        {
+            LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
+            LogRecordWithDLSN record = reader.readNext(false);
+            assertTrue((record != null) && (record.getDlsn().compareTo(DLSN.InitialDLSN) == 0));
+            reader.close();
+        }
+
+        Map<Long, LogSegmentMetadata> segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
+                LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName()));
+
+        LOG.info("Read segments before truncating first segment : {}", segmentList);
+
+        MetadataUpdater updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(
+                confLocal, metadataStore);
+        FutureUtils.result(updater.setLogSegmentTruncated(segmentList.get(1L)));
+
+        segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
+                LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName()));
+
+        LOG.info("Read segments after truncated first segment : {}", segmentList);
+
+        {
+            LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
+            LogRecordWithDLSN record = reader.readNext(false);
+            assertTrue("Unexpected record : " + record,
+                    (record != null) && (record.getDlsn().compareTo(new DLSN(2, 0, 0)) == 0));
+            reader.close();
+        }
+
+        {
+            LogReader reader = dlm.getInputStream(1);
+            LogRecordWithDLSN record = reader.readNext(false);
+            assertTrue((record != null) && (record.getDlsn().compareTo(new DLSN(2, 0, 0)) == 0));
+            reader.close();
+        }
+
+        updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, metadataStore);
+        FutureUtils.result(updater.setLogSegmentActive(segmentList.get(1L)));
+
+        segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
+                LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName()));
+
+        LOG.info("Read segments after marked first segment as active : {}", segmentList);
+
+        updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, metadataStore);
+        FutureUtils.result(updater.setLogSegmentTruncated(segmentList.get(2L)));
+
+        segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
+                LogMetadata.getLogSegmentsPath(uri, name, confLocal.getUnpartitionedStreamName()));
+
+        LOG.info("Read segments after truncated second segment : {}", segmentList);
+
+        {
+            AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN);
+            long expectedTxId = 1L;
+            boolean exceptionEncountered = false;
+            try {
+                for (int i = 0; i < 3 * 10; i++) {
+                    LogRecordWithDLSN record = Await.result(reader.readNext());
+                    DLMTestUtil.verifyLargeLogRecord(record);
+                    assertEquals(expectedTxId, record.getTransactionId());
+                    expectedTxId++;
+                }
+            } catch (AlreadyTruncatedTransactionException exc) {
+                exceptionEncountered = true;
+            }
+            assertTrue(exceptionEncountered);
+            Utils.close(reader);
+        }
+
+        updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
+        FutureUtils.result(updater.setLogSegmentActive(segmentList.get(2L)));
+
+        BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+        Assert.assertTrue(Await.result(writer.truncate(truncDLSN)));
+        BKLogWriteHandler handler = writer.getCachedWriteHandler();
+        List<LogSegmentMetadata> cachedSegments = handler.getCachedLogSegments(LogSegmentMetadata.COMPARATOR);
+        for (LogSegmentMetadata segment: cachedSegments) {
+            if (segment.getLastDLSN().compareTo(truncDLSN) < 0) {
+                Assert.assertTrue(segment.isTruncated());
+                Assert.assertTrue(!segment.isPartiallyTruncated());
+            } else if (segment.getFirstDLSN().compareTo(truncDLSN) < 0) {
+                Assert.assertTrue(!segment.isTruncated());
+                Assert.assertTrue(segment.isPartiallyTruncated());
+            } else {
+                Assert.assertTrue(!segment.isTruncated());
+                Assert.assertTrue(!segment.isPartiallyTruncated());
+            }
+        }
+
+        segmentList = DLMTestUtil.readLogSegments(zookeeperClient,
+                LogMetadata.getLogSegmentsPath(uri, name, conf.getUnpartitionedStreamName()));
+
+        Assert.assertTrue(segmentList.get(truncDLSN.getLogSegmentSequenceNo()).getMinActiveDLSN().compareTo(truncDLSN) == 0);
+
+        {
+            LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
+            LogRecordWithDLSN record = reader.readNext(false);
+            assertTrue(record != null);
+            assertEquals(truncDLSN, record.getDlsn());
+            reader.close();
+        }
+
+        {
+            LogReader reader = dlm.getInputStream(1);
+            LogRecordWithDLSN record = reader.readNext(false);
+            assertTrue(record != null);
+            assertEquals(truncDLSN, record.getDlsn());
+            reader.close();
+        }
+
+        {
+            AsyncLogReader reader = dlm.getAsyncLogReader(DLSN.InitialDLSN);
+            LogRecordWithDLSN record = Await.result(reader.readNext());
+            assertTrue(record != null);
+            assertEquals(truncDLSN, record.getDlsn());
+            Utils.close(reader);
+        }
+
+
+        {
+            LogReader reader = dlm.getInputStream(beyondTruncDLSN);
+            LogRecordWithDLSN record = reader.readNext(false);
+            assertTrue(record != null);
+            assertEquals(beyondTruncDLSN, record.getDlsn());
+            reader.close();
+        }
+
+        {
+            LogReader reader = dlm.getInputStream(beyondTruncTxId);
+            LogRecordWithDLSN record = reader.readNext(false);
+            assertTrue(record != null);
+            assertEquals(beyondTruncDLSN, record.getDlsn());
+            assertEquals(beyondTruncTxId, record.getTransactionId());
+            reader.close();
+        }
+
+        {
+            AsyncLogReader reader = dlm.getAsyncLogReader(beyondTruncDLSN);
+            LogRecordWithDLSN record = Await.result(reader.readNext());
+            assertTrue(record != null);
+            assertEquals(beyondTruncDLSN, record.getDlsn());
+            Utils.close(reader);
+        }
+
+        zookeeperClient.close();
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java
new file mode 100644
index 0000000..43d1008
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKDistributedLogNamespace.java
@@ -0,0 +1,440 @@
+/**
+ * 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 java.io.IOException;
+import java.net.URI;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import com.google.common.collect.Sets;
+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.util.DLUtils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.Stat;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+import static org.hamcrest.core.IsNot.not;
+import static org.junit.Assert.*;
+
+public class TestBKDistributedLogNamespace extends TestDistributedLogBase {
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    static final Logger LOG = LoggerFactory.getLogger(TestBKDistributedLogNamespace.class);
+
+    protected static DistributedLogConfiguration conf =
+            new DistributedLogConfiguration().setLockTimeout(10)
+                .setEnableLedgerAllocatorPool(true).setLedgerAllocatorPoolName("test");
+
+    private ZooKeeperClient zooKeeperClient;
+
+    @Before
+    public void setup() throws Exception {
+        zooKeeperClient =
+            TestZooKeeperClientBuilder.newBuilder()
+                .uri(createDLMURI("/"))
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        zooKeeperClient.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateIfNotExists() throws Exception {
+        URI uri = createDLMURI("/" + runtime.getMethodName());
+        ensureURICreated(zooKeeperClient.get(), uri);
+        DistributedLogConfiguration newConf = new DistributedLogConfiguration();
+        newConf.addConfiguration(conf);
+        newConf.setCreateStreamIfNotExists(false);
+        String streamName = "test-stream";
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(newConf).uri(uri).build();
+        DistributedLogManager dlm = namespace.openLog(streamName);
+        LogWriter writer;
+        try {
+            writer = dlm.startLogSegmentNonPartitioned();
+            writer.write(DLMTestUtil.getLogRecordInstance(1L));
+            fail("Should fail to write data if stream doesn't exist.");
+        } catch (IOException ioe) {
+            // expected
+        }
+        dlm.close();
+
+        // create the stream
+        namespace.createLog(streamName);
+
+        DistributedLogManager newDLM = namespace.openLog(streamName);
+        LogWriter newWriter = newDLM.startLogSegmentNonPartitioned();
+        newWriter.write(DLMTestUtil.getLogRecordInstance(1L));
+        newWriter.close();
+        newDLM.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testInvalidStreamName() throws Exception {
+        assertFalse(DLUtils.isReservedStreamName("test"));
+        assertTrue(DLUtils.isReservedStreamName(".test"));
+
+        URI uri = createDLMURI("/" + runtime.getMethodName());
+
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf).uri(uri).build();
+
+        try {
+            namespace.openLog(".test1");
+            fail("Should fail to create invalid stream .test");
+        } catch (InvalidStreamNameException isne) {
+            // expected
+        }
+
+        DistributedLogManager dlm = namespace.openLog("test1");
+        LogWriter writer = dlm.startLogSegmentNonPartitioned();
+        writer.write(DLMTestUtil.getLogRecordInstance(1));
+        writer.close();
+        dlm.close();
+
+        try {
+            namespace.openLog(".test2");
+            fail("Should fail to create invalid stream .test2");
+        } catch (InvalidStreamNameException isne) {
+            // expected
+        }
+
+        try {
+            namespace.openLog("/test2");
+            fail("should fail to create invalid stream /test2");
+        } catch (InvalidStreamNameException isne) {
+            // expected
+        }
+
+        try {
+            char[] chars = new char[6];
+            for (int i = 0; i < chars.length; i++) {
+                chars[i] = 'a';
+            }
+            chars[0] = 0;
+            String streamName = new String(chars);
+            namespace.openLog(streamName);
+            fail("should fail to create invalid stream " + streamName);
+        } catch (InvalidStreamNameException isne) {
+            // expected
+        }
+
+        try {
+            char[] chars = new char[6];
+            for (int i = 0; i < chars.length; i++) {
+                chars[i] = 'a';
+            }
+            chars[3] = '\u0010';
+            String streamName = new String(chars);
+            namespace.openLog(streamName);
+            fail("should fail to create invalid stream " + streamName);
+        } catch (InvalidStreamNameException isne) {
+            // expected
+        }
+
+        DistributedLogManager newDLM =
+                namespace.openLog("test_2-3");
+        LogWriter newWriter = newDLM.startLogSegmentNonPartitioned();
+        newWriter.write(DLMTestUtil.getLogRecordInstance(1));
+        newWriter.close();
+        newDLM.close();
+
+        Iterator<String> streamIter = namespace.getLogs();
+        Set<String> streamSet = Sets.newHashSet(streamIter);
+
+        assertEquals(2, streamSet.size());
+        assertTrue(streamSet.contains("test1"));
+        assertTrue(streamSet.contains("test_2-3"));
+
+        namespace.close();
+    }
+
+    @Test(timeout = 60000)
+    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()
+                .conf(conf).uri(uri).build();
+        final CountDownLatch[] latches = new CountDownLatch[3];
+        for (int i = 0; i < 3; i++) {
+            latches[i] = new CountDownLatch(1);
+        }
+        final AtomicInteger numUpdates = new AtomicInteger(0);
+        final AtomicInteger numFailures = new AtomicInteger(0);
+        final AtomicReference<Collection<String>> receivedStreams = new AtomicReference<Collection<String>>(null);
+        namespace.registerNamespaceListener(new NamespaceListener() {
+            @Override
+            public void onStreamsChanged(Iterator<String> streams) {
+                Set<String> streamSet = Sets.newHashSet(streams);
+                int updates = numUpdates.incrementAndGet();
+                if (streamSet.size() != updates - 1) {
+                    numFailures.incrementAndGet();
+                }
+
+                receivedStreams.set(streamSet);
+                latches[updates - 1].countDown();
+            }
+        });
+        latches[0].await();
+        namespace.createLog("test1");
+        latches[1].await();
+        namespace.createLog("test2");
+        latches[2].await();
+        assertEquals(0, numFailures.get());
+        assertNotNull(receivedStreams.get());
+        Set<String> streamSet = new HashSet<String>();
+        streamSet.addAll(receivedStreams.get());
+        assertEquals(2, receivedStreams.get().size());
+        assertEquals(2, streamSet.size());
+        assertTrue(streamSet.contains("test1"));
+        assertTrue(streamSet.contains("test2"));
+    }
+
+    private void initDlogMeta(String dlNamespace, String un, String streamName) throws Exception {
+        URI uri = createDLMURI(dlNamespace);
+        DistributedLogConfiguration newConf = new DistributedLogConfiguration();
+        newConf.addConfiguration(conf);
+        newConf.setCreateStreamIfNotExists(true);
+        newConf.setZkAclId(un);
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(newConf).uri(uri).build();
+        DistributedLogManager dlm = namespace.openLog(streamName);
+        LogWriter writer = dlm.startLogSegmentNonPartitioned();
+        for (int i = 0; i < 10; i++) {
+            writer.write(DLMTestUtil.getLogRecordInstance(1L));
+        }
+        writer.close();
+        dlm.close();
+        namespace.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testAclPermsZkAccessConflict() throws Exception {
+
+        String namespace = "/" + runtime.getMethodName();
+        initDlogMeta(namespace, "test-un", "test-stream");
+        URI uri = createDLMURI(namespace);
+
+        ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder()
+            .name("unpriv")
+            .uri(uri)
+            .build();
+
+        try {
+            zkc.get().create(uri.getPath() + "/test-stream/test-garbage",
+                new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            fail("write should have failed due to perms");
+        } catch (KeeperException.NoAuthException ex) {
+            LOG.info("caught exception trying to write with no perms", ex);
+        }
+
+        try {
+            zkc.get().setData(uri.getPath() + "/test-stream", new byte[0], 0);
+            fail("write should have failed due to perms");
+        } catch (KeeperException.NoAuthException ex) {
+            LOG.info("caught exception trying to write with no perms", ex);
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testAclPermsZkAccessNoConflict() throws Exception {
+
+        String namespace = "/" + runtime.getMethodName();
+        initDlogMeta(namespace, "test-un", "test-stream");
+        URI uri = createDLMURI(namespace);
+
+        ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder()
+            .name("unpriv")
+            .uri(uri)
+            .build();
+
+        zkc.get().getChildren(uri.getPath() + "/test-stream", false, new Stat());
+        zkc.get().getData(uri.getPath() + "/test-stream", false, new Stat());
+    }
+
+    @Test(timeout = 60000)
+    public void testAclModifyPermsDlmConflict() throws Exception {
+        String streamName = "test-stream";
+
+        // Reopening and writing again with the same un will succeed.
+        initDlogMeta("/" + runtime.getMethodName(), "test-un", streamName);
+
+        try {
+            // Reopening and writing again with a different un will fail.
+            initDlogMeta("/" + runtime.getMethodName(), "not-test-un", streamName);
+            fail("write should have failed due to perms");
+        } catch (ZKException ex) {
+            LOG.info("caught exception trying to write with no perms {}", ex);
+            assertEquals(KeeperException.Code.NOAUTH, ex.getKeeperExceptionCode());
+        } catch (Exception ex) {
+            LOG.info("caught wrong exception trying to write with no perms {}", ex);
+            fail("wrong exception " + ex.getClass().getName() + " expected " + LockingException.class.getName());
+        }
+
+        // Should work again.
+        initDlogMeta("/" + runtime.getMethodName(), "test-un", streamName);
+    }
+
+    @Test(timeout = 60000)
+    public void testAclModifyPermsDlmNoConflict() throws Exception {
+        String streamName = "test-stream";
+
+        // Establish the uri.
+        initDlogMeta("/" + runtime.getMethodName(), "test-un", streamName);
+
+        // Reopening and writing again with the same un will succeed.
+        initDlogMeta("/" + runtime.getMethodName(), "test-un", streamName);
+    }
+
+    static void validateBadAllocatorConfiguration(DistributedLogConfiguration conf, URI uri) throws Exception {
+        try {
+            BKNamespaceDriver.validateAndGetFullLedgerAllocatorPoolPath(conf, uri);
+            fail("Should throw exception when bad allocator configuration provided");
+        } catch (IOException ioe) {
+            // expected
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testValidateAndGetFullLedgerAllocatorPoolPath() throws Exception {
+        DistributedLogConfiguration testConf = new DistributedLogConfiguration();
+        testConf.setEnableLedgerAllocatorPool(true);
+
+        String namespace = "/" + runtime.getMethodName();
+        URI uri = createDLMURI(namespace);
+
+        testConf.setLedgerAllocatorPoolName("test");
+
+        testConf.setLedgerAllocatorPoolPath("test");
+        validateBadAllocatorConfiguration(testConf, uri);
+
+        testConf.setLedgerAllocatorPoolPath(".");
+        validateBadAllocatorConfiguration(testConf, uri);
+
+        testConf.setLedgerAllocatorPoolPath("..");
+        validateBadAllocatorConfiguration(testConf, uri);
+
+        testConf.setLedgerAllocatorPoolPath("./");
+        validateBadAllocatorConfiguration(testConf, uri);
+
+        testConf.setLedgerAllocatorPoolPath(".test/");
+        validateBadAllocatorConfiguration(testConf, uri);
+
+        testConf.setLedgerAllocatorPoolPath(".test");
+        testConf.setLedgerAllocatorPoolName(null);
+        validateBadAllocatorConfiguration(testConf, uri);
+    }
+
+    @Test(timeout = 60000)
+    public void testUseNamespaceAfterCloseShouldFailFast() throws Exception {
+        URI uri = createDLMURI("/" + runtime.getMethodName());
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+            .conf(conf)
+            .uri(uri)
+            .build();
+        // before closing the namespace, no exception should be thrown
+        String logName = "test-stream";
+        // create a log
+        namespace.createLog(logName);
+        // log exists
+        Assert.assertTrue(namespace.logExists(logName));
+        // create a dlm
+        DistributedLogManager dlm = namespace.openLog(logName);
+        // do some writes
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) (dlm.startAsyncLogSegmentNonPartitioned());
+        for (long i = 0; i < 3; i++) {
+            LogRecord record = DLMTestUtil.getLargeLogRecordInstance(i);
+            writer.write(record);
+        }
+        writer.closeAndComplete();
+        // do some reads
+        LogReader reader = dlm.getInputStream(0);
+        for (long i = 0; i < 3; i++) {
+            Assert.assertEquals(reader.readNext(false).getTransactionId(), i);
+        }
+        namespace.deleteLog(logName);
+        Assert.assertFalse(namespace.logExists(logName));
+
+        // now try to close the namespace
+        namespace.close();
+        try {
+            namespace.createLog(logName);
+            fail("Should throw exception after namespace is closed");
+        } catch (AlreadyClosedException e) {
+            // No-ops
+        }
+        try {
+            namespace.openLog(logName);
+            fail("Should throw exception after namespace is closed");
+        } catch (AlreadyClosedException e) {
+            // No-ops
+        }
+        try {
+            namespace.logExists(logName);
+            fail("Should throw exception after namespace is closed");
+        } catch (AlreadyClosedException e) {
+            // No-ops
+        }
+        try {
+            namespace.getLogs();
+            fail("Should throw exception after namespace is closed");
+        } catch (AlreadyClosedException e) {
+            // No-ops
+        }
+        try {
+            namespace.deleteLog(logName);
+            fail("Should throw exception after namespace is closed");
+        } catch (AlreadyClosedException e) {
+            // No-ops
+        }
+        try {
+            namespace.createAccessControlManager();
+            fail("Should throw exception after namespace is closed");
+        } catch (AlreadyClosedException e) {
+            // No-ops
+        }
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java
new file mode 100644
index 0000000..854fd61
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogReadHandler.java
@@ -0,0 +1,398 @@
+/**
+ * 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.base.Optional;
+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.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;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test {@link BKLogReadHandler}
+ */
+public class TestBKLogReadHandler extends TestDistributedLogBase {
+
+    static final Logger LOG = LoggerFactory.getLogger(TestBKLogReadHandler.class);
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    private void prepareLogSegmentsNonPartitioned(String name, int numSegments, int numEntriesPerSegment) throws Exception {
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        long txid = 1;
+        for (int sid = 0; sid < numSegments; ++sid) {
+            LogWriter out = dlm.startLogSegmentNonPartitioned();
+            for (int eid = 0; eid < numEntriesPerSegment; ++eid) {
+                LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
+                out.write(record);
+                ++txid;
+            }
+            out.close();
+        }
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testGetFirstDLSNWithOpenLedger() throws Exception {
+        String dlName = runtime.getMethodName();
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(0);
+
+        int numEntriesPerSegment = 10;
+        DistributedLogManager dlm1 = createNewDLM(confLocal, dlName);
+        long txid = 1;
+
+        ArrayList<Future<DLSN>> futures = new ArrayList<Future<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));
+        // commit
+        LogRecord controlRecord = new LogRecord(txid, DistributedLogConstants.CONTROL_RECORD_CONTENT);
+        controlRecord.setControl();
+        FutureUtils.result(out.write(controlRecord));
+
+        DLSN last = dlm1.getLastDLSN();
+        assertEquals(new DLSN(1,9,0), last);
+        DLSN first = Await.result(dlm1.getFirstDLSNAsync());
+        assertEquals(new DLSN(1,0,0), first);
+        Utils.close(out);
+    }
+
+    @Test(timeout = 60000)
+    public void testGetFirstDLSNNoLogSegments() throws Exception {
+        String dlName = runtime.getMethodName();
+        BKDistributedLogManager dlm = createNewDLM(conf, dlName);
+        BKLogReadHandler readHandler = dlm.createReadHandler();
+        Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
+        try {
+            Await.result(futureRecord);
+            fail("should have thrown exception");
+        } catch (LogNotFoundException ex) {
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testGetFirstDLSNWithLogSegments() throws Exception {
+        String dlName = runtime.getMethodName();
+        BKDistributedLogManager dlm = createNewDLM(conf, dlName);
+        DLMTestUtil.generateCompletedLogSegments(dlm, conf, 3, 3);
+        BKLogReadHandler readHandler = dlm.createReadHandler();
+        Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
+        try {
+            LogRecordWithDLSN record = Await.result(futureRecord);
+            assertEquals(new DLSN(1, 0, 0), record.getDlsn());
+        } catch (Exception ex) {
+            fail("should not have thrown exception: " + ex);
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testGetFirstDLSNAfterCleanTruncation() throws Exception {
+        String dlName = runtime.getMethodName();
+        prepareLogSegmentsNonPartitioned(dlName, 3, 10);
+        DistributedLogManager dlm = createNewDLM(conf, dlName);
+        BKLogReadHandler readHandler =
+            ((BKDistributedLogManager) dlm).createReadHandler();
+        AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+        Future<Boolean> futureSuccess = writer.truncate(new DLSN(2, 0, 0));
+        Boolean success = Await.result(futureSuccess);
+        assertTrue(success);
+        Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
+        LogRecordWithDLSN record = Await.result(futureRecord);
+        assertEquals(new DLSN(2, 0, 0), record.getDlsn());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetFirstDLSNAfterPartialTruncation() throws Exception {
+        String dlName = runtime.getMethodName();
+        prepareLogSegmentsNonPartitioned(dlName, 3, 10);
+        DistributedLogManager dlm = createNewDLM(conf, dlName);
+        BKLogReadHandler readHandler =
+            ((BKDistributedLogManager) dlm).createReadHandler();
+        AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+
+        // Only truncates at ledger boundary.
+        Future<Boolean> futureSuccess = writer.truncate(new DLSN(2, 5, 0));
+        Boolean success = Await.result(futureSuccess);
+        assertTrue(success);
+        Future<LogRecordWithDLSN> futureRecord = readHandler.asyncGetFirstLogRecord();
+        LogRecordWithDLSN record = Await.result(futureRecord);
+        assertEquals(new DLSN(2, 0, 0), record.getDlsn());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordCountEmptyLedger() throws Exception {
+        String dlName = runtime.getMethodName();
+        DistributedLogManager dlm = createNewDLM(conf, dlName);
+        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
+        Future<Long> count = null;
+        count = readHandler.asyncGetLogRecordCount(DLSN.InitialDLSN);
+        try {
+            Await.result(count);
+            fail("log is empty, should have returned log empty ex");
+        } catch (LogNotFoundException ex) {
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordCountTotalCount() throws Exception {
+        String dlName = runtime.getMethodName();
+        prepareLogSegmentsNonPartitioned(dlName, 11, 3);
+        DistributedLogManager dlm = createNewDLM(conf, dlName);
+        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
+        Future<Long> count = null;
+        count = readHandler.asyncGetLogRecordCount(DLSN.InitialDLSN);
+        assertEquals(33, Await.result(count).longValue());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordCountAtLedgerBoundary() throws Exception {
+        String dlName = runtime.getMethodName();
+        prepareLogSegmentsNonPartitioned(dlName, 11, 3);
+        DistributedLogManager dlm = createNewDLM(conf, dlName);
+        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
+        Future<Long> count = null;
+        count = readHandler.asyncGetLogRecordCount(new DLSN(2, 0, 0));
+        assertEquals(30, Await.result(count).longValue());
+        count = readHandler.asyncGetLogRecordCount(new DLSN(3, 0, 0));
+        assertEquals(27, Await.result(count).longValue());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordCountPastEnd() throws Exception {
+        String dlName = runtime.getMethodName();
+        prepareLogSegmentsNonPartitioned(dlName, 11, 3);
+        DistributedLogManager dlm = createNewDLM(conf, dlName);
+        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
+        Future<Long> count = null;
+        count = readHandler.asyncGetLogRecordCount(new DLSN(12, 0, 0));
+        assertEquals(0, Await.result(count).longValue());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordCountLastRecord() throws Exception {
+        String dlName = runtime.getMethodName();
+        prepareLogSegmentsNonPartitioned(dlName, 11, 3);
+        DistributedLogManager dlm = createNewDLM(conf, dlName);
+        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
+        Future<Long> count = null;
+        count = readHandler.asyncGetLogRecordCount(new DLSN(11, 2, 0));
+        assertEquals(1, Await.result(count).longValue());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordCountInteriorRecords() throws Exception {
+        String dlName = runtime.getMethodName();
+        prepareLogSegmentsNonPartitioned(dlName, 5, 10);
+        DistributedLogManager dlm = createNewDLM(conf, dlName);
+        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
+        Future<Long> count = null;
+        count = readHandler.asyncGetLogRecordCount(new DLSN(3, 5, 0));
+        assertEquals(25, Await.result(count).longValue());
+        count = readHandler.asyncGetLogRecordCount(new DLSN(2, 5, 0));
+        assertEquals(35, Await.result(count).longValue());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordCountWithControlRecords() throws Exception {
+        DistributedLogManager dlm = createNewDLM(conf, runtime.getMethodName());
+        long txid = 1;
+        txid += DLMTestUtil.generateLogSegmentNonPartitioned(dlm, 5, 5, txid);
+        txid += DLMTestUtil.generateLogSegmentNonPartitioned(dlm, 0, 10, txid);
+        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
+        Future<Long> count = null;
+        count = readHandler.asyncGetLogRecordCount(new DLSN(1, 0, 0));
+        assertEquals(15, Await.result(count).longValue());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordCountWithAllControlRecords() throws Exception {
+        DistributedLogManager dlm = createNewDLM(conf, runtime.getMethodName());
+        long txid = 1;
+        txid += DLMTestUtil.generateLogSegmentNonPartitioned(dlm, 5, 0, txid);
+        txid += DLMTestUtil.generateLogSegmentNonPartitioned(dlm, 10, 0, txid);
+        BKLogReadHandler readHandler = ((BKDistributedLogManager) dlm).createReadHandler();
+        Future<Long> count = null;
+        count = readHandler.asyncGetLogRecordCount(new DLSN(1, 0, 0));
+        assertEquals(0, Await.result(count).longValue());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordCountWithSingleInProgressLedger() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
+
+        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)));
+
+        BKLogReadHandler readHandler = bkdlm.createReadHandler();
+        List<LogSegmentMetadata> ledgerList = FutureUtils.result(
+                readHandler.readLogSegmentsFromStore(
+                        LogSegmentMetadata.COMPARATOR,
+                        LogSegmentFilter.DEFAULT_FILTER,
+                        null
+                )
+        ).getValue();
+        assertEquals(1, ledgerList.size());
+        assertTrue(ledgerList.get(0).isInProgress());
+
+        Future<Long> count = null;
+        count = readHandler.asyncGetLogRecordCount(new DLSN(1, 0, 0));
+        assertEquals(2, Await.result(count).longValue());
+
+        Utils.close(out);
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordCountWithCompletedAndInprogressLedgers() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
+
+        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)));
+
+        BKLogReadHandler readHandler = bkdlm.createReadHandler();
+        List<LogSegmentMetadata> ledgerList = FutureUtils.result(
+                readHandler.readLogSegmentsFromStore(
+                        LogSegmentMetadata.COMPARATOR,
+                        LogSegmentFilter.DEFAULT_FILTER,
+                        null)
+        ).getValue();
+        assertEquals(2, ledgerList.size());
+        assertFalse(ledgerList.get(0).isInProgress());
+        assertTrue(ledgerList.get(1).isInProgress());
+
+        Future<Long> count = null;
+        count = readHandler.asyncGetLogRecordCount(new DLSN(1, 0, 0));
+        assertEquals(7, Await.result(count).longValue());
+
+        Utils.close(out);
+    }
+
+    @Test(timeout = 60000)
+    public void testLockStreamWithMissingLog() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
+        BKLogReadHandler readHandler = bkdlm.createReadHandler();
+        try {
+            Await.result(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());
+            fail("Subscriber should fail lock stream if log not found");
+        } catch (LogNotFoundException ex) {
+            // expected
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testLockStreamDifferentSubscribers() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
+        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, 1);
+        BKLogReadHandler readHandler = bkdlm.createReadHandler();
+        Await.result(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());
+        BKDistributedLogManager bkdlm20 = createNewDLM(conf, streamName);
+        BKLogReadHandler s20Handler =
+                bkdlm20.createReadHandler(Optional.of("s2"));
+        Await.result(s20Handler.lockStream());
+
+        readHandler.asyncClose();
+        bkdlm.close();
+        s10Handler.asyncClose();
+        bkdlm10.close();
+        s20Handler.asyncClose();
+        bkdlm20.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testLockStreamSameSubscriber() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
+        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5, 1);
+        BKLogReadHandler readHandler = bkdlm.createReadHandler();
+        Await.result(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());
+
+        BKDistributedLogManager bkdlm11 = createNewDLM(conf, streamName);
+        BKLogReadHandler s11Handler =
+                bkdlm11.createReadHandler(Optional.of("s1"));
+        try {
+            Await.result(s11Handler.lockStream(), Duration.apply(10000, TimeUnit.MILLISECONDS));
+            fail("Should fail lock stream using same subscriber id");
+        } catch (OwnershipAcquireFailedException oafe) {
+            // expected
+        } catch (TimeoutException te) {
+            // expected.
+        }
+
+        readHandler.asyncClose();
+        bkdlm.close();
+        s10Handler.asyncClose();
+        bkdlm10.close();
+        s11Handler.asyncClose();
+        bkdlm11.close();
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java
new file mode 100644
index 0000000..ccbfc44
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogSegmentWriter.java
@@ -0,0 +1,790 @@
+/**
+ * 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 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.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.util.OrderedScheduler;
+import org.apache.distributedlog.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;
+import org.apache.bookkeeper.feature.SettableFeatureProvider;
+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;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.*;
+
+/**
+ * Test Case for BookKeeper Based Log Segment Writer
+ */
+public class TestBKLogSegmentWriter extends TestDistributedLogBase {
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    private OrderedScheduler scheduler;
+    private OrderedScheduler lockStateExecutor;
+    private ZooKeeperClient zkc;
+    private ZooKeeperClient zkc0;
+    private BookKeeperClient bkc;
+
+    @Before
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+        scheduler = OrderedScheduler.newBuilder().corePoolSize(1).build();
+        lockStateExecutor = OrderedScheduler.newBuilder().corePoolSize(1).build();
+        // build zookeeper client
+        URI uri = createDLMURI("");
+        zkc = TestZooKeeperClientBuilder.newBuilder(conf)
+                .name("test-zkc")
+                .uri(uri)
+                .build();
+        zkc0 = TestZooKeeperClientBuilder.newBuilder(conf)
+                .name("test-zkc0")
+                .uri(uri)
+                .build();
+        // build bookkeeper client
+        BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, uri);
+        bkc = BookKeeperClientBuilder.newBuilder()
+                .dlConfig(conf)
+                .name("test-bkc")
+                .ledgersPath(bkdlConfig.getBkLedgersPath())
+                .zkServers(BKNamespaceDriver.getZKServersFromDLUri(uri))
+                .build();
+    }
+
+    @After
+    @Override
+    public void teardown() throws Exception {
+        if (null != bkc) {
+            bkc.close();
+        }
+        if (null != zkc) {
+            zkc.close();
+        }
+        if (null != lockStateExecutor) {
+            lockStateExecutor.shutdown();
+        }
+        if (null != scheduler) {
+            scheduler.shutdown();
+        }
+        super.teardown();
+    }
+
+    private DistributedLogConfiguration newLocalConf() {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        return confLocal;
+    }
+
+    private ZKDistributedLock createLock(String path,
+                                         ZooKeeperClient zkClient,
+                                         boolean acquireLock)
+            throws Exception {
+        try {
+            Await.result(Utils.zkAsyncCreateFullPathOptimistic(zkClient, path, new byte[0],
+                    ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT));
+        } catch (KeeperException.NodeExistsException nee) {
+            // node already exists
+        }
+        SessionLockFactory lockFactory = new ZKSessionLockFactory(
+                zkClient,
+                "test-lock",
+                lockStateExecutor,
+                0,
+                Long.MAX_VALUE,
+                conf.getZKSessionTimeoutMilliseconds(),
+                NullStatsLogger.INSTANCE
+        );
+        ZKDistributedLock lock = new ZKDistributedLock(
+                lockStateExecutor,
+                lockFactory,
+                path,
+                Long.MAX_VALUE,
+                NullStatsLogger.INSTANCE);
+        if (acquireLock) {
+            return FutureUtils.result(lock.asyncAcquire());
+        } else {
+            return lock;
+        }
+    }
+
+    private void closeWriterAndLock(BKLogSegmentWriter writer,
+                                    ZKDistributedLock lock)
+            throws IOException {
+        try {
+            FutureUtils.result(writer.asyncClose());
+        } finally {
+            Utils.closeQuietly(lock);
+        }
+    }
+
+    private void abortWriterAndLock(BKLogSegmentWriter writer,
+                                    ZKDistributedLock lock)
+            throws IOException {
+        try {
+            Abortables.abort(writer, false);
+        } finally {
+            Utils.closeQuietly(lock);
+        }
+    }
+
+    private BKLogSegmentWriter createLogSegmentWriter(DistributedLogConfiguration conf,
+                                                      long logSegmentSequenceNumber,
+                                                      long startTxId,
+                                                      ZKDistributedLock lock) throws Exception {
+        LedgerHandle lh = bkc.get().createLedger(3, 2, 2,
+                BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
+        return new BKLogSegmentWriter(
+                runtime.getMethodName(),
+                runtime.getMethodName(),
+                conf,
+                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION,
+                new BKLogSegmentEntryWriter(lh),
+                lock,
+                startTxId,
+                logSegmentSequenceNumber,
+                scheduler,
+                NullStatsLogger.INSTANCE,
+                NullStatsLogger.INSTANCE,
+                new AlertStatsLogger(NullStatsLogger.INSTANCE, "test"),
+                PermitLimiter.NULL_PERMIT_LIMITER,
+                new SettableFeatureProvider("", 0),
+                ConfUtils.getConstDynConf(conf));
+    }
+
+    private LedgerHandle openLedgerNoRecovery(LedgerHandle lh) throws Exception {
+        return bkc.get().openLedgerNoRecovery(lh.getId(),
+                BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
+    }
+
+    private LedgerHandle openLedger(LedgerHandle lh) throws Exception {
+        return bkc.get().openLedger(lh.getId(),
+                BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
+    }
+
+    private void fenceLedger(LedgerHandle lh) throws Exception {
+        bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32,
+                conf.getBKDigestPW().getBytes(UTF_8));
+    }
+
+    /**
+     * Close a segment log writer should flush buffered data.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testCloseShouldFlush() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
+        BKLogSegmentWriter writer =
+                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();
+        // add 10 records
+        int numRecords = 10;
+        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
+        for (int i = 0; i < numRecords; i++) {
+            futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
+        }
+        assertEquals("Last tx id should be " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxId());
+        assertEquals("Last acked tx id should be -1",
+                -1L, writer.getLastTxIdAcknowledged());
+        assertEquals("Last DLSN should be " + DLSN.InvalidDLSN,
+                DLSN.InvalidDLSN, writer.getLastDLSN());
+        assertEquals("Position should be " + numRecords,
+                10, writer.getPositionWithinLogSegment());
+        // close the writer should flush buffered data and release lock
+        closeWriterAndLock(writer, lock);
+        Await.result(lockFuture0);
+        lock0.checkOwnership();
+        assertEquals("Last tx id should still be " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxId());
+        assertEquals("Last acked tx id should become " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxIdAcknowledged());
+        assertEquals("Position should still be " + numRecords,
+                10, writer.getPositionWithinLogSegment());
+        List<DLSN> dlsns = Await.result(Future.collect(futureList));
+        assertEquals("All records should be written",
+                numRecords, dlsns.size());
+        for (int i = 0; i < numRecords; i++) {
+            DLSN dlsn = dlsns.get(i);
+            assertEquals("Incorrent ledger sequence number",
+                    0L, dlsn.getLogSegmentSequenceNo());
+            assertEquals("Incorrent entry id",
+                    0L, dlsn.getEntryId());
+            assertEquals("Inconsistent slot id",
+                    i, dlsn.getSlotId());
+        }
+        assertEquals("Last DLSN should be " + dlsns.get(dlsns.size() - 1),
+                dlsns.get(dlsns.size() - 1), writer.getLastDLSN());
+        LedgerHandle lh = getLedgerHandle(writer);
+        LedgerHandle readLh = openLedgerNoRecovery(lh);
+        assertTrue("Ledger " + lh.getId() + " should be closed", readLh.isClosed());
+        assertEquals("There should be two entries in ledger " + lh.getId(),
+                1L, readLh.getLastAddConfirmed());
+    }
+
+    /**
+     * Abort a segment log writer should just abort pending writes and not flush buffered data.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testAbortShouldNotFlush() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
+        BKLogSegmentWriter writer =
+                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();
+        // add 10 records
+        int numRecords = 10;
+        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
+        for (int i = 0; i < numRecords; i++) {
+            futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
+        }
+        assertEquals("Last tx id should be " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxId());
+        assertEquals("Last acked tx id should be -1",
+                -1L, writer.getLastTxIdAcknowledged());
+        assertEquals("Last DLSN should be " + DLSN.InvalidDLSN,
+                DLSN.InvalidDLSN, writer.getLastDLSN());
+        assertEquals("Position should be " + numRecords,
+                10, writer.getPositionWithinLogSegment());
+        // close the writer should flush buffered data and release lock
+        abortWriterAndLock(writer, lock);
+        Await.result(lockFuture0);
+        lock0.checkOwnership();
+        assertEquals("Last tx id should still be " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxId());
+        assertEquals("Last acked tx id should still be " + (numRecords - 1),
+                -1L, writer.getLastTxIdAcknowledged());
+        assertEquals("Last DLSN should still be " + DLSN.InvalidDLSN,
+                DLSN.InvalidDLSN, writer.getLastDLSN());
+        assertEquals("Position should still be " + numRecords,
+                10, writer.getPositionWithinLogSegment());
+
+        for (int i = 0; i < numRecords; i++) {
+            try {
+                Await.result(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",
+                        wce.getCause() instanceof BKTransmitException);
+                BKTransmitException bkte = (BKTransmitException) wce.getCause();
+                assertEquals("Record " + i + " should be aborted",
+                        BKException.Code.InterruptedException, bkte.getBKResultCode());
+            }
+        }
+
+        // check no entries were written
+        LedgerHandle lh = getLedgerHandle(writer);
+        LedgerHandle readLh = openLedgerNoRecovery(lh);
+        assertTrue("Ledger " + lh.getId() + " should not be closed", readLh.isClosed());
+        assertEquals("There should be no entries in ledger " + lh.getId(),
+                LedgerHandle.INVALID_ENTRY_ID, readLh.getLastAddConfirmed());
+    }
+
+
+    /**
+     * Close a log segment writer that already detect ledger fenced, should not flush buffered data.
+     * And should throw exception on closing.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testCloseShouldNotFlushIfLedgerFenced() throws Exception {
+        testCloseShouldNotFlushIfInErrorState(BKException.Code.LedgerFencedException);
+    }
+
+    /**
+     * Close a log segment writer that is already in error state, should not flush buffered data.
+     *
+     * @throws Exception
+     */
+    void testCloseShouldNotFlushIfInErrorState(int rcToFailComplete) throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
+        BKLogSegmentWriter writer =
+                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();
+        // add 10 records
+        int numRecords = 10;
+        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
+        for (int i = 0; i < numRecords; i++) {
+            futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
+        }
+        assertEquals("Last tx id should be " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxId());
+        assertEquals("Last acked tx id should be -1",
+                -1L, writer.getLastTxIdAcknowledged());
+        assertEquals("Last DLSN should be " + DLSN.InvalidDLSN,
+                DLSN.InvalidDLSN, writer.getLastDLSN());
+        assertEquals("Position should be " + numRecords,
+                10, writer.getPositionWithinLogSegment());
+        writer.setTransmitResult(rcToFailComplete);
+        // close the writer should release lock but not flush data
+        try {
+            closeWriterAndLock(writer, lock);
+            fail("Close a log segment writer in error state should throw exception");
+        } catch (BKTransmitException bkte) {
+            assertEquals("Inconsistent rc is thrown",
+                    rcToFailComplete, bkte.getBKResultCode());
+        }
+        Await.result(lockFuture0);
+        lock0.checkOwnership();
+        assertEquals("Last tx id should still be " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxId());
+        assertEquals("Last acked tx id should still be " + (numRecords - 1),
+                -1L, writer.getLastTxIdAcknowledged());
+        assertEquals("Last DLSN should still be " + DLSN.InvalidDLSN,
+                DLSN.InvalidDLSN, writer.getLastDLSN());
+        assertEquals("Position should still be " + numRecords,
+                10, writer.getPositionWithinLogSegment());
+
+        for (int i = 0; i < numRecords; i++) {
+            try {
+                Await.result(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",
+                        wce.getCause() instanceof BKTransmitException);
+                BKTransmitException bkte = (BKTransmitException) wce.getCause();
+                assertEquals("Record " + i + " should be aborted",
+                        rcToFailComplete, bkte.getBKResultCode());
+            }
+        }
+
+        // check no entries were written
+        LedgerHandle lh = getLedgerHandle(writer);
+        LedgerHandle readLh = openLedgerNoRecovery(lh);
+        assertFalse("Ledger " + lh.getId() + " should not be closed", readLh.isClosed());
+        assertEquals("There should be no entries in ledger " + lh.getId(),
+                LedgerHandle.INVALID_ENTRY_ID, readLh.getLastAddConfirmed());
+    }
+
+    /**
+     * Close the writer when ledger is fenced: it should release the lock, fail on flushing data and throw exception
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testCloseShouldFailIfLedgerFenced() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
+        BKLogSegmentWriter writer =
+                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();
+        // add 10 records
+        int numRecords = 10;
+        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
+        for (int i = 0; i < numRecords; i++) {
+            futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
+        }
+        assertEquals("Last tx id should be " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxId());
+        assertEquals("Last acked tx id should be -1",
+                -1L, writer.getLastTxIdAcknowledged());
+        assertEquals("Last DLSN should be " + DLSN.InvalidDLSN,
+                DLSN.InvalidDLSN, writer.getLastDLSN());
+        assertEquals("Position should be " + numRecords,
+                10, writer.getPositionWithinLogSegment());
+        // fence the ledger
+        fenceLedger(getLedgerHandle(writer));
+        // close the writer: it should release the lock, fail on flushing data and throw exception
+        try {
+            closeWriterAndLock(writer, lock);
+            fail("Close a log segment writer when ledger is fenced should throw exception");
+        } catch (BKTransmitException bkte) {
+            assertEquals("Inconsistent rc is thrown",
+                    BKException.Code.LedgerFencedException, bkte.getBKResultCode());
+        }
+
+        Await.result(lockFuture0);
+        lock0.checkOwnership();
+
+        assertEquals("Last tx id should still be " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxId());
+        assertEquals("Last acked tx id should still be " + (numRecords - 1),
+                -1L, writer.getLastTxIdAcknowledged());
+        assertEquals("Last DLSN should still be " + DLSN.InvalidDLSN,
+                DLSN.InvalidDLSN, writer.getLastDLSN());
+        assertEquals("Position should still be " + numRecords,
+                10, writer.getPositionWithinLogSegment());
+
+        for (int i = 0; i < numRecords; i++) {
+            try {
+                Await.result(futureList.get(i));
+                fail("Should be aborted record " + i + " with transmit exception");
+            } catch (BKTransmitException bkte) {
+                assertEquals("Record " + i + " should be aborted",
+                        BKException.Code.LedgerFencedException, bkte.getBKResultCode());
+            }
+        }
+
+        // check no entries were written
+        LedgerHandle lh = getLedgerHandle(writer);
+        LedgerHandle readLh = openLedgerNoRecovery(lh);
+        assertTrue("Ledger " + lh.getId() + " should be closed", readLh.isClosed());
+        assertEquals("There should be no entries in ledger " + lh.getId(),
+                LedgerHandle.INVALID_ENTRY_ID, readLh.getLastAddConfirmed());
+    }
+
+    /**
+     * Abort should wait for outstanding transmits to be completed and cancel buffered data.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testAbortShouldFailAllWrites() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
+        BKLogSegmentWriter writer =
+                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();
+        // add 10 records
+        int numRecords = 10;
+        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
+        for (int i = 0; i < numRecords; i++) {
+            futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
+        }
+        assertEquals("Last tx id should be " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxId());
+        assertEquals("Last acked tx id should be -1",
+                -1L, writer.getLastTxIdAcknowledged());
+        assertEquals("Last DLSN should be " + DLSN.InvalidDLSN,
+                DLSN.InvalidDLSN, writer.getLastDLSN());
+        assertEquals("Position should be " + numRecords,
+                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;
+            }
+        });
+
+        // transmit the buffered data
+        FutureUtils.result(writer.flush());
+
+        // add another 10 records
+        List<Future<DLSN>> anotherFutureList = new ArrayList<Future<DLSN>>(numRecords);
+        for (int i = numRecords; i < 2 * numRecords; i++) {
+            anotherFutureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
+        }
+        assertEquals("Last tx id should become " + (2 * numRecords - 1),
+                2 * numRecords - 1, writer.getLastTxId());
+        assertEquals("Last acked tx id should become " + (numRecords - 1),
+                (long) (numRecords - 1), writer.getLastTxIdAcknowledged());
+        assertEquals("Last DLSN should still be " + DLSN.InvalidDLSN,
+                DLSN.InvalidDLSN, writer.getLastDLSN());
+        assertEquals("Position should become " + (2 * numRecords),
+                2 * numRecords, writer.getPositionWithinLogSegment());
+
+        // abort the writer: it waits for outstanding transmits and abort buffered data
+        abortWriterAndLock(writer, lock);
+
+        Await.result(lockFuture0);
+        lock0.checkOwnership();
+
+        // release defer latch so completion would go through
+        deferLatch.countDown();
+
+        List<DLSN> dlsns = Await.result(Future.collect(futureList));
+        assertEquals("All first 10 records should be written",
+                numRecords, dlsns.size());
+        for (int i = 0; i < numRecords; i++) {
+            DLSN dlsn = dlsns.get(i);
+            assertEquals("Incorrent ledger sequence number",
+                    0L, dlsn.getLogSegmentSequenceNo());
+            assertEquals("Incorrent entry id",
+                    0L, dlsn.getEntryId());
+            assertEquals("Inconsistent slot id",
+                    i, dlsn.getSlotId());
+        }
+        for (int i = 0; i < numRecords; i++) {
+            try {
+                Await.result(anotherFutureList.get(i));
+                fail("Should be aborted record " + (numRecords + i) + " with transmit exception");
+            } catch (WriteCancelledException wce) {
+                // writes should be cancelled.
+            }
+        }
+
+        assertEquals("Last tx id should still be " + (2 * numRecords - 1),
+                2 * numRecords - 1, writer.getLastTxId());
+        assertEquals("Last acked tx id should be still " + (numRecords - 1),
+                (long) (numRecords - 1), writer.getLastTxIdAcknowledged());
+        assertEquals("Last DLSN should become " + futureList.get(futureList.size() - 1),
+                dlsns.get(futureList.size() - 1), writer.getLastDLSN());
+        assertEquals("Position should become " + 2 * numRecords,
+                2 * numRecords, writer.getPositionWithinLogSegment());
+
+        // check only 1 entry were written
+        LedgerHandle lh = getLedgerHandle(writer);
+        LedgerHandle readLh = openLedgerNoRecovery(lh);
+        assertTrue("Ledger " + lh.getId() + " should not be closed", readLh.isClosed());
+        assertEquals("Only one entry is written for ledger " + lh.getId(),
+                0L, lh.getLastAddPushed());
+        assertEquals("Only one entry is written for ledger " + lh.getId(),
+                0L, readLh.getLastAddConfirmed());
+    }
+
+    /**
+     * Log Segment Writer should only update last tx id only for user records.
+     */
+    @Test(timeout = 60000)
+    public void testUpdateLastTxIdForUserRecords() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
+        BKLogSegmentWriter writer =
+                createLogSegmentWriter(confLocal, 0L, -1L, lock);
+        // add 10 records
+        int numRecords = 10;
+        List<Future<DLSN>> futureList = new ArrayList<Future<DLSN>>(numRecords);
+        for (int i = 0; i < numRecords; i++) {
+            futureList.add(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(i)));
+        }
+        LogRecord controlRecord = DLMTestUtil.getLogRecordInstance(9999L);
+        controlRecord.setControl();
+        futureList.add(writer.asyncWrite(controlRecord));
+        assertEquals("Last tx id should be " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxId());
+        assertEquals("Last DLSN should be " + DLSN.InvalidDLSN,
+                DLSN.InvalidDLSN, writer.getLastDLSN());
+        assertEquals("Position should be " + numRecords,
+                numRecords, writer.getPositionWithinLogSegment());
+
+        // close the writer to flush the output buffer
+        closeWriterAndLock(writer, lock);
+
+        List<DLSN> dlsns = Await.result(Future.collect(futureList));
+        assertEquals("All 11 records should be written",
+                numRecords + 1, dlsns.size());
+        for (int i = 0; i < numRecords; i++) {
+            DLSN dlsn = dlsns.get(i);
+            assertEquals("Incorrent ledger sequence number",
+                    0L, dlsn.getLogSegmentSequenceNo());
+            assertEquals("Incorrent entry id",
+                    0L, dlsn.getEntryId());
+            assertEquals("Inconsistent slot id",
+                    i, dlsn.getSlotId());
+        }
+        DLSN dlsn = dlsns.get(numRecords);
+        assertEquals("Incorrent ledger sequence number",
+                0L, dlsn.getLogSegmentSequenceNo());
+        assertEquals("Incorrent entry id",
+                1L, dlsn.getEntryId());
+        assertEquals("Inconsistent slot id",
+                0L, dlsn.getSlotId());
+
+        assertEquals("Last tx id should be " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxId());
+        assertEquals("Last acked tx id should be " + (numRecords - 1),
+                numRecords - 1, writer.getLastTxIdAcknowledged());
+        assertEquals("Position should be " + numRecords,
+                numRecords, writer.getPositionWithinLogSegment());
+        assertEquals("Last DLSN should be " + dlsn,
+                dlsns.get(numRecords - 1), writer.getLastDLSN());
+    }
+
+    /**
+     * Non durable write should fail if writer is closed.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testNondurableWriteAfterWriterIsClosed() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setDurableWriteEnabled(false);
+        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
+        BKLogSegmentWriter writer =
+                createLogSegmentWriter(confLocal, 0L, -1L, lock);
+
+        // close the writer
+        closeWriterAndLock(writer, lock);
+        FutureUtils.result(writer.asyncClose());
+
+        try {
+            Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(1)));
+            fail("Should fail the write if the writer is closed");
+        } catch (WriteException we) {
+            // expected
+        }
+    }
+
+    /**
+     * Non durable write should fail if writer is marked as end of stream.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testNondurableWriteAfterEndOfStream() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setDurableWriteEnabled(false);
+        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
+        BKLogSegmentWriter writer =
+                createLogSegmentWriter(confLocal, 0L, -1L, lock);
+
+        FutureUtils.result(writer.markEndOfStream());
+
+        try {
+            Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(1)));
+            fail("Should fail the write if the writer is marked as end of stream");
+        } catch (EndOfStreamException we) {
+            // expected
+        }
+
+        closeWriterAndLock(writer, lock);
+    }
+
+    /**
+     * Non durable write should fail if the log segment is fenced.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testNondurableWriteAfterLedgerIsFenced() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setDurableWriteEnabled(false);
+        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
+        BKLogSegmentWriter writer =
+                createLogSegmentWriter(confLocal, 0L, -1L, lock);
+
+        // fence the ledger
+        fenceLedger(getLedgerHandle(writer));
+
+        LogRecord record = DLMTestUtil.getLogRecordInstance(1);
+        record.setControl();
+        try {
+            Await.result(writer.asyncWrite(record));
+            fail("Should fail the writer if the log segment is already fenced");
+        } catch (BKTransmitException bkte) {
+            // expected
+            assertEquals(BKException.Code.LedgerFencedException, bkte.getBKResultCode());
+        }
+
+        try {
+            Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(2)));
+            fail("Should fail the writer if the log segment is already fenced");
+        } catch (WriteException we) {
+            // expected
+        }
+
+        abortWriterAndLock(writer, lock);
+    }
+
+    /**
+     * Non durable write should fail if writer is marked as end of stream.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testNondurableWrite() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setDurableWriteEnabled(false);
+        ZKDistributedLock lock = createLock("/test/lock-" + runtime.getMethodName(), zkc, true);
+        BKLogSegmentWriter writer =
+                createLogSegmentWriter(confLocal, 0L, -1L, lock);
+
+        assertEquals(DLSN.InvalidDLSN,
+                Await.result(writer.asyncWrite(DLMTestUtil.getLogRecordInstance(2))));
+        assertEquals(-1L, ((BKLogSegmentEntryWriter) writer.getEntryWriter())
+                .getLedgerHandle().getLastAddPushed());
+
+        closeWriterAndLock(writer, lock);
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java
new file mode 100644
index 0000000..2566d34
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKLogWriteHandler.java
@@ -0,0 +1,90 @@
+/**
+ * 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 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.util.FailpointUtils;
+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;
+
+import java.io.IOException;
+import java.net.URI;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test {@link BKLogWriteHandler}
+ */
+public class TestBKLogWriteHandler extends TestDistributedLogBase {
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    /**
+     * Testcase: when write handler encounters exceptions on starting log segment
+     * it should abort the transaction and return the ledger to the pool.
+     */
+    @Test(timeout = 60000)
+    public void testAbortTransactionOnStartLogSegment() throws Exception {
+        URI uri = createDLMURI("/" + runtime.getMethodName());
+        ensureURICreated(zkc, uri);
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setEnableLedgerAllocatorPool(true);
+        confLocal.setLedgerAllocatorPoolCoreSize(1);
+        confLocal.setLedgerAllocatorPoolName("test-allocator-pool");
+
+        BKDistributedLogNamespace namespace = (BKDistributedLogNamespace)
+                DistributedLogNamespaceBuilder.newBuilder()
+                        .conf(confLocal)
+                        .uri(uri)
+                        .build();
+        DistributedLogManager dlm = namespace.openLog("test-stream");
+        FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_StartLogSegmentOnAssignLogSegmentSequenceNumber,
+                FailpointUtils.FailPointActions.FailPointAction_Throw);
+        try {
+            AsyncLogWriter writer =  FutureUtils.result(dlm.openAsyncLogWriter());
+            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1L)));
+            fail("Should fail opening the writer");
+        } catch (IOException ioe) {
+            // expected
+        } finally {
+            FailpointUtils.removeFailpoint(
+                    FailpointUtils.FailPointName.FP_StartLogSegmentOnAssignLogSegmentSequenceNumber);
+        }
+
+        LedgerAllocator allocator = ((BKNamespaceDriver) namespace.getNamespaceDriver())
+                .getLedgerAllocator();
+        assertTrue(allocator instanceof LedgerAllocatorPool);
+        LedgerAllocatorPool allocatorPool = (LedgerAllocatorPool) allocator;
+        assertEquals(0, allocatorPool.obtainMapSize());
+
+        AsyncLogWriter writer = FutureUtils.result(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
new file mode 100644
index 0000000..bb8503f
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestBKSyncLogReader.java
@@ -0,0 +1,306 @@
+/**
+ * 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 org.apache.distributedlog.exceptions.LogNotFoundException;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test Sync Log Reader
+ */
+public class TestBKSyncLogReader extends TestDistributedLogBase {
+
+    static final Logger logger = LoggerFactory.getLogger(TestBKSyncLogReader.class);
+
+    @Rule
+    public TestName testName = new TestName();
+
+    @Test(timeout = 60000)
+    public void testCreateReaderBeyondLastTransactionId() throws Exception {
+        String name = testName.getMethodName();
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+        for (long i = 1; i < 10; i++) {
+            LogRecord op = DLMTestUtil.getLogRecordInstance(i);
+            out.write(op);
+        }
+        out.closeAndComplete();
+
+        LogReader reader = dlm.getInputStream(20L);
+        assertNull(reader.readNext(false));
+
+        // write another 20 records
+        out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+        for (long i = 10; i < 30; i++) {
+            LogRecord op = DLMTestUtil.getLogRecordInstance(i);
+            out.write(op);
+        }
+        out.closeAndComplete();
+
+        for (int i = 0; i < 10; i++) {
+            LogRecord record = waitForNextRecord(reader);
+            assertEquals(20L + i, record.getTransactionId());
+        }
+        assertNull(reader.readNext(false));
+    }
+
+    @Test(timeout = 60000)
+    public void testDeletingLogWhileReading() throws Exception {
+        String name = testName.getMethodName();
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+        for (long i = 1; i < 10; i++) {
+            LogRecord op = DLMTestUtil.getLogRecordInstance(i);
+            out.write(op);
+        }
+        out.closeAndComplete();
+
+        LogReader reader = dlm.getInputStream(1L);
+        for (int i = 1; i < 10; i++) {
+            LogRecord record = waitForNextRecord(reader);
+            assertEquals((long) i, record.getTransactionId());
+        }
+
+        DistributedLogManager deleteDLM = createNewDLM(conf, name);
+        deleteDLM.delete();
+
+        LogRecord record;
+        try {
+            record = reader.readNext(false);
+            while (null == record) {
+                record = reader.readNext(false);
+            }
+            fail("Should fail reading next with LogNotFound");
+        } catch (LogNotFoundException lnfe) {
+            // expected
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testReadingFromEmptyLog() throws Exception {
+        String name = testName.getMethodName();
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(Integer.MAX_VALUE);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+        // write a record but not commit
+        LogRecord op = DLMTestUtil.getLogRecordInstance(1L);
+        out.write(op);
+
+        LogReader reader = dlm.getInputStream(1L);
+        assertNull(reader.readNext(true));
+        assertNull(reader.readNext(false));
+
+        op = DLMTestUtil.getLogRecordInstance(2L);
+        out.write(op);
+
+        // reader is able to read first record
+        LogRecord record = waitForNextRecord(reader);
+        assertNotNull(record);
+        assertEquals(1L, record.getTransactionId());
+        DLMTestUtil.verifyLogRecord(record);
+
+        assertNull(reader.readNext(true));
+
+        out.close();
+        reader.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReadRecordsAfterReadAheadCaughtUp() throws Exception {
+        String name = testName.getMethodName();
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(Integer.MAX_VALUE);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+        for (long i = 1L; i <= 10L; i++) {
+            LogRecord record = DLMTestUtil.getLogRecordInstance(i);
+            out.write(record);
+        }
+        out.setReadyToFlush();
+        out.flushAndSync();
+
+        logger.info("Write first 10 records");
+
+        // all 10 records are added to the stream
+        // then open a reader to read
+        BKSyncLogReader reader = (BKSyncLogReader) dlm.getInputStream(1L);
+
+        // wait until readahead caught up
+        while (!reader.getReadAheadReader().isReadAheadCaughtUp()) {
+            TimeUnit.MILLISECONDS.sleep(20);
+        }
+
+        logger.info("ReadAhead is caught up with first 10 records");
+
+        for (long i = 11L; i <= 20L; i++) {
+            LogRecord record = DLMTestUtil.getLogRecordInstance(i);
+            out.write(record);
+        }
+        out.setReadyToFlush();
+        out.flushAndSync();
+
+        logger.info("Write another 10 records");
+
+        // wait until readahead move on
+        while (reader.getReadAheadReader().getNextEntryPosition().getEntryId() < 21) {
+            TimeUnit.MILLISECONDS.sleep(20);
+        }
+
+        logger.info("ReadAhead is caught up with another 10 records");
+
+        // resume reading from sync reader. so it should be able to read all 20 records
+        // and return null to claim it as caughtup
+        LogRecord record = reader.readNext(false);
+        int numReads = 0;
+        long expectedTxId = 1L;
+        while (null != record) {
+            ++numReads;
+            assertEquals(expectedTxId, record.getTransactionId());
+            DLMTestUtil.verifyLogRecord(record);
+            ++expectedTxId;
+            record = reader.readNext(false);
+        }
+        assertEquals(20, numReads);
+
+        out.close();
+        reader.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReadRecordsWhenReadAheadCatchingUp() throws Exception {
+        String name = testName.getMethodName();
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(Integer.MAX_VALUE);
+        confLocal.setReadAheadMaxRecords(1);
+        confLocal.setReadAheadBatchSize(1);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+        for (long i = 1L; i <= 10L; i++) {
+            LogRecord record = DLMTestUtil.getLogRecordInstance(i);
+            out.write(record);
+        }
+        out.setReadyToFlush();
+        out.flushAndSync();
+
+        logger.info("Write first 10 records");
+
+        // open a reader to read
+        BKSyncLogReader reader = (BKSyncLogReader) dlm.getInputStream(1L);
+        // resume reading from sync reader. so it should be able to read all 10 records
+        // and return null to claim it as caughtup
+        LogRecord record = reader.readNext(false);
+        int numReads = 0;
+        long expectedTxId = 1L;
+        while (null != record) {
+            ++numReads;
+            assertEquals(expectedTxId, record.getTransactionId());
+            DLMTestUtil.verifyLogRecord(record);
+            ++expectedTxId;
+            record = reader.readNext(false);
+        }
+        assertEquals(10, numReads);
+
+        out.close();
+        reader.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReadRecordsWhenReadAheadCatchingUp2() throws Exception {
+        String name = testName.getMethodName();
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(Integer.MAX_VALUE);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        final BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+        for (long i = 1L; i <= 10L; i++) {
+            LogRecord record = DLMTestUtil.getLogRecordInstance(i);
+            out.write(record);
+        }
+        out.setReadyToFlush();
+        out.flushAndSync();
+        final AtomicLong nextTxId = new AtomicLong(11L);
+
+        logger.info("Write first 10 records");
+
+        ScheduledExecutorService executorService =
+                Executors.newSingleThreadScheduledExecutor();
+        executorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                long txid = nextTxId.getAndIncrement();
+                LogRecord record = DLMTestUtil.getLogRecordInstance(txid);
+                try {
+                    out.write(record);
+                } catch (IOException e) {
+                    // ignore the ioe
+                }
+            }
+        }, 0, 400, TimeUnit.MILLISECONDS);
+
+        // open a reader to read
+        BKSyncLogReader reader = (BKSyncLogReader) dlm.getInputStream(1L);
+        // resume reading from sync reader. so it should be able to read all 10 records
+        // and return null to claim it as caughtup
+        LogRecord record = reader.readNext(false);
+        int numReads = 0;
+        long expectedTxId = 1L;
+        while (null != record) {
+            ++numReads;
+            assertEquals(expectedTxId, record.getTransactionId());
+            DLMTestUtil.verifyLogRecord(record);
+            ++expectedTxId;
+            record = reader.readNext(false);
+        }
+        assertTrue(numReads >= 10);
+
+        executorService.shutdown();
+        out.close();
+        reader.close();
+        dlm.close();
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java
new file mode 100644
index 0000000..a4db3ad
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDLMTestUtil.java
@@ -0,0 +1,62 @@
+/**
+ * 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 java.io.File;
+
+import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
+import org.apache.bookkeeper.util.IOUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestDLMTestUtil {
+    static final Logger LOG = LoggerFactory.getLogger(TestDLMTestUtil.class);
+
+    @Rule
+    public TestName testNames = new TestName();
+
+    @Test(timeout = 60000)
+    public void testRunZookeeperOnAnyPort() throws Exception {
+        Pair<ZooKeeperServerShim, Integer> serverAndPort1 = null;
+        Pair<ZooKeeperServerShim, Integer> serverAndPort2 = null;
+        Pair<ZooKeeperServerShim, Integer> serverAndPort3 = null;
+        try {
+            File zkTmpDir1 = IOUtils.createTempDir("zookeeper1", "distrlog");
+            serverAndPort1 = LocalDLMEmulator.runZookeeperOnAnyPort(7000, zkTmpDir1);
+            File zkTmpDir2 = IOUtils.createTempDir("zookeeper2", "distrlog");
+            serverAndPort2 = LocalDLMEmulator.runZookeeperOnAnyPort(7000, zkTmpDir2);
+            File zkTmpDir3 = IOUtils.createTempDir("zookeeper3", "distrlog");
+            serverAndPort3 = LocalDLMEmulator.runZookeeperOnAnyPort(7000, zkTmpDir3);
+        } catch (Exception ex) {
+            if (null != serverAndPort1) {
+                serverAndPort1.getLeft().stop();
+            }
+            if (null != serverAndPort2) {
+                serverAndPort2.getLeft().stop();
+            }
+            if (null != serverAndPort3) {
+                serverAndPort3.getLeft().stop();
+            }
+        }
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java
new file mode 100644
index 0000000..2769175
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogBase.java
@@ -0,0 +1,244 @@
+/**
+ * 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.assertTrue;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Ticker;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.distributedlog.impl.logsegment.BKLogSegmentEntryWriter;
+import org.apache.distributedlog.injector.AsyncFailureInjector;
+import org.apache.distributedlog.injector.AsyncRandomFailureInjector;
+import org.apache.distributedlog.io.AsyncCloseable;
+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.namespace.NamespaceDriver;
+import org.apache.distributedlog.util.ConfUtils;
+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.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.feature.SettableFeatureProvider;
+import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.util.IOUtils;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+public class TestDistributedLogBase {
+    static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogBase.class);
+
+    // Num worker threads should be one, since the exec service is used for the ordered
+    // future pool in test cases, and setting to > 1 will therefore result in unordered
+    // write ops.
+    protected static DistributedLogConfiguration conf =
+        new DistributedLogConfiguration()
+                .setEnableReadAhead(true)
+                .setReadAheadMaxRecords(1000)
+                .setReadAheadBatchSize(10)
+                .setLockTimeout(1)
+                .setNumWorkerThreads(1)
+                .setReadAheadNoSuchLedgerExceptionOnReadLACErrorThresholdMillis(20)
+                .setSchedulerShutdownTimeoutMs(0)
+                .setDLLedgerMetadataLayoutVersion(LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
+    protected ZooKeeper zkc;
+    protected static LocalDLMEmulator bkutil;
+    protected static ZooKeeperServerShim zks;
+    protected static String zkServers;
+    protected static int zkPort;
+    protected static int numBookies = 3;
+    protected static final List<File> tmpDirs = new ArrayList<File>();
+
+    @BeforeClass
+    public static void setupCluster() throws Exception {
+        File zkTmpDir = IOUtils.createTempDir("zookeeper", "distrlog");
+        tmpDirs.add(zkTmpDir);
+        Pair<ZooKeeperServerShim, Integer> serverAndPort = LocalDLMEmulator.runZookeeperOnAnyPort(zkTmpDir);
+        zks = serverAndPort.getLeft();
+        zkPort = serverAndPort.getRight();
+        bkutil = LocalDLMEmulator.newBuilder()
+                .numBookies(numBookies)
+                .zkHost("127.0.0.1")
+                .zkPort(zkPort)
+                .serverConf(DLMTestUtil.loadTestBkConf())
+                .shouldStartZK(false)
+                .build();
+        bkutil.start();
+        zkServers = "127.0.0.1:" + zkPort;
+        Thread.setDefaultUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+            @Override
+            public void uncaughtException(Thread t, Throwable e) {
+                LOG.warn("Uncaught exception at Thread {} : ", t.getName(), e);
+            }
+        });
+    }
+
+    @AfterClass
+    public static void teardownCluster() throws Exception {
+        bkutil.teardown();
+        zks.stop();
+        for (File dir : tmpDirs) {
+            FileUtils.deleteDirectory(dir);
+        }
+    }
+
+    @Before
+    public void setup() throws Exception {
+        try {
+            zkc = LocalDLMEmulator.connectZooKeeper("127.0.0.1", zkPort);
+        } catch (Exception ex) {
+            LOG.error("hit exception connecting to zookeeper at {}:{}", new Object[] { "127.0.0.1", zkPort, ex });
+            throw ex;
+        }
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (null != zkc) {
+            zkc.close();
+        }
+    }
+
+    protected LogRecord waitForNextRecord(LogReader reader) throws Exception {
+        LogRecord record = reader.readNext(false);
+        while (null == record) {
+            record = reader.readNext(false);
+        }
+        return record;
+    }
+
+    public URI createDLMURI(String path) throws Exception {
+        return DLMTestUtil.createDLMURI(zkPort, path);
+    }
+
+    protected void ensureURICreated(URI uri) throws Exception {
+        ensureURICreated(zkc, uri);
+    }
+
+    protected void ensureURICreated(ZooKeeper zkc, URI uri) throws Exception {
+        try {
+            zkc.create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        } catch (KeeperException.NodeExistsException nee) {
+            // ignore
+        }
+    }
+
+    public BKDistributedLogManager createNewDLM(DistributedLogConfiguration conf,
+                                                String name) throws Exception {
+        return createNewDLM(conf, name, PermitLimiter.NULL_PERMIT_LIMITER);
+    }
+
+    public BKDistributedLogManager createNewDLM(DistributedLogConfiguration conf,
+                                                String name,
+                                                PermitLimiter writeLimiter)
+            throws Exception {
+        URI uri = createDLMURI("/" + name);
+        ensureURICreated(uri);
+        final DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .uri(uri)
+                .conf(conf)
+                .build();
+        final OrderedScheduler scheduler = OrderedScheduler.newBuilder()
+                .corePoolSize(1)
+                .name("test-scheduler")
+                .build();
+        AsyncCloseable resourcesCloseable = new AsyncCloseable() {
+            @Override
+            public Future<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();
+            }
+        };
+        AsyncFailureInjector failureInjector = AsyncRandomFailureInjector.newBuilder()
+                .injectDelays(conf.getEIInjectReadAheadDelay(),
+                        conf.getEIInjectReadAheadDelayPercent(),
+                        conf.getEIInjectMaxReadAheadDelayMs())
+                .injectErrors(false, 10)
+                .injectStops(conf.getEIInjectReadAheadStall(), 10)
+                .injectCorruption(conf.getEIInjectReadAheadBrokenEntries())
+                .build();
+        return new BKDistributedLogManager(
+                name,
+                conf,
+                ConfUtils.getConstDynConf(conf),
+                uri,
+                namespace.getNamespaceDriver(),
+                new LogSegmentMetadataCache(conf, Ticker.systemTicker()),
+                scheduler,
+                DistributedLogConstants.UNKNOWN_CLIENT_ID,
+                DistributedLogConstants.LOCAL_REGION_ID,
+                writeLimiter,
+                new SettableFeatureProvider("", 0),
+                failureInjector,
+                NullStatsLogger.INSTANCE,
+                NullStatsLogger.INSTANCE,
+                Optional.of(resourcesCloseable));
+    }
+
+    protected LogSegmentMetadataStore getLogSegmentMetadataStore(DistributedLogNamespace namespace)
+            throws IOException {
+        return namespace.getNamespaceDriver().getLogStreamMetadataStore(NamespaceDriver.Role.READER)
+                .getLogSegmentMetadataStore();
+    }
+
+    protected ZooKeeperClient getZooKeeperClient(DistributedLogNamespace namespace) throws Exception {
+        NamespaceDriver driver = namespace.getNamespaceDriver();
+        assertTrue(driver instanceof BKNamespaceDriver);
+        return ((BKNamespaceDriver) driver).getWriterZKC();
+    }
+
+    @SuppressWarnings("deprecation")
+    protected BookKeeperClient getBookKeeperClient(DistributedLogNamespace namespace) throws Exception {
+        NamespaceDriver driver = namespace.getNamespaceDriver();
+        assertTrue(driver instanceof BKNamespaceDriver);
+        return ((BKNamespaceDriver) driver).getReaderBKC();
+    }
+
+    protected LedgerHandle getLedgerHandle(BKLogSegmentWriter segmentWriter) {
+        LogSegmentEntryWriter entryWriter = segmentWriter.getEntryWriter();
+        assertTrue(entryWriter instanceof BKLogSegmentEntryWriter);
+        return ((BKLogSegmentEntryWriter) entryWriter).getLedgerHandle();
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java
new file mode 100644
index 0000000..e849465
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestDistributedLogConfiguration.java
@@ -0,0 +1,132 @@
+/**
+ * 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.base.Optional;
+
+import org.apache.distributedlog.net.DNSResolverForRacks;
+import org.apache.distributedlog.net.DNSResolverForRows;
+import org.apache.bookkeeper.net.DNSToSwitchMapping;
+import org.apache.commons.configuration.StrictConfigurationComparator;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+public class TestDistributedLogConfiguration {
+
+    static final class TestDNSResolver implements DNSToSwitchMapping {
+
+        public TestDNSResolver() {}
+
+        @Override
+        public List<String> resolve(List<String> list) {
+            return list;
+        }
+
+        @Override
+        public void reloadCachedMappings() {
+            // no-op
+        }
+    }
+
+    @Test(timeout = 20000)
+    public void loadStreamConfGoodOverrideAccepted() throws Exception {
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        assertEquals(conf.getPeriodicFlushFrequencyMilliSeconds(),
+            DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT);
+        assertEquals(conf.getReaderIdleErrorThresholdMillis(),
+            DistributedLogConfiguration.BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT);
+        DistributedLogConfiguration override = new DistributedLogConfiguration();
+        override.setPeriodicFlushFrequencyMilliSeconds(
+            DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT+1);
+        override.setReaderIdleErrorThresholdMillis(
+            DistributedLogConfiguration.BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT - 1);
+        conf.loadStreamConf(Optional.of(override));
+        assertEquals(conf.getPeriodicFlushFrequencyMilliSeconds(),
+            DistributedLogConfiguration.BKDL_PERIODIC_FLUSH_FREQUENCY_MILLISECONDS_DEFAULT+1);
+        assertEquals(conf.getReaderIdleErrorThresholdMillis(),
+            DistributedLogConfiguration.BKDL_READER_IDLE_ERROR_THRESHOLD_MILLIS_DEFAULT - 1);
+    }
+
+    @SuppressWarnings("deprecation")
+    @Test(timeout = 20000)
+    public void loadStreamConfBadOverrideIgnored() throws Exception {
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        assertEquals(conf.getBKClientWriteTimeout(),
+            DistributedLogConfiguration.BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT);
+        DistributedLogConfiguration override = new DistributedLogConfiguration();
+        override.setBKClientWriteTimeout(
+            DistributedLogConfiguration.BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT+1);
+        conf.loadStreamConf(Optional.of(override));
+        assertEquals(conf.getBKClientWriteTimeout(),
+            DistributedLogConfiguration.BKDL_BKCLIENT_WRITE_TIMEOUT_DEFAULT);
+    }
+
+    @Test(timeout = 20000)
+    public void loadStreamConfNullOverrides() throws Exception {
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        DistributedLogConfiguration confClone = (DistributedLogConfiguration)conf.clone();
+        Optional<DistributedLogConfiguration> streamConfiguration = Optional.absent();
+        conf.loadStreamConf(streamConfiguration);
+
+        StrictConfigurationComparator comp = new StrictConfigurationComparator();
+        assertTrue(comp.compare(conf, confClone));
+    }
+
+    @Test(timeout = 200000)
+    public void getEnsemblePlacementResolverClass() throws Exception {
+        DistributedLogConfiguration conf1 = new DistributedLogConfiguration();
+        assertEquals(DNSResolverForRacks.class, conf1.getEnsemblePlacementDnsResolverClass());
+        DistributedLogConfiguration conf2 = new DistributedLogConfiguration()
+                .setRowAwareEnsemblePlacementEnabled(true);
+        assertEquals(DNSResolverForRows.class, conf2.getEnsemblePlacementDnsResolverClass());
+        DistributedLogConfiguration conf3 = new DistributedLogConfiguration()
+                .setRowAwareEnsemblePlacementEnabled(true)
+                .setEnsemblePlacementDnsResolverClass(TestDNSResolver.class);
+        assertEquals(TestDNSResolver.class, conf3.getEnsemblePlacementDnsResolverClass());
+    }
+
+    @Test(timeout = 200000)
+    public void validateConfiguration(){
+        boolean exceptionThrown=false;
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        // validate default configuration
+        conf.validate();
+        // test equal, should not throw exception
+        conf.setReadLACLongPollTimeout(conf.getBKClientReadTimeout() * 1000);
+        try {
+            conf.validate();
+        } catch (IllegalArgumentException e){
+            exceptionThrown=true;
+        }
+        assertFalse(exceptionThrown);
+        // test invalid case, should throw exception
+        exceptionThrown=false;
+        conf.setReadLACLongPollTimeout(conf.getBKClientReadTimeout() * 1000 * 2);
+        try {
+            conf.validate();
+        } catch (IllegalArgumentException e){
+            exceptionThrown=true;
+        }
+        assertTrue(exceptionThrown);
+    }
+
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntry.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntry.java
new file mode 100644
index 0000000..6d8bd0c
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntry.java
@@ -0,0 +1,345 @@
+/**
+ * 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.base.Optional;
+import com.google.common.collect.Lists;
+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.junit.Assert;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+import static org.junit.Assert.*;
+
+/**
+ * Test Case of {@link Entry}
+ */
+public class TestEntry {
+
+    @Test(timeout = 20000)
+    public void testEmptyRecordSet() throws Exception {
+        Writer writer = Entry.newEntry(
+                "test-empty-record-set",
+                1024,
+                true,
+                CompressionCodec.Type.NONE,
+                NullStatsLogger.INSTANCE);
+        assertEquals("zero bytes", 0, writer.getNumBytes());
+        assertEquals("zero records", 0, writer.getNumRecords());
+
+        Buffer buffer = writer.getBuffer();
+        Entry recordSet = Entry.newBuilder()
+                .setData(buffer.getData(), 0, buffer.size())
+                .setLogSegmentInfo(1L, 0L)
+                .setEntryId(0L)
+                .build();
+        Reader reader = recordSet.reader();
+        Assert.assertNull("Empty record set should return null",
+                reader.nextRecord());
+    }
+
+    @Test(timeout = 20000)
+    public void testWriteTooLongRecord() throws Exception {
+        Writer writer = Entry.newEntry(
+                "test-write-too-long-record",
+                1024,
+                false,
+                CompressionCodec.Type.NONE,
+                NullStatsLogger.INSTANCE);
+        assertEquals("zero bytes", 0, writer.getNumBytes());
+        assertEquals("zero records", 0, writer.getNumRecords());
+
+        LogRecord largeRecord = new LogRecord(1L, new byte[MAX_LOGRECORD_SIZE + 1]);
+        try {
+            writer.writeRecord(largeRecord, new Promise<DLSN>());
+            Assert.fail("Should fail on writing large record");
+        } catch (LogRecordTooLongException lrtle) {
+            // expected
+        }
+        assertEquals("zero bytes", 0, writer.getNumBytes());
+        assertEquals("zero records", 0, writer.getNumRecords());
+
+        Buffer buffer = writer.getBuffer();
+        Assert.assertEquals("zero bytes", 0, buffer.size());
+    }
+
+    @Test(timeout = 20000)
+    public void testWriteRecords() throws Exception {
+        Writer writer = Entry.newEntry(
+                "test-write-records",
+                1024,
+                true,
+                CompressionCodec.Type.NONE,
+                NullStatsLogger.INSTANCE);
+        assertEquals("zero bytes", 0, writer.getNumBytes());
+        assertEquals("zero records", 0, writer.getNumRecords());
+
+        List<Future<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>();
+            writer.writeRecord(record, writePromise);
+            writePromiseList.add(writePromise);
+            assertEquals((i + 1) + " records", (i + 1), writer.getNumRecords());
+        }
+
+        // write large record
+        LogRecord largeRecord = new LogRecord(1L, new byte[MAX_LOGRECORD_SIZE + 1]);
+        try {
+            writer.writeRecord(largeRecord, new Promise<DLSN>());
+            Assert.fail("Should fail on writing large record");
+        } catch (LogRecordTooLongException lrtle) {
+            // expected
+        }
+        assertEquals("5 records", 5, writer.getNumRecords());
+
+        // write another 5 records
+        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>();
+            writer.writeRecord(record, writePromise);
+            writePromiseList.add(writePromise);
+            assertEquals((i + 6) + " records", (i + 6), writer.getNumRecords());
+        }
+
+        Buffer buffer = writer.getBuffer();
+
+        // Test transmit complete
+        writer.completeTransmit(1L, 1L);
+        List<DLSN> writeResults = Await.result(Future.collect(writePromiseList));
+        for (int i = 0; i < 10; i++) {
+            Assert.assertEquals(new DLSN(1L, 1L, i), writeResults.get(i));
+        }
+
+        // Test reading from buffer
+        Entry recordSet = Entry.newBuilder()
+                .setData(buffer.getData(), 0, buffer.size())
+                .setLogSegmentInfo(1L, 1L)
+                .setEntryId(0L)
+                .build();
+        Reader reader = recordSet.reader();
+        LogRecordWithDLSN record = reader.nextRecord();
+        int numReads = 0;
+        long expectedTxid = 0L;
+        while (null != record) {
+            Assert.assertEquals(expectedTxid, record.getTransactionId());
+            Assert.assertEquals(expectedTxid, record.getSequenceId());
+            Assert.assertEquals(new DLSN(1L, 0L, expectedTxid), record.getDlsn());
+            ++numReads;
+            ++expectedTxid;
+            record = reader.nextRecord();
+        }
+        Assert.assertEquals(10, numReads);
+    }
+
+    @Test(timeout = 20000)
+    public void testWriteRecordSet() throws Exception {
+        Writer writer = Entry.newEntry(
+                "test-write-recordset",
+                1024,
+                true,
+                CompressionCodec.Type.NONE,
+                NullStatsLogger.INSTANCE);
+        assertEquals("zero bytes", 0, writer.getNumBytes());
+        assertEquals("zero records", 0, writer.getNumRecords());
+
+        List<Future<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>();
+            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();
+        // 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>();
+            recordSetWriter.writeRecord(record, writePromise);
+            recordSetPromiseList.add(writePromise);
+            assertEquals((i + 1) + " records", (i + 1), recordSetWriter.getNumRecords());
+        }
+        final ByteBuffer recordSetBuffer = recordSetWriter.getBuffer();
+        byte[] data = new byte[recordSetBuffer.remaining()];
+        recordSetBuffer.get(data);
+        LogRecord setRecord = new LogRecord(5L, data);
+        setRecord.setPositionWithinLogSegment(5);
+        setRecord.setRecordSet();
+        Promise<DLSN> writePromise = new Promise<DLSN>();
+        writePromise.addEventListener(new FutureEventListener<DLSN>() {
+            @Override
+            public void onSuccess(DLSN dlsn) {
+                recordSetWriter.completeTransmit(
+                        dlsn.getLogSegmentSequenceNo(),
+                        dlsn.getEntryId(),
+                        dlsn.getSlotId());
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                recordSetWriter.abortTransmit(cause);
+            }
+        });
+        writer.writeRecord(setRecord, writePromise);
+        writePromiseList.add(writePromise);
+
+        // write last 5 records
+        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>();
+            writer.writeRecord(record, writePromise);
+            writePromiseList.add(writePromise);
+            assertEquals((i + 11) + " records", (i + 11), writer.getNumRecords());
+        }
+
+        Buffer buffer = writer.getBuffer();
+
+        // Test transmit complete
+        writer.completeTransmit(1L, 1L);
+        List<DLSN> writeResults = Await.result(Future.collect(writePromiseList));
+        for (int i = 0; i < 5; i++) {
+            Assert.assertEquals(new DLSN(1L, 1L, i), writeResults.get(i));
+        }
+        Assert.assertEquals(new DLSN(1L, 1L, 5), writeResults.get(5));
+        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));
+        for (int i = 0; i < 5; i++) {
+            Assert.assertEquals(new DLSN(1L, 1L, (5 + i)), recordSetWriteResults.get(i));
+        }
+
+        // Test reading from buffer
+        verifyReadResult(buffer, 1L, 1L, 1L, true,
+                new DLSN(1L, 1L, 2L), 3, 5, 5,
+                new DLSN(1L, 1L, 2L), 2L);
+        verifyReadResult(buffer, 1L, 1L, 1L, true,
+                new DLSN(1L, 1L, 7L), 0, 3, 5,
+                new DLSN(1L, 1L, 7L), 7L);
+        verifyReadResult(buffer, 1L, 1L, 1L, true,
+                new DLSN(1L, 1L, 12L), 0, 0, 3,
+                new DLSN(1L, 1L, 12L), 12L);
+        verifyReadResult(buffer, 1L, 1L, 1L, false,
+                new DLSN(1L, 1L, 2L), 3, 5, 5,
+                new DLSN(1L, 1L, 2L), 2L);
+        verifyReadResult(buffer, 1L, 1L, 1L, false,
+                new DLSN(1L, 1L, 7L), 0, 3, 5,
+                new DLSN(1L, 1L, 7L), 7L);
+        verifyReadResult(buffer, 1L, 1L, 1L, false,
+                new DLSN(1L, 1L, 12L), 0, 0, 3,
+                new DLSN(1L, 1L, 12L), 12L);
+    }
+
+    void verifyReadResult(Buffer data,
+                          long lssn, long entryId, long startSequenceId,
+                          boolean deserializeRecordSet,
+                          DLSN skipTo,
+                          int firstNumRecords,
+                          int secondNumRecords,
+                          int lastNumRecords,
+                          DLSN expectedDLSN,
+                          long expectedTxId) throws Exception {
+        Entry recordSet = Entry.newBuilder()
+                .setData(data.getData(), 0, data.size())
+                .setLogSegmentInfo(lssn, startSequenceId)
+                .setEntryId(entryId)
+                .deserializeRecordSet(deserializeRecordSet)
+                .skipTo(skipTo)
+                .build();
+        Reader reader = recordSet.reader();
+
+        LogRecordWithDLSN record;
+        for (int i = 0; i < firstNumRecords; i++) { // first
+            record = reader.nextRecord();
+            assertNotNull(record);
+            assertEquals(expectedDLSN, record.getDlsn());
+            assertEquals(expectedTxId, record.getTransactionId());
+            assertNotNull("record " + record + " payload is null",
+                    record.getPayload());
+            assertEquals("record-" + expectedTxId, new String(record.getPayload(), UTF_8));
+            expectedDLSN = expectedDLSN.getNextDLSN();
+            ++expectedTxId;
+        }
+
+        boolean verifyDeserializedRecords = true;
+        if (firstNumRecords > 0) {
+            verifyDeserializedRecords = deserializeRecordSet;
+        }
+        if (verifyDeserializedRecords) {
+            long txIdOfRecordSet = 5;
+            for (int i = 0; i < secondNumRecords; i++) {
+                record = reader.nextRecord();
+                assertNotNull(record);
+                assertEquals(expectedDLSN, record.getDlsn());
+                assertEquals(txIdOfRecordSet, record.getTransactionId());
+                assertNotNull("record " + record + " payload is null",
+                        record.getPayload());
+                assertEquals("record-" + expectedTxId, new String(record.getPayload(), UTF_8));
+                expectedDLSN = expectedDLSN.getNextDLSN();
+                ++expectedTxId;
+            }
+        } else {
+            record = reader.nextRecord();
+            assertNotNull(record);
+            assertEquals(expectedDLSN, record.getDlsn());
+            assertEquals(expectedTxId, record.getTransactionId());
+            for (int i = 0; i < secondNumRecords; i++) {
+                expectedDLSN = expectedDLSN.getNextDLSN();
+                ++expectedTxId;
+            }
+        }
+
+        for (int i = 0; i < lastNumRecords; i++) {
+            record = reader.nextRecord();
+            assertNotNull(record);
+            assertEquals(expectedDLSN, record.getDlsn());
+            assertEquals(expectedTxId, record.getTransactionId());
+            assertNotNull("record " + record + " payload is null",
+                    record.getPayload());
+            assertEquals("record-" + expectedTxId, new String(record.getPayload(), UTF_8));
+            expectedDLSN = expectedDLSN.getNextDLSN();
+            ++expectedTxId;
+        }
+
+    }
+
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntryPosition.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntryPosition.java
new file mode 100644
index 0000000..60a0dab
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestEntryPosition.java
@@ -0,0 +1,59 @@
+/**
+ * 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 org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test Case for {@link EntryPosition}
+ */
+public class TestEntryPosition {
+
+    private void checkPosition(EntryPosition position,
+                               long lssn,
+                               long entryId) {
+        assertEquals(position.getLogSegmentSequenceNumber(), lssn);
+        assertEquals(position.getEntryId(), entryId);
+    }
+
+    @Test
+    public void testAdvance() {
+        EntryPosition position = new EntryPosition(9L, 99L);
+
+        checkPosition(position, 9L, 99L);
+
+        // advance (8L, 100L) takes no effect
+        assertFalse(position.advance(8L, 100L));
+        checkPosition(position, 9L, 99L);
+        // advance (9L, 98L) takes no effect
+        assertFalse(position.advance(9L, 98L));
+        checkPosition(position, 9L, 99L);
+        // advance (9L, 99L) takes no effect
+        assertFalse(position.advance(9L, 99L));
+        checkPosition(position, 9L, 99L);
+        // advance (9L, 100L) takes effects
+        assertTrue(position.advance(9L, 100L));
+        checkPosition(position, 9L, 100L);
+        // advance (10L, 0L) takes effects
+        assertTrue(position.advance(10L, 0L));
+        checkPosition(position, 10L, 0L);
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestEnvelopedEntry.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestEnvelopedEntry.java
new file mode 100644
index 0000000..6d78f03
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestEnvelopedEntry.java
@@ -0,0 +1,81 @@
+/**
+ * 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 java.io.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+
+import org.apache.distributedlog.io.Buffer;
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class TestEnvelopedEntry {
+
+    static final Logger LOG = LoggerFactory.getLogger(TestEnvelopedEntry.class);
+
+    private String getString(boolean compressible) {
+        if (compressible) {
+            StringBuilder builder = new StringBuilder();
+            for(int i = 0; i < 1000; i++) {
+                builder.append('A');
+            }
+            return builder.toString();
+        }
+        return "DistributedLogEnvelopedEntry";
+    }
+
+    @Test(timeout = 20000)
+    public void testEnvelope() throws Exception {
+        byte[] data = getString(false).getBytes();
+        EnvelopedEntry writeEntry = new EnvelopedEntry(EnvelopedEntry.CURRENT_VERSION,
+                                                  CompressionCodec.Type.NONE,
+                                                  data,
+                                                  data.length,
+                                                  new NullStatsLogger());
+        Buffer outBuf = new Buffer(2 * data.length);
+        writeEntry.writeFully(new DataOutputStream(outBuf));
+        EnvelopedEntry readEntry = new EnvelopedEntry(EnvelopedEntry.CURRENT_VERSION,
+                                                      new NullStatsLogger());
+        readEntry.readFully(new DataInputStream(new ByteArrayInputStream(outBuf.getData())));
+        byte[] newData = readEntry.getDecompressedPayload();
+        Assert.assertEquals("Written data should equal read data", new String(data), new String(newData));
+    }
+
+    @Test(timeout = 20000)
+    public void testLZ4Compression() throws Exception {
+        byte[] data = getString(true).getBytes();
+        EnvelopedEntry writeEntry = new EnvelopedEntry(EnvelopedEntry.CURRENT_VERSION,
+                                                       CompressionCodec.Type.LZ4,
+                                                       data,
+                                                       data.length,
+                                                       new NullStatsLogger());
+        Buffer outBuf = new Buffer(data.length);
+        writeEntry.writeFully(new DataOutputStream(outBuf));
+        Assert.assertTrue(data.length > outBuf.size());
+        EnvelopedEntry readEntry = new EnvelopedEntry(EnvelopedEntry.CURRENT_VERSION,
+                                                      new NullStatsLogger());
+        readEntry.readFully(new DataInputStream(new ByteArrayInputStream(outBuf.getData())));
+        byte[] newData = readEntry.getDecompressedPayload();
+        Assert.assertEquals("Written data should equal read data", new String(data), new String(newData));
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java
new file mode 100644
index 0000000..fd3c4ee
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestInterleavedReaders.java
@@ -0,0 +1,338 @@
+/**
+ * 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 org.apache.distributedlog.util.FutureUtils;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+public class TestInterleavedReaders extends TestDistributedLogBase {
+    static final Logger LOG = LoggerFactory.getLogger(TestInterleavedReaders.class);
+
+    static {
+        conf.setOutputBufferSize(0);
+        conf.setImmediateFlushEnabled(true);
+    }
+
+    private int drainStreams(LogReader reader0, int num0, LogReader reader1, int num1)
+            throws Exception {
+        // Allow time for watches to fire
+        Thread.sleep(15);
+        int numTrans = 0;
+        LogRecord record;
+        int i = 0;
+        while (i < num0) {
+            record = reader0.readNext(false);
+            if (null != record) {
+                assertTrue((record.getTransactionId() % 2 == 0));
+                DLMTestUtil.verifyLogRecord(record);
+                numTrans++;
+                i++;
+                LOG.info("Read record {}", record);
+            }
+        }
+        i = 0;
+        while (i < num1) {
+            record = reader1.readNext(false);
+            if (null != record) {
+                assertTrue((record.getTransactionId() % 2 == 1));
+                DLMTestUtil.verifyLogRecord(record);
+                numTrans++;
+                i++;
+                LOG.info("Read record {}", record);
+            }
+        }
+        return numTrans;
+    }
+
+    @Test(timeout = 60000)
+    public void testInterleavedReaders() throws Exception {
+        String name = "distrlog-interleaved";
+        BKDistributedLogManager dlmwrite0 = createNewDLM(conf, name + "-0");
+        BKDistributedLogManager dlmreader0 = createNewDLM(conf, name + "-0");
+        BKDistributedLogManager dlmwrite1 = createNewDLM(conf, name + "-1");
+        BKDistributedLogManager dlmreader1 = createNewDLM(conf, name + "-1");
+
+        LogReader reader0 = null;
+        LogReader reader1 = null;
+        long txid = 1;
+        int numTrans = 0;
+
+        BKAsyncLogWriter writer0 = dlmwrite0.startAsyncLogSegmentNonPartitioned();
+        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++)));
+            }
+            FutureUtils.result(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            FutureUtils.result(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            if (null == reader0) {
+                reader0 = dlmreader0.getInputStream(1);
+            }
+            if (null == reader1) {
+                reader1 = dlmreader1.getInputStream(1);
+            }
+            numTrans += drainStreams(reader0, 10, reader1, 10);
+            assertEquals((txid - 1), numTrans);
+        }
+        reader0.close();
+        reader1.close();
+        dlmreader0.close();
+        dlmwrite0.close();
+        dlmreader1.close();
+        dlmwrite1.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testInterleavedReadersWithRollingEdge() throws Exception {
+        String name = "distrlog-interleaved-rolling-edge";
+        BKDistributedLogManager dlmwrite0 = createNewDLM(conf, name + "-0");
+        BKDistributedLogManager dlmreader0 = createNewDLM(conf, name + "-0");
+        BKDistributedLogManager dlmwrite1 = createNewDLM(conf, name + "-1");
+        BKDistributedLogManager dlmreader1 = createNewDLM(conf, name + "-1");
+
+        LogReader reader0 = null;
+        LogReader reader1 = null;
+        long txid = 1;
+        int numTrans = 0;
+
+        BKAsyncLogWriter writer0 = dlmwrite0.startAsyncLogSegmentNonPartitioned();
+        BKAsyncLogWriter writer1 = dlmwrite1.startAsyncLogSegmentNonPartitioned();
+        for (long j = 1; j <= 4; j++) {
+            if (j > 1) {
+                writer0.setForceRolling(true);
+                writer1.setForceRolling(true);
+            }
+            for (int k = 1; k <= 2; k++) {
+                FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                FutureUtils.result(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)));
+            LOG.info("Completed {} write", j);
+            if (null == reader0) {
+                reader0 = dlmreader0.getInputStream(1);
+            }
+            if (null == reader1) {
+                reader1 = dlmreader1.getInputStream(1);
+            }
+            numTrans += drainStreams(reader0, 2, reader1, 2);
+            assertEquals((txid - 1), numTrans);
+        }
+        reader0.close();
+        reader1.close();
+        dlmreader0.close();
+        dlmwrite0.close();
+        dlmreader1.close();
+        dlmwrite1.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testInterleavedReadersWithRolling() throws Exception {
+        String name = "distrlog-interleaved-rolling";
+        BKDistributedLogManager dlmwrite0 = createNewDLM(conf, name + "-0");
+        BKDistributedLogManager dlmreader0 = createNewDLM(conf, name + "-0");
+        BKDistributedLogManager dlmwrite1 = createNewDLM(conf, name + "-1");
+        BKDistributedLogManager dlmreader1 = createNewDLM(conf, name + "-1");
+
+        LogReader reader0 = null;
+        LogReader reader1 = null;
+        long txid = 1;
+        int numTrans = 0;
+
+        BKAsyncLogWriter writer0 = dlmwrite0.startAsyncLogSegmentNonPartitioned();
+        BKAsyncLogWriter writer1 = dlmwrite1.startAsyncLogSegmentNonPartitioned();
+        for (long j = 1; j <= 2; j++) {
+            for (int k = 1; k <= 6; k++) {
+                if (k == 3) {
+                    writer0.setForceRolling(true);
+                    writer1.setForceRolling(true);
+                }
+                FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                FutureUtils.result(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)));
+            if (null == reader0) {
+                reader0 = dlmreader0.getInputStream(1);
+            }
+            if (null == reader1) {
+                reader1 = dlmreader1.getInputStream(1);
+            }
+            numTrans += drainStreams(reader0, 6, reader1, 6);
+            assertEquals((txid - 1), numTrans);
+        }
+        reader0.close();
+        reader1.close();
+        dlmreader0.close();
+        dlmwrite0.close();
+        dlmreader1.close();
+        dlmwrite1.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testInterleavedReadersWithCleanup() throws Exception {
+        String name = "distrlog-interleaved-cleanup";
+        BKDistributedLogManager dlmwrite0 = createNewDLM(conf, name + "-0");
+        BKDistributedLogManager dlmwrite1 = createNewDLM(conf, name + "-1");
+        long txid = 1;
+        Long retentionPeriodOverride = null;
+
+        BKAsyncLogWriter writer0 = dlmwrite0.startAsyncLogSegmentNonPartitioned();
+        BKAsyncLogWriter writer1 = dlmwrite1.startAsyncLogSegmentNonPartitioned();
+        for (long j = 1; j <= 4; j++) {
+            for (int k = 1; k <= 10; k++) {
+                if (k == 5) {
+                    writer0.setForceRolling(true);
+                    writer0.overRideMinTimeStampToKeep(retentionPeriodOverride);
+                    writer1.setForceRolling(true);
+                    writer1.overRideMinTimeStampToKeep(retentionPeriodOverride);
+                }
+                DLSN dlsn1 = FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                LOG.info("writer1 write record {}", dlsn1);
+                DLSN dlsn0 = FutureUtils.result(writer0.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                LOG.info("writer0 write record {}", dlsn0);
+                if (k == 5) {
+                    writer0.setForceRolling(false);
+                    writer1.setForceRolling(false);
+                    retentionPeriodOverride = System.currentTimeMillis();
+                }
+                Thread.sleep(5);
+            }
+            FutureUtils.result(writer1.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+            FutureUtils.result(writer0.writeControlRecord(DLMTestUtil.getLogRecordInstance(txid-1)));
+        }
+        writer0.close();
+        writer1.close();
+
+        DistributedLogManager dlmreader0 = createNewDLM(conf, name + "-0");
+        DistributedLogManager dlmreader1 = createNewDLM(conf, name + "-1");
+        LogReader reader0 = dlmreader0.getInputStream(1);
+        LogReader reader1 = dlmreader1.getInputStream(1);
+        int numTrans = drainStreams(reader0, 15, reader1, 15);
+        assertEquals(30, numTrans);
+        reader0.close();
+        reader1.close();
+        dlmreader0.close();
+        dlmwrite0.close();
+        dlmreader1.close();
+        dlmwrite1.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testInterleavedReadersWithRecovery() throws Exception {
+        String name = "distrlog-interleaved-recovery";
+        BKDistributedLogManager dlmwrite0 = createNewDLM(conf, name + "-0");
+        BKDistributedLogManager dlmreader0 = createNewDLM(conf, name + "-0");
+        BKDistributedLogManager dlmwrite1 = createNewDLM(conf, name + "-1");
+        BKDistributedLogManager dlmreader1 = createNewDLM(conf, name + "-1");
+
+        LogReader reader0 = null;
+        LogReader reader1 = null;
+        long txid = 1;
+        int numTrans = 0;
+
+        BKAsyncLogWriter writer0 = dlmwrite0.startAsyncLogSegmentNonPartitioned();
+        BKAsyncLogWriter writer1 = dlmwrite1.startAsyncLogSegmentNonPartitioned();
+        for (long j = 1; j <= 2; j++) {
+            for (int k = 1; k <= 6; k++) {
+                if (k == 3) {
+                    writer0.setForceRecovery(true);
+                    writer1.setForceRecovery(true);
+                }
+                DLSN dlsn1 = FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                LOG.info("writer1 write record {} - txid = {}", dlsn1, txid-1);
+                DLSN dlsn0 = FutureUtils.result(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)));
+            if (null == reader0) {
+                reader0 = dlmreader0.getInputStream(1);
+            }
+            if (null == reader1) {
+                reader1 = dlmreader1.getInputStream(1);
+            }
+            numTrans += drainStreams(reader0, 6, reader1, 6);
+            assertEquals((txid - 1), numTrans);
+        }
+        reader0.close();
+        reader1.close();
+        assertEquals(txid - 1,
+            dlmreader0.getLogRecordCount() + dlmreader1.getLogRecordCount());
+        dlmreader0.close();
+        dlmwrite0.close();
+        dlmreader1.close();
+        dlmwrite1.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testInterleavedReadersWithRollingEdgeUnPartitioned() throws Exception {
+        String name = "distrlog-interleaved-rolling-edge-unpartitioned";
+        BKDistributedLogManager dlmwrite0 = createNewDLM(conf, name + "-0");
+        BKDistributedLogManager dlmreader0 = createNewDLM(conf, name + "-0");
+        BKDistributedLogManager dlmwrite1 = createNewDLM(conf, name + "-1");
+        BKDistributedLogManager dlmreader1 = createNewDLM(conf, name + "-1");
+
+        LogReader reader0 = null;
+        LogReader reader1 = null;
+        long txid = 1;
+        int numTrans = 0;
+
+        BKAsyncLogWriter writer0 = dlmwrite0.startAsyncLogSegmentNonPartitioned();
+        BKAsyncLogWriter writer1 = dlmwrite1.startAsyncLogSegmentNonPartitioned();
+        for (long j = 1; j <= 4; j++) {
+            if (j > 1) {
+                writer0.setForceRolling(true);
+                writer1.setForceRolling(true);
+            }
+            for (int k = 1; k <= 2; k++) {
+                FutureUtils.result(writer1.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                FutureUtils.result(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)));
+            if (null == reader0) {
+                reader0 = dlmreader0.getInputStream(1);
+            }
+            if (null == reader1) {
+                reader1 = dlmreader1.getInputStream(1);
+            }
+            numTrans += drainStreams(reader0, 2, reader1, 2);
+            assertEquals((txid - 1), numTrans);
+        }
+        reader0.close();
+        reader1.close();
+        dlmreader0.close();
+        dlmreader1.close();
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java
new file mode 100644
index 0000000..152e4d8
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentCreation.java
@@ -0,0 +1,106 @@
+/**
+ * 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 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.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+public class TestLogSegmentCreation extends TestDistributedLogBase {
+
+    static Logger LOG = LoggerFactory.getLogger(TestLogSegmentCreation.class);
+
+    @Test(timeout = 60000)
+    public void testCreateLogSegmentAfterLoseLock() throws Exception {
+        URI uri = createDLMURI("/LogSegmentCreation");
+        String name = "distrlog-createlogsegment-afterloselock";
+        DistributedLogConfiguration conf = new DistributedLogConfiguration()
+                .setLockTimeout(99999)
+                .setOutputBufferSize(0)
+                .setImmediateFlushEnabled(true)
+                .setEnableLedgerAllocatorPool(true)
+                .setLedgerAllocatorPoolName("test");
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf).uri(uri).build();
+        DistributedLogManager dlm = namespace.openLog(name);
+        final int numSegments = 3;
+        for (int i = 0; i < numSegments; i++) {
+            BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+            out.write(DLMTestUtil.getLogRecordInstance(i));
+            out.closeAndComplete();
+        }
+
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        LOG.info("Segments : {}", segments);
+        assertEquals(3, segments.size());
+
+        final DistributedLogManager dlm1 = namespace.openLog(name);
+        final DistributedLogManager dlm2 = namespace.openLog(name);
+
+        BKAsyncLogWriter writer1 = (BKAsyncLogWriter) dlm1.startAsyncLogSegmentNonPartitioned();
+        LOG.info("Created writer 1.");
+        BKSyncLogWriter writer2 = (BKSyncLogWriter) dlm2.startLogSegmentNonPartitioned();
+        LOG.info("Created writer 2.");
+        writer2.write(DLMTestUtil.getLogRecordInstance(numSegments));
+        writer2.closeAndComplete();
+
+        try {
+            Await.result(writer1.write(DLMTestUtil.getLogRecordInstance(numSegments + 1)));
+            fail("Should fail on writing new log records.");
+        } catch (Throwable t) {
+            LOG.error("Failed to write entry : ", t);
+        }
+
+        segments = dlm.getLogSegments();
+
+        boolean hasInprogress = false;
+        boolean hasDuplicatedSegment = false;
+        long nextSeqNo = segments.get(0).getLogSegmentSequenceNumber();
+        for (int i = 1; i < segments.size(); i++) {
+            LogSegmentMetadata segment = segments.get(i);
+            assertTrue(segment.getLogSegmentSequenceNumber() >= nextSeqNo);
+            if (segment.getLogSegmentSequenceNumber() == nextSeqNo) {
+                hasDuplicatedSegment = true;
+            }
+            nextSeqNo = segment.getLogSegmentSequenceNumber();
+            if (segment.isInProgress()) {
+                hasInprogress = true;
+            }
+        }
+        assertEquals(4, segments.size());
+        assertFalse(hasInprogress);
+        assertFalse(hasDuplicatedSegment);
+
+        LOG.info("Segments : duplicated = {}, inprogress = {}, {}",
+                 new Object[] { hasDuplicatedSegment, hasInprogress, segments });
+
+        dlm1.close();
+        dlm2.close();
+        dlm.close();
+
+        namespace.close();
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java
new file mode 100644
index 0000000..31df059
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentMetadata.java
@@ -0,0 +1,159 @@
+/**
+ * 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 org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataBuilder;
+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.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Test {@link LogSegmentMetadata}
+ */
+public class TestLogSegmentMetadata extends ZooKeeperClusterTestCase {
+
+    static final Logger LOG = LoggerFactory.getLogger(TestLogSegmentMetadata.class);
+
+    static final int TEST_REGION_ID = 0xf - 1;
+
+    private ZooKeeperClient zkc;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .zkServers(zkServers)
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        zkc.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testReadMetadata() throws Exception {
+        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"));
+        assertEquals(metadata1, read1);
+        assertEquals(TEST_REGION_ID, read1.getRegionId());
+    }
+
+    @Test(timeout = 60000)
+    public void testReadMetadataCrossVersion() throws Exception {
+        LogSegmentMetadata metadata1 = new LogSegmentMetadataBuilder("/metadata2",
+            1, 1000, 1).setRegionId(TEST_REGION_ID).build();
+        metadata1.write(zkc);
+        // synchronous read
+        LogSegmentMetadata read1 = FutureUtils.result(LogSegmentMetadata.read(zkc, "/metadata2", true));
+        assertEquals(read1.getLogSegmentId(), metadata1.getLogSegmentId());
+        assertEquals(read1.getFirstTxId(), metadata1.getFirstTxId());
+        assertEquals(read1.getLastTxId(), metadata1.getLastTxId());
+        assertEquals(read1.getLogSegmentSequenceNumber(), metadata1.getLogSegmentSequenceNumber());
+        assertEquals(DistributedLogConstants.LOCAL_REGION_ID, read1.getRegionId());
+    }
+
+    @Test(timeout = 60000)
+    public void testReadMetadataCrossVersionFailure() throws Exception {
+        LogSegmentMetadata metadata1 = new LogSegmentMetadataBuilder("/metadata-failure",
+            1, 1000, 1).setRegionId(TEST_REGION_ID).build();
+        metadata1.write(zkc);
+        // synchronous read
+        try {
+            LogSegmentMetadata read1 = FutureUtils.result(LogSegmentMetadata.read(zkc, "/metadata-failure"));
+            fail("The previous statement should throw an exception");
+        } catch (UnsupportedMetadataVersionException e) {
+            // Expected
+        }
+    }
+
+
+    @Test(timeout = 60000)
+    public void testMutateTruncationStatus() {
+        LogSegmentMetadata metadata =
+                new LogSegmentMetadataBuilder(
+                        "/metadata", LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES, 1L, 0L)
+                        .setRegionId(0).setLogSegmentSequenceNo(1L).build();
+        metadata = metadata.completeLogSegment("/completed-metadata", 1000L, 1000, 1000L, 0L, 0L);
+
+        LogSegmentMetadata partiallyTruncatedSegment =
+                metadata.mutator()
+                        .setTruncationStatus(TruncationStatus.PARTIALLY_TRUNCATED)
+                        .setMinActiveDLSN(new DLSN(1L, 500L, 0L))
+                        .build();
+
+        LogSegmentMetadata fullyTruncatedSegment =
+                partiallyTruncatedSegment.mutator()
+                        .setTruncationStatus(TruncationStatus.TRUNCATED)
+                        .build();
+
+        assertEquals(new DLSN(1L, 500L, 0L), fullyTruncatedSegment.getMinActiveDLSN());
+    }
+
+    @Test(timeout = 60000)
+    public void testParseInvalidMetadata() throws Exception {
+        try {
+            LogSegmentMetadata.parseData("/metadata1", new byte[0], false);
+            fail("Should fail to parse invalid metadata");
+        } catch (IOException ioe) {
+            // expected
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testReadLogSegmentWithSequenceId() throws Exception {
+        LogSegmentMetadata metadata =
+                new LogSegmentMetadataBuilder(
+                        "/metadata", LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID, 1L, 0L)
+                        .setRegionId(0)
+                        .setLogSegmentSequenceNo(1L)
+                        .setStartSequenceId(999L)
+                        .build();
+        // write inprogress log segment with v5
+        String data = metadata.getFinalisedData();
+        LogSegmentMetadata parsedMetadata = LogSegmentMetadata.parseData("/metadatav5", data.getBytes(UTF_8), false);
+        assertEquals(999L, parsedMetadata.getStartSequenceId());
+
+        LogSegmentMetadata metadatav4 =
+                new LogSegmentMetadataBuilder(
+                        "/metadata", LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES, 1L, 0L)
+                        .setRegionId(0)
+                        .setLogSegmentSequenceNo(1L)
+                        .setStartSequenceId(999L)
+                        .build();
+        String datav4 = metadatav4.getFinalisedData();
+        LogSegmentMetadata parsedMetadatav4 =
+                LogSegmentMetadata.parseData("/metadatav4", datav4.getBytes(UTF_8), false);
+        assertTrue(parsedMetadatav4.getStartSequenceId() < 0);
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java
new file mode 100644
index 0000000..8c01a5c
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestLogSegmentsZK.java
@@ -0,0 +1,245 @@
+/**
+ * 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 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.util.DLUtils;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.zookeeper.data.Stat;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.*;
+
+public class TestLogSegmentsZK extends TestDistributedLogBase {
+
+    static Logger LOG = LoggerFactory.getLogger(TestLogSegmentsZK.class);
+
+    private static MaxLogSegmentSequenceNo getMaxLogSegmentSequenceNo(ZooKeeperClient zkc, URI uri, String streamName,
+                                                                      DistributedLogConfiguration conf) throws Exception {
+        Stat stat = new Stat();
+        String logSegmentsPath = LogMetadata.getLogSegmentsPath(
+                uri, streamName, conf.getUnpartitionedStreamName());
+        byte[] data = zkc.get().getData(logSegmentsPath, false, stat);
+        Versioned<byte[]> maxLSSNData = new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
+        return new MaxLogSegmentSequenceNo(maxLSSNData);
+    }
+
+    private static void updateMaxLogSegmentSequenceNo(ZooKeeperClient zkc, URI uri, String streamName,
+                                                      DistributedLogConfiguration conf, byte[] data) throws Exception {
+        String logSegmentsPath = LogMetadata.getLogSegmentsPath(
+                uri, streamName, conf.getUnpartitionedStreamName());
+        zkc.get().setData(logSegmentsPath, data, -1);
+    }
+
+    @Rule
+    public TestName testName = new TestName();
+
+    private URI createURI() throws Exception {
+        return createDLMURI("/" + testName.getMethodName());
+    }
+
+    /**
+     * Create Log Segment for an pre-create stream. No max ledger sequence number recorded.
+     */
+    @Test(timeout = 60000)
+    public void testCreateLogSegmentOnPrecreatedStream() throws Exception {
+        URI uri = createURI();
+        String streamName = testName.getMethodName();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration()
+                .setLockTimeout(99999)
+                .setOutputBufferSize(0)
+                .setImmediateFlushEnabled(true)
+                .setEnableLedgerAllocatorPool(true)
+                .setLedgerAllocatorPoolName("test");
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
+
+        namespace.createLog(streamName);
+        MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
+        assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO, max1.getSequenceNumber());
+        DistributedLogManager dlm = namespace.openLog(streamName);
+        final int numSegments = 3;
+        for (int i = 0; i < numSegments; i++) {
+            BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+            out.write(DLMTestUtil.getLogRecordInstance(i));
+            out.closeAndComplete();
+        }
+        MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
+        assertEquals(3, max2.getSequenceNumber());
+        dlm.close();
+        namespace.close();
+    }
+
+    /**
+     * Create Log Segment when no max sequence number recorded in /ledgers. e.g. old version.
+     */
+    @Test(timeout = 60000)
+    public void testCreateLogSegmentMissingMaxSequenceNumber() throws Exception {
+        URI uri = createURI();
+        String streamName = testName.getMethodName();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration()
+                .setLockTimeout(99999)
+                .setOutputBufferSize(0)
+                .setImmediateFlushEnabled(true)
+                .setEnableLedgerAllocatorPool(true)
+                .setLedgerAllocatorPoolName("test");
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
+
+        namespace.createLog(streamName);
+        MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
+        assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO, max1.getSequenceNumber());
+        DistributedLogManager dlm = namespace.openLog(streamName);
+        final int numSegments = 3;
+        for (int i = 0; i < numSegments; i++) {
+            BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+            out.write(DLMTestUtil.getLogRecordInstance(i));
+            out.closeAndComplete();
+        }
+        MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
+        assertEquals(3, max2.getSequenceNumber());
+
+        // nuke the max ledger sequence number
+        updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf, new byte[0]);
+        DistributedLogManager dlm1 = namespace.openLog(streamName);
+        try {
+            dlm1.startLogSegmentNonPartitioned();
+            fail("Should fail with unexpected exceptions");
+        } catch (UnexpectedException ue) {
+            // expected
+        } finally {
+            dlm1.close();
+        }
+
+        // invalid max ledger sequence number
+        updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf, "invalid-max".getBytes(UTF_8));
+        DistributedLogManager dlm2 = namespace.openLog(streamName);
+        try {
+            dlm2.startLogSegmentNonPartitioned();
+            fail("Should fail with unexpected exceptions");
+        } catch (UnexpectedException ue) {
+            // expected
+        } finally {
+            dlm2.close();
+        }
+
+        dlm.close();
+        namespace.close();
+    }
+
+    /**
+     * Create Log Segment while max sequence number isn't match with list of log segments.
+     */
+    @Test(timeout = 60000)
+    public void testCreateLogSegmentUnmatchMaxSequenceNumber() throws Exception {
+        URI uri = createURI();
+        String streamName = testName.getMethodName();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration()
+                .setLockTimeout(99999)
+                .setOutputBufferSize(0)
+                .setImmediateFlushEnabled(true)
+                .setEnableLedgerAllocatorPool(true)
+                .setLedgerAllocatorPoolName("test");
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
+
+        namespace.createLog(streamName);
+        MaxLogSegmentSequenceNo max1 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
+        assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO, max1.getSequenceNumber());
+        DistributedLogManager dlm = namespace.openLog(streamName);
+        final int numSegments = 3;
+        for (int i = 0; i < numSegments; i++) {
+            BKSyncLogWriter out = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+            out.write(DLMTestUtil.getLogRecordInstance(i));
+            out.closeAndComplete();
+        }
+        MaxLogSegmentSequenceNo max2 = getMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf);
+        assertEquals(3, max2.getSequenceNumber());
+
+        // update the max ledger sequence number
+        updateMaxLogSegmentSequenceNo(getZooKeeperClient(namespace), uri, streamName, conf,
+                DLUtils.serializeLogSegmentSequenceNumber(99));
+
+        DistributedLogManager dlm1 = namespace.openLog(streamName);
+        try {
+            BKSyncLogWriter out1 = (BKSyncLogWriter) dlm1.startLogSegmentNonPartitioned();
+            out1.write(DLMTestUtil.getLogRecordInstance(numSegments+1));
+            out1.closeAndComplete();
+            fail("Should fail creating new log segment when encountered unmatch max ledger sequence number");
+        } catch (DLIllegalStateException lse) {
+            // expected
+        } finally {
+            dlm1.close();
+        }
+
+        DistributedLogManager dlm2 = namespace.openLog(streamName);
+        List<LogSegmentMetadata> segments = dlm2.getLogSegments();
+        try {
+            assertEquals(3, segments.size());
+            assertEquals(1L, segments.get(0).getLogSegmentSequenceNumber());
+            assertEquals(2L, segments.get(1).getLogSegmentSequenceNumber());
+            assertEquals(3L, segments.get(2).getLogSegmentSequenceNumber());
+        } finally {
+            dlm2.close();
+        }
+
+        dlm.close();
+        namespace.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testCompleteLogSegmentConflicts() throws Exception {
+        URI uri = createURI();
+        String streamName = testName.getMethodName();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration()
+                .setLockTimeout(99999)
+                .setOutputBufferSize(0)
+                .setImmediateFlushEnabled(true)
+                .setEnableLedgerAllocatorPool(true)
+                .setLedgerAllocatorPoolName("test");
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder().conf(conf).uri(uri).build();
+
+        namespace.createLog(streamName);
+        DistributedLogManager dlm1 = namespace.openLog(streamName);
+        DistributedLogManager dlm2 = namespace.openLog(streamName);
+
+        // dlm1 is writing
+        BKSyncLogWriter out1 = (BKSyncLogWriter) dlm1.startLogSegmentNonPartitioned();
+        out1.write(DLMTestUtil.getLogRecordInstance(1));
+        // before out1 complete, out2 is in on recovery
+        // it completed the log segments which bump the version of /ledgers znode
+        BKAsyncLogWriter out2 = (BKAsyncLogWriter) dlm2.startAsyncLogSegmentNonPartitioned();
+
+        try {
+            out1.closeAndComplete();
+            fail("Should fail closeAndComplete since other people already completed it.");
+        } catch (IOException ioe) {
+        }
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java
new file mode 100644
index 0000000..5bfbf45
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReads.java
@@ -0,0 +1,348 @@
+/**
+ * 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 java.util.concurrent.ScheduledFuture;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.distributedlog.annotations.DistributedLogAnnotations;
+import org.apache.distributedlog.exceptions.IdleReaderException;
+import org.apache.distributedlog.util.FutureUtils;
+import org.junit.Assert;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.apache.distributedlog.NonBlockingReadsTestUtil.*;
+import static org.junit.Assert.*;
+
+/**
+ * {@link https://issues.apache.org/jira/browse/DL-12}
+ */
+@DistributedLogAnnotations.FlakyTest
+@Ignore
+public class TestNonBlockingReads extends TestDistributedLogBase {
+    static final Logger LOG = LoggerFactory.getLogger(TestNonBlockingReads.class);
+
+    static {
+        conf.setOutputBufferSize(0);
+        conf.setImmediateFlushEnabled(true);
+    }
+
+    @Test(timeout = 100000)
+    public void testNonBlockingRead() throws Exception {
+        String name = "distrlog-non-blocking-reader";
+        final DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setReadAheadBatchSize(1);
+        confLocal.setReadAheadMaxRecords(1);
+        confLocal.setReaderIdleWarnThresholdMillis(100);
+        confLocal.setReadLACLongPollTimeout(49);
+        final DistributedLogManager dlm = createNewDLM(confLocal, name);
+        ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
+        ScheduledFuture writerClosedFuture = null;
+        try {
+            final Thread currentThread = Thread.currentThread();
+            writerClosedFuture = executor.schedule(
+                    new Runnable() {
+                        @Override
+                        public void run() {
+                            try {
+                                writeRecordsForNonBlockingReads(confLocal, dlm, false);
+                            } catch (Exception exc) {
+                                currentThread.interrupt();
+                            }
+
+                        }
+                    }, 100, TimeUnit.MILLISECONDS);
+
+            readNonBlocking(dlm, false);
+            assertFalse(currentThread.isInterrupted());
+        } finally {
+            if (writerClosedFuture != null){
+                // ensure writer.closeAndComplete is done before we close dlm
+                writerClosedFuture.get();
+            }
+            executor.shutdown();
+            dlm.close();
+        }
+    }
+
+    @Test(timeout = 100000)
+    public void testNonBlockingReadRecovery() throws Exception {
+        String name = "distrlog-non-blocking-reader-recovery";
+        final DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setReadAheadBatchSize(10);
+        confLocal.setReadAheadMaxRecords(10);
+        final DistributedLogManager dlm = createNewDLM(confLocal, name);
+        ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
+        ScheduledFuture writerClosedFuture = null;
+        try {
+            final Thread currentThread = Thread.currentThread();
+            writerClosedFuture = executor.schedule(
+                    new Runnable() {
+                        @Override
+                        public void run() {
+                            try {
+                                writeRecordsForNonBlockingReads(confLocal, dlm, true);
+                            } catch (Exception exc) {
+                                currentThread.interrupt();
+                            }
+
+                        }
+                    }, 100, TimeUnit.MILLISECONDS);
+
+
+            readNonBlocking(dlm, false);
+            assertFalse(currentThread.isInterrupted());
+        } finally {
+            if (writerClosedFuture != null){
+                // ensure writer.closeAndComplete is done before we close dlm
+                writerClosedFuture.get();
+            }
+            executor.shutdown();
+            dlm.close();
+        }
+    }
+
+    @Test(timeout = 100000)
+    public void testNonBlockingReadIdleError() throws Exception {
+        String name = "distrlog-non-blocking-reader-error";
+        final DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setReadAheadBatchSize(1);
+        confLocal.setReadAheadMaxRecords(1);
+        confLocal.setReadLACLongPollTimeout(24);
+        confLocal.setReaderIdleWarnThresholdMillis(50);
+        confLocal.setReaderIdleErrorThresholdMillis(100);
+        final DistributedLogManager dlm = createNewDLM(confLocal, name);
+        ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
+        ScheduledFuture writerClosedFuture = null;
+        try {
+            final Thread currentThread = Thread.currentThread();
+            writerClosedFuture = executor.schedule(
+                    new Runnable() {
+                        @Override
+                        public void run() {
+                            try {
+                                writeRecordsForNonBlockingReads(confLocal, dlm, false);
+                            } catch (Exception exc) {
+                                currentThread.interrupt();
+                            }
+
+                        }
+                    }, 100, TimeUnit.MILLISECONDS);
+
+            boolean exceptionEncountered = false;
+            try {
+                readNonBlocking(dlm, false, DEFAULT_SEGMENT_SIZE, true);
+            } catch (IdleReaderException exc) {
+                exceptionEncountered = true;
+            }
+            assertTrue(exceptionEncountered);
+            assertFalse(currentThread.isInterrupted());
+        } finally {
+            if (writerClosedFuture != null){
+                // ensure writer.closeAndComplete is done before we close dlm
+                writerClosedFuture.get();
+            }
+            executor.shutdown();
+            dlm.close();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testNonBlockingReadAheadStall() throws Exception {
+        String name = "distrlog-non-blocking-reader-stall";
+        final DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setReadAheadBatchSize(1);
+        confLocal.setReadAheadMaxRecords(3);
+        confLocal.setReadLACLongPollTimeout(249);
+        confLocal.setReaderIdleWarnThresholdMillis(500);
+        confLocal.setReaderIdleErrorThresholdMillis(30000);
+        final DistributedLogManager dlm = createNewDLM(confLocal, name);
+        ScheduledThreadPoolExecutor executor = new ScheduledThreadPoolExecutor(1);
+        ScheduledFuture writerClosedFuture = null;
+        try {
+            final Thread currentThread = Thread.currentThread();
+            writerClosedFuture = executor.schedule(
+                    new Runnable() {
+                        @Override
+                        public void run() {
+                            try {
+                                writeRecordsForNonBlockingReads(confLocal, dlm, false, 3);
+                            } catch (Exception exc) {
+                                currentThread.interrupt();
+                            }
+
+                        }
+                    }, 10, TimeUnit.MILLISECONDS);
+
+            boolean exceptionEncountered = false;
+            try {
+                readNonBlocking(dlm, false, 3, false);
+            } catch (IdleReaderException exc) {
+                LOG.info("Exception encountered", exc);
+                exceptionEncountered = true;
+            }
+            assertFalse(exceptionEncountered);
+            assertFalse(currentThread.isInterrupted());
+        } finally {
+            if (writerClosedFuture != null){
+                // ensure writer.closeAndComplete is done before we close dlm
+                writerClosedFuture.get();
+            }
+            executor.shutdown();
+            dlm.close();
+        }
+    }
+
+    private long createStreamWithInconsistentMetadata(String name) throws Exception {
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        ZooKeeperClient zkClient = TestZooKeeperClientBuilder.newBuilder()
+                .uri(createDLMURI("/"))
+                .build();
+        long txid = 1;
+
+        long numRecordsWritten = 0;
+        int segmentSize = 10;
+        for (long i = 0; i < 3; i++) {
+            BKAsyncLogWriter out = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
+            for (long j = 1; j <= segmentSize; j++) {
+                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+                FutureUtils.result(out.write(op));
+                numRecordsWritten++;
+            }
+            out.closeAndComplete();
+        }
+
+        BKLogWriteHandler blplm = ((BKDistributedLogManager) (dlm)).createWriteHandler(true);
+        String completedZNode = blplm.completedLedgerZNode(txid - segmentSize, txid - 1, 3);
+        LogSegmentMetadata metadata = FutureUtils.result(LogSegmentMetadata.read(zkClient, completedZNode));
+        zkClient.get().delete(completedZNode, -1);
+        LogSegmentMetadata metadataToChange =
+                metadata.mutator()
+                        .setLastEntryId(metadata.getLastEntryId() + 100)
+                        .setLastTxId(metadata.getLastTxId() + 100)
+                        .build();
+        metadataToChange.write(zkClient);
+
+        txid += 100;
+
+
+        for (long i = 0; i < 3; i++) {
+            BKAsyncLogWriter out = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
+            for (long j = 1; j <= segmentSize; j++) {
+                LogRecord op = DLMTestUtil.getLogRecordInstance(txid++);
+                FutureUtils.result(out.write(op));
+                numRecordsWritten++;
+            }
+            out.closeAndComplete();
+        }
+        dlm.close();
+
+        return numRecordsWritten;
+    }
+
+    @Test(timeout = 60000)
+    public void testHandleInconsistentMetadata() throws Exception {
+        String name = "distrlog-inconsistent-metadata-blocking-read";
+        long numRecordsWritten = createStreamWithInconsistentMetadata(name);
+
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        try {
+            LogReader reader = dlm.getInputStream(45);
+            long numRecordsRead = 0;
+            LogRecord record = reader.readNext(false);
+            long lastTxId = -1;
+            while (numRecordsRead < numRecordsWritten / 2) {
+                if (null != record) {
+                    DLMTestUtil.verifyLogRecord(record);
+                    Assert.assertTrue(lastTxId < record.getTransactionId());
+                    lastTxId = record.getTransactionId();
+                    numRecordsRead++;
+                } else {
+                    Thread.sleep(1);
+                }
+                record = reader.readNext(false);
+            }
+            reader.close();
+            assertEquals(numRecordsWritten / 2, numRecordsRead);
+        } finally {
+            dlm.close();
+        }
+    }
+
+    @Test(timeout = 15000)
+    public void testHandleInconsistentMetadataNonBlocking() throws Exception {
+        String name = "distrlog-inconsistent-metadata-nonblocking-read";
+        long numRecordsWritten = createStreamWithInconsistentMetadata(name);
+
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        try {
+            LogReader reader = dlm.getInputStream(45);
+            long numRecordsRead = 0;
+            long lastTxId = -1;
+            while (numRecordsRead < (numRecordsWritten / 2)) {
+                LogRecord record = reader.readNext(false);
+                if (record != null) {
+                    DLMTestUtil.verifyLogRecord(record);
+                    Assert.assertTrue(lastTxId < record.getTransactionId());
+                    lastTxId = record.getTransactionId();
+                    numRecordsRead++;
+                } else {
+                    Thread.sleep(1);
+                }
+            }
+            reader.close();
+        } finally {
+            dlm.close();
+        }
+    }
+
+    @Test(timeout = 15000)
+    public void testHandleInconsistentMetadataDLSNNonBlocking() throws Exception {
+        String name = "distrlog-inconsistent-metadata-nonblocking-read-dlsn";
+        long numRecordsWritten = createStreamWithInconsistentMetadata(name);
+
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        try {
+            LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
+            long numRecordsRead = 0;
+            long lastTxId = -1;
+            while (numRecordsRead < numRecordsWritten) {
+                LogRecord record = reader.readNext(false);
+                if (record != null) {
+                    DLMTestUtil.verifyLogRecord(record);
+                    Assert.assertTrue(lastTxId < record.getTransactionId());
+                    lastTxId = record.getTransactionId();
+                    numRecordsRead++;
+                } else {
+                    Thread.sleep(1);
+                }
+            }
+            reader.close();
+        } finally {
+            dlm.close();
+        }
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java
new file mode 100644
index 0000000..8f445c4
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestNonBlockingReadsMultiReader.java
@@ -0,0 +1,168 @@
+/**
+ * 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.util.concurrent.RateLimiter;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.Utils;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.*;
+
+public class TestNonBlockingReadsMultiReader extends TestDistributedLogBase {
+
+        static class ReaderThread extends Thread {
+
+        final LogReader reader;
+        final boolean nonBlockReading;
+        volatile boolean running = true;
+        final AtomicInteger readCount = new AtomicInteger(0);
+
+        ReaderThread(String name, LogReader reader, boolean nonBlockReading) {
+            super(name);
+            this.reader = reader;
+            this.nonBlockReading = nonBlockReading;
+        }
+
+        @Override
+        public void run() {
+            while (running) {
+                try {
+                    LogRecord r = reader.readNext(nonBlockReading);
+                    if (r != null) {
+                        readCount.incrementAndGet();
+                        if (readCount.get() % 1000 == 0) {
+                            LOG.info("{} reading {}", getName(), r.getTransactionId());
+                        }
+                    }
+                } catch (DLInterruptedException die) {
+                    Thread.currentThread().interrupt();
+                } catch (IOException e) {
+                    break;
+                }
+            }
+        }
+
+        void stopReading() {
+            LOG.info("Stopping reader.");
+            running = false;
+            interrupt();
+            try {
+                join();
+            } catch (InterruptedException e) {
+                LOG.error("Interrupted on waiting reader thread {} exiting : ", getName(), e);
+            }
+        }
+
+        int getReadCount() {
+            return readCount.get();
+        }
+
+    }
+
+    @Test(timeout = 60000)
+    public void testMultiReaders() throws Exception {
+        String name = "distrlog-multireaders";
+        final RateLimiter limiter = RateLimiter.create(1000);
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.setOutputBufferSize(0);
+        confLocal.setImmediateFlushEnabled(true);
+
+        DistributedLogManager dlmwrite = createNewDLM(confLocal, name);
+
+        final AsyncLogWriter writer = dlmwrite.startAsyncLogSegmentNonPartitioned();
+        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(0)));
+        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(1)));
+        final AtomicInteger writeCount = new AtomicInteger(2);
+
+        DistributedLogManager dlmread = createNewDLM(conf, name);
+
+        BKSyncLogReader reader0 = (BKSyncLogReader) dlmread.getInputStream(0);
+
+        try {
+            ReaderThread[] readerThreads = new ReaderThread[1];
+            readerThreads[0] = new ReaderThread("reader0-non-blocking", reader0, false);
+            // readerThreads[1] = new ReaderThread("reader1-non-blocking", reader0, false);
+
+            final AtomicBoolean running = new AtomicBoolean(true);
+            Thread writerThread = new Thread("WriteThread") {
+                @Override
+                public void run() {
+                    try {
+                        long txid = 2;
+                        DLSN dlsn = DLSN.InvalidDLSN;
+                        while (running.get()) {
+                            limiter.acquire();
+                            long curTxId = txid++;
+                            dlsn = FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(curTxId)));
+                            writeCount.incrementAndGet();
+                            if (curTxId % 1000 == 0) {
+                                LOG.info("writer write {}", curTxId);
+                            }
+                        }
+                        LOG.info("Completed writing record at {}", dlsn);
+                        Utils.close(writer);
+                    } catch (DLInterruptedException die) {
+                        Thread.currentThread().interrupt();
+                    } catch (IOException e) {
+
+                    }
+                }
+            };
+
+            for (ReaderThread rt : readerThreads) {
+                rt.start();
+            }
+
+            writerThread.start();
+
+            TimeUnit.SECONDS.sleep(5);
+
+            LOG.info("Stopping writer");
+
+            running.set(false);
+            writerThread.join();
+
+            LOG.info("Writer stopped after writing {} records, waiting for reader to complete",
+                    writeCount.get());
+            while (writeCount.get() > (readerThreads[0].getReadCount())) {
+                LOG.info("Write Count = {}, Read Count = {}",
+                        new Object[] { writeCount.get(), readerThreads[0].getReadCount() });
+                TimeUnit.MILLISECONDS.sleep(100);
+            }
+            assertEquals(writeCount.get(),
+                    (readerThreads[0].getReadCount()));
+
+            for (ReaderThread readerThread : readerThreads) {
+                readerThread.stopReading();
+            }
+        } finally {
+            dlmwrite.close();
+            reader0.close();
+            dlmread.close();
+        }
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java
new file mode 100644
index 0000000..ac9984b
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadAheadEntryReader.java
@@ -0,0 +1,463 @@
+/**
+ * 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.base.Optional;
+import com.google.common.base.Ticker;
+import com.google.common.collect.Lists;
+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.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;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test Case {@link ReadAheadEntryReader}
+ */
+public class TestReadAheadEntryReader extends TestDistributedLogBase {
+
+    private static final int MAX_CACHED_ENTRIES = 5;
+    private static final int NUM_PREFETCH_ENTRIES = 10;
+
+    @Rule
+    public TestName runtime = new TestName();
+    private DistributedLogConfiguration baseConf;
+    private OrderedScheduler scheduler;
+    private BookKeeperClient bkc;
+    private ZooKeeperClient zkc;
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        baseConf = new DistributedLogConfiguration();
+        baseConf.addConfiguration(conf);
+        baseConf.setOutputBufferSize(0);
+        baseConf.setPeriodicFlushFrequencyMilliSeconds(0);
+        baseConf.setImmediateFlushEnabled(false);
+        baseConf.setReadAheadMaxRecords(MAX_CACHED_ENTRIES);
+        baseConf.setNumPrefetchEntriesPerLogSegment(NUM_PREFETCH_ENTRIES);
+        baseConf.setMaxPrefetchEntriesPerLogSegment(NUM_PREFETCH_ENTRIES);
+        zkc = ZooKeeperClientBuilder.newBuilder()
+                .name("test-zk")
+                .zkServers(bkutil.getZkServers())
+                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                .zkAclId(conf.getZkAclId())
+                .build();
+        bkc = BookKeeperClientBuilder.newBuilder()
+                .name("test-bk")
+                .dlConfig(conf)
+                .ledgersPath("/ledgers")
+                .zkServers(bkutil.getZkServers())
+                .build();
+        scheduler = OrderedScheduler.newBuilder()
+                .name("test-read-ahead-entry-reader")
+                .corePoolSize(1)
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (null != bkc) {
+            bkc.close();
+        }
+        if (null != scheduler) {
+            scheduler.shutdown();
+        }
+        if (null != zkc) {
+            zkc.close();
+        }
+        super.teardown();
+    }
+
+    private ReadAheadEntryReader createEntryReader(String streamName,
+                                                   DLSN fromDLSN,
+                                                   BKDistributedLogManager dlm,
+                                                   DistributedLogConfiguration conf)
+            throws Exception {
+        BKLogReadHandler readHandler = dlm.createReadHandler(
+                Optional.<String>absent(),
+                true);
+        LogSegmentEntryStore entryStore = new BKLogSegmentEntryStore(
+                conf,
+                ConfUtils.getConstDynConf(conf),
+                zkc,
+                bkc,
+                scheduler,
+                null,
+                NullStatsLogger.INSTANCE,
+                AsyncFailureInjector.NULL);
+        return new ReadAheadEntryReader(
+                streamName,
+                fromDLSN,
+                conf,
+                readHandler,
+                entryStore,
+                scheduler,
+                Ticker.systemTicker(),
+                new AlertStatsLogger(NullStatsLogger.INSTANCE, "test-alert"));
+    }
+
+    private void ensureOrderSchedulerEmpty(String streamName) throws Exception {
+        final Promise<Void> promise = new Promise<Void>();
+        scheduler.submit(streamName, new Runnable() {
+            @Override
+            public void run() {
+                FutureUtils.setValue(promise, null);
+            }
+        });
+        FutureUtils.result(promise);
+    }
+
+    void generateCompletedLogSegments(DistributedLogManager dlm,
+                                      long numCompletedSegments,
+                                      long segmentSize) throws Exception {
+        generateCompletedLogSegments(dlm, numCompletedSegments, segmentSize, 1L);
+    }
+
+    void generateCompletedLogSegments(DistributedLogManager dlm,
+                                      long numCompletedSegments,
+                                      long segmentSize,
+                                      long startTxId) throws Exception {
+
+        long txid = startTxId;
+        for (long i = 0; i < numCompletedSegments; i++) {
+            AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+            for (long j = 1; j <= segmentSize; j++) {
+                FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                LogRecord ctrlRecord = DLMTestUtil.getLogRecordInstance(txid);
+                ctrlRecord.setControl();
+                FutureUtils.result(writer.write(ctrlRecord));
+            }
+            Utils.close(writer);
+        }
+    }
+
+    AsyncLogWriter createInprogressLogSegment(DistributedLogManager dlm,
+                                              DistributedLogConfiguration conf,
+                                              long segmentSize) throws Exception {
+        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+        for (long i = 1L; i <= segmentSize; i++) {
+            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
+            LogRecord ctrlRecord = DLMTestUtil.getLogRecordInstance(i);
+            ctrlRecord.setControl();
+            FutureUtils.result(writer.write(ctrlRecord));
+        }
+        return writer;
+    }
+
+    void expectAlreadyTruncatedTransactionException(ReadAheadEntryReader reader,
+                                                    String errMsg)
+            throws Exception {
+        try {
+            reader.checkLastException();
+            fail(errMsg);
+        } catch (AlreadyTruncatedTransactionException atte) {
+            // expected
+        }
+    }
+
+    void expectIllegalStateException(ReadAheadEntryReader reader,
+                                     String errMsg)
+            throws Exception {
+        try {
+            reader.checkLastException();
+            fail(errMsg);
+        } catch (DLIllegalStateException le) {
+            // expected
+        }
+    }
+
+    void expectNoException(ReadAheadEntryReader reader) throws Exception {
+        reader.checkLastException();
+    }
+
+    //
+    // Test Positioning
+    //
+
+    @Test(timeout = 60000)
+    public void testStartWithEmptySegmentList() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager dlm = createNewDLM(baseConf, streamName);
+        ReadAheadEntryReader readAheadEntryReader =
+                createEntryReader(streamName, DLSN.InitialDLSN, dlm, baseConf);
+
+        readAheadEntryReader.start(Lists.<LogSegmentMetadata>newArrayList());
+
+        ensureOrderSchedulerEmpty(streamName);
+        assertFalse("ReadAhead should not be initialized with empty segment list",
+                readAheadEntryReader.isInitialized());
+        assertTrue("ReadAhead should be empty when it isn't initialized",
+                readAheadEntryReader.isCacheEmpty());
+        assertFalse("ReadAhead should not be marked as caught up when it isn't initialized",
+                readAheadEntryReader.isReadAheadCaughtUp());
+
+        // generate list of log segments
+        generateCompletedLogSegments(dlm, 1, MAX_CACHED_ENTRIES / 2 + 1);
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(segments.size() + " log segments found, expected to be only one",
+                1, segments.size());
+
+        // notify the readahead reader with new segment lsit
+        readAheadEntryReader.onSegmentsUpdated(segments);
+
+        // check the reader state after initialization
+        ensureOrderSchedulerEmpty(streamName);
+        assertTrue("ReadAhead should be initialized with non-empty segment list",
+                readAheadEntryReader.isInitialized());
+        assertNotNull("current segment reader should be initialized",
+                readAheadEntryReader.getCurrentSegmentReader());
+        assertEquals("current segment sequence number should be " + segments.get(0).getLogSegmentSequenceNumber(),
+                segments.get(0).getLogSegmentSequenceNumber(), readAheadEntryReader.getCurrentSegmentSequenceNumber());
+        assertNull("there should be no next segment reader",
+                readAheadEntryReader.getNextSegmentReader());
+        assertTrue("there should be no remaining segment readers",
+                readAheadEntryReader.getSegmentReaders().isEmpty());
+
+        Utils.close(readAheadEntryReader);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testInitializeMultipleClosedLogSegments0() throws Exception {
+        // 5 completed log segments, start from the begin
+        testInitializeMultipleClosedLogSegments(5, DLSN.InitialDLSN, 0);
+    }
+
+    @Test(timeout = 60000)
+    public void testInitializeMultipleClosedLogSegments1() throws Exception {
+        // 5 completed log segments, start from the 4th segment and it should skip first 3 log segments
+        testInitializeMultipleClosedLogSegments(5, new DLSN(4L, 0L, 0L), 3);
+    }
+
+    private void testInitializeMultipleClosedLogSegments(
+            int numLogSegments,
+            DLSN fromDLSN,
+            int expectedCurrentSegmentIdx
+    ) throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager dlm = createNewDLM(baseConf, streamName);
+
+        // generate list of log segments
+        generateCompletedLogSegments(dlm, 1, MAX_CACHED_ENTRIES / 2 + 1, 1L);
+        generateCompletedLogSegments(dlm, numLogSegments - 1, 1, MAX_CACHED_ENTRIES + 2);
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(segments.size() + " log segments found, expected to be " + numLogSegments,
+                numLogSegments, segments.size());
+
+        ReadAheadEntryReader readAheadEntryReader =
+                createEntryReader(streamName, fromDLSN, dlm, baseConf);
+        readAheadEntryReader.start(segments);
+
+        ensureOrderSchedulerEmpty(streamName);
+        assertTrue("ReadAhead should be initialized with non-empty segment list",
+                readAheadEntryReader.isInitialized());
+        assertNotNull("current segment reader should be initialized",
+                readAheadEntryReader.getCurrentSegmentReader());
+        assertTrue("current segment reader should be open and started",
+                readAheadEntryReader.getCurrentSegmentReader().isReaderOpen()
+                        && readAheadEntryReader.getCurrentSegmentReader().isReaderStarted());
+        assertEquals("current segment reader should read " + segments.get(expectedCurrentSegmentIdx),
+                segments.get(expectedCurrentSegmentIdx),
+                readAheadEntryReader.getCurrentSegmentReader().getSegment());
+        assertEquals("current segment sequence number should be "
+                + segments.get(expectedCurrentSegmentIdx).getLogSegmentSequenceNumber(),
+                segments.get(expectedCurrentSegmentIdx).getLogSegmentSequenceNumber(),
+                readAheadEntryReader.getCurrentSegmentSequenceNumber());
+        assertNull("next segment reader should not be initialized since it is a closed log segment",
+                readAheadEntryReader.getNextSegmentReader());
+        assertEquals("there should be " + (numLogSegments - (expectedCurrentSegmentIdx + 1))
+                + " remaining segment readers",
+                numLogSegments - (expectedCurrentSegmentIdx + 1),
+                readAheadEntryReader.getSegmentReaders().size());
+        int segmentIdx = expectedCurrentSegmentIdx + 1;
+        for (ReadAheadEntryReader.SegmentReader reader : readAheadEntryReader.getSegmentReaders()) {
+            LogSegmentMetadata expectedSegment = segments.get(segmentIdx);
+            assertEquals("Segment should " + expectedSegment,
+                    expectedSegment, reader.getSegment());
+            assertTrue("Segment reader for " + expectedSegment + " should be open",
+                    reader.isReaderOpen());
+            assertFalse("Segment reader for " + expectedSegment + " should not be started",
+                    reader.isReaderStarted());
+            ++segmentIdx;
+        }
+
+        Utils.close(readAheadEntryReader);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testPositioningAtInvalidLogSegment() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager dlm = createNewDLM(baseConf, streamName);
+
+        // generate list of log segments
+        generateCompletedLogSegments(dlm, 3, 3);
+        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+        FutureUtils.result(writer.truncate(new DLSN(2L, 1L, 0L)));
+
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+
+        // positioning on a truncated log segment (segment 1)
+        ReadAheadEntryReader readAheadEntryReader =
+                createEntryReader(streamName, DLSN.InitialDLSN, dlm, baseConf);
+        readAheadEntryReader.start(segments);
+        // ensure initialization to complete
+        ensureOrderSchedulerEmpty(streamName);
+        expectNoException(readAheadEntryReader);
+        Entry.Reader entryReader =
+                readAheadEntryReader.getNextReadAheadEntry(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        assertEquals(2L, entryReader.getLSSN());
+        assertEquals(1L, entryReader.getEntryId());
+        Utils.close(readAheadEntryReader);
+
+        // positioning on a partially truncated log segment (segment 2) before min active dlsn
+        readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 0L, 0L), dlm, baseConf);
+        readAheadEntryReader.start(segments);
+        // ensure initialization to complete
+        ensureOrderSchedulerEmpty(streamName);
+        expectNoException(readAheadEntryReader);
+        entryReader =
+                readAheadEntryReader.getNextReadAheadEntry(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        assertEquals(2L, entryReader.getLSSN());
+        assertEquals(1L, entryReader.getEntryId());
+        Utils.close(readAheadEntryReader);
+
+        // positioning on a partially truncated log segment (segment 2) after min active dlsn
+        readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 2L, 0L), dlm, baseConf);
+        readAheadEntryReader.start(segments);
+        // ensure initialization to complete
+        ensureOrderSchedulerEmpty(streamName);
+        expectNoException(readAheadEntryReader);
+        entryReader =
+                readAheadEntryReader.getNextReadAheadEntry(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        assertEquals(2L, entryReader.getLSSN());
+        assertEquals(2L, entryReader.getEntryId());
+        Utils.close(readAheadEntryReader);
+
+        Utils.close(writer);
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testPositioningIgnoreTruncationStatus() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(baseConf);
+        confLocal.setIgnoreTruncationStatus(true);
+
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager dlm = createNewDLM(confLocal, streamName);
+
+        // generate list of log segments
+        generateCompletedLogSegments(dlm, 3, 2);
+        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+        FutureUtils.result(writer.truncate(new DLSN(2L, 1L, 0L)));
+
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+
+        // positioning on a truncated log segment (segment 1)
+        ReadAheadEntryReader readAheadEntryReader =
+                createEntryReader(streamName, DLSN.InitialDLSN, dlm, confLocal);
+        readAheadEntryReader.start(segments);
+        // ensure initialization to complete
+        ensureOrderSchedulerEmpty(streamName);
+        expectNoException(readAheadEntryReader);
+        Utils.close(readAheadEntryReader);
+
+        // positioning on a partially truncated log segment (segment 2) before min active dlsn
+        readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 0L, 0L), dlm, confLocal);
+        readAheadEntryReader.start(segments);
+        // ensure initialization to complete
+        ensureOrderSchedulerEmpty(streamName);
+        expectNoException(readAheadEntryReader);
+        Utils.close(readAheadEntryReader);
+
+        // positioning on a partially truncated log segment (segment 2) after min active dlsn
+        readAheadEntryReader = createEntryReader(streamName, new DLSN(2L, 1L, 0L), dlm, confLocal);
+        readAheadEntryReader.start(segments);
+        // ensure initialization to complete
+        ensureOrderSchedulerEmpty(streamName);
+        expectNoException(readAheadEntryReader);
+        Utils.close(readAheadEntryReader);
+
+        Utils.close(writer);
+        dlm.close();
+    }
+
+    //
+    // Test Reinitialization
+    //
+
+    @Test(timeout = 60000)
+    public void testLogSegmentSequenceNumberGap() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager dlm = createNewDLM(baseConf, streamName);
+
+        // generate list of log segments
+        generateCompletedLogSegments(dlm, 3, 2);
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+
+        ReadAheadEntryReader readAheadEntryReader =
+                createEntryReader(streamName, DLSN.InitialDLSN, dlm, baseConf);
+        readAheadEntryReader.start(segments.subList(0, 1));
+        int expectedCurrentSegmentIdx = 0;
+        ensureOrderSchedulerEmpty(streamName);
+        assertTrue("ReadAhead should be initialized with non-empty segment list",
+                readAheadEntryReader.isInitialized());
+        assertNotNull("current segment reader should be initialized",
+                readAheadEntryReader.getCurrentSegmentReader());
+        assertTrue("current segment reader should be open and started",
+                readAheadEntryReader.getCurrentSegmentReader().isReaderOpen()
+                        && readAheadEntryReader.getCurrentSegmentReader().isReaderStarted());
+        assertEquals("current segment reader should read " + segments.get(expectedCurrentSegmentIdx),
+                segments.get(expectedCurrentSegmentIdx),
+                readAheadEntryReader.getCurrentSegmentReader().getSegment());
+        assertEquals("current segment sequence number should be "
+                + segments.get(expectedCurrentSegmentIdx).getLogSegmentSequenceNumber(),
+                segments.get(expectedCurrentSegmentIdx).getLogSegmentSequenceNumber(),
+                readAheadEntryReader.getCurrentSegmentSequenceNumber());
+        assertNull("next segment reader should not be initialized since it is a closed log segment",
+                readAheadEntryReader.getNextSegmentReader());
+
+        readAheadEntryReader.onSegmentsUpdated(segments.subList(2, 3));
+        ensureOrderSchedulerEmpty(streamName);
+        expectIllegalStateException(readAheadEntryReader,
+                "inconsistent log segment found");
+
+        Utils.close(readAheadEntryReader);
+        dlm.close();
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadUtils.java
new file mode 100644
index 0000000..029e872
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReadUtils.java
@@ -0,0 +1,344 @@
+/**
+ * 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 java.util.List;
+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.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;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test {@link ReadUtils}
+ */
+public class TestReadUtils extends TestDistributedLogBase {
+
+    static final Logger LOG = LoggerFactory.getLogger(TestReadUtils.class);
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    private Future<Optional<LogRecordWithDLSN>> getLogRecordNotLessThanTxId(
+            BKDistributedLogManager bkdlm, int logsegmentIdx, long transactionId) throws Exception {
+        List<LogSegmentMetadata> logSegments = bkdlm.getLogSegments();
+        return ReadUtils.getLogRecordNotLessThanTxId(
+                bkdlm.getStreamName(),
+                logSegments.get(logsegmentIdx),
+                transactionId,
+                Executors.newSingleThreadExecutor(),
+                bkdlm.getReaderEntryStore(),
+                10
+        );
+    }
+
+    private Future<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),
+                bkdlm.getReaderEntryStore(), dlsn
+        );
+    }
+
+    private Future<LogRecordWithDLSN> getLastUserRecord(BKDistributedLogManager bkdlm, int ledgerNo) throws Exception {
+        BKLogReadHandler readHandler = bkdlm.createReadHandler();
+        List<LogSegmentMetadata> ledgerList = FutureUtils.result(
+                readHandler.readLogSegmentsFromStore(
+                        LogSegmentMetadata.COMPARATOR,
+                        LogSegmentFilter.DEFAULT_FILTER,
+                        null)
+        ).getValue();
+        return ReadUtils.asyncReadLastRecord(
+                bkdlm.getStreamName(), ledgerList.get(ledgerNo), false, false, false, 2, 16, new AtomicInteger(0), Executors.newFixedThreadPool(1),
+                bkdlm.getReaderEntryStore()
+        );
+    }
+
+    @Test(timeout = 60000)
+    public void testForwardScanFirstRecord() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
+        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);
+        assertEquals("should be an exact match", dlsn, logrec.getDlsn());
+        bkdlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testForwardScanNotFirstRecord() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
+        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);
+        assertEquals("should be an exact match", dlsn, logrec.getDlsn());
+        bkdlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testForwardScanValidButNonExistentRecord() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
+        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);
+        assertEquals(new DLSN(1,1,0), logrec.getDlsn());
+        bkdlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testForwardScanForRecordAfterLedger() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
+        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);
+        assertEquals(null, logrec);
+        bkdlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testForwardScanForRecordBeforeLedger() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
+        long txid = 1;
+        txid += DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5 /* user recs */ , txid);
+        txid += DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0, 5 /* user recs */ , txid);
+        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);
+        assertEquals(new DLSN(2,0,0), logrec.getDlsn());
+        bkdlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testForwardScanControlRecord() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
+        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);
+        assertEquals(new DLSN(1,5,0), logrec.getDlsn());
+        bkdlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLastRecordUserRecord() throws Exception {
+        String streamName = runtime.getMethodName();
+        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);
+        assertEquals(new DLSN(1,9,0), logrec.getDlsn());
+        bkdlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLastRecordControlRecord() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = (BKDistributedLogManager) createNewDLM(conf, streamName);
+
+        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.close(out);
+
+        Future<LogRecordWithDLSN> futureLogrec = getLastUserRecord(bkdlm, 0);
+        LogRecordWithDLSN logrec = Await.result(futureLogrec);
+        assertEquals(new DLSN(1,2,0), logrec.getDlsn());
+        bkdlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLastRecordAllControlRecords() throws Exception {
+        String streamName = runtime.getMethodName();
+        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);
+        assertEquals(null, logrec);
+        bkdlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testGetEntriesToSearch() throws Exception {
+        assertTrue(ReadUtils.getEntriesToSearch(2L, 1L, 10).isEmpty());
+        assertEquals(Lists.newArrayList(1L),
+                ReadUtils.getEntriesToSearch(1L, 1L, 10));
+        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 10L),
+                ReadUtils.getEntriesToSearch(1L, 10L, 10));
+        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L),
+                ReadUtils.getEntriesToSearch(1L, 9L, 10));
+        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L),
+                ReadUtils.getEntriesToSearch(1L, 8L, 10));
+        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 11L),
+                ReadUtils.getEntriesToSearch(1L, 11L, 10));
+        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 12L),
+                ReadUtils.getEntriesToSearch(1L, 12L, 10));
+    }
+
+    @Test(timeout = 60000)
+    public void testGetEntriesToSearchByTxnId() throws Exception {
+        LogRecordWithDLSN firstRecord =
+                DLMTestUtil.getLogRecordWithDLSNInstance(new DLSN(1L, 0L, 0L), 999L);
+        LogRecordWithDLSN secondRecord =
+                DLMTestUtil.getLogRecordWithDLSNInstance(new DLSN(1L, 10L, 0L), 99L);
+        LogRecordWithDLSN thirdRecord =
+                DLMTestUtil.getLogRecordWithDLSNInstance(new DLSN(1L, 100L, 0L), 1099L);
+        // out-of-order sequence
+        assertTrue(ReadUtils.getEntriesToSearch(888L, firstRecord, secondRecord, 10).isEmpty());
+        // same transaction id
+        assertTrue(ReadUtils.getEntriesToSearch(888L, firstRecord, firstRecord, 10).isEmpty());
+        // small nways (nways = 2)
+        assertEquals(2, ReadUtils.getEntriesToSearch(888L, firstRecord, thirdRecord, 2).size());
+        // small nways with equal transaction id
+        assertEquals(3, ReadUtils.getEntriesToSearch(1099L, firstRecord, thirdRecord, 2).size());
+        LogRecordWithDLSN record1 =
+                DLMTestUtil.getLogRecordWithDLSNInstance(new DLSN(1L, 0L, 0L), 88L);
+        LogRecordWithDLSN record2 =
+                DLMTestUtil.getLogRecordWithDLSNInstance(new DLSN(1L, 12L, 0L), 888L);
+        LogRecordWithDLSN record3 =
+                DLMTestUtil.getLogRecordWithDLSNInstance(new DLSN(1L, 12L, 0L), 999L);
+        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 10L, 11L),
+                ReadUtils.getEntriesToSearch(888L, record1, record2, 10));
+        assertEquals(Lists.newArrayList(1L, 2L, 3L, 4L, 5L, 6L, 7L, 8L, 9L, 11L),
+                ReadUtils.getEntriesToSearch(888L, record1, record3, 10));
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordNotLessThanTxIdWithGreaterTxId() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
+        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 1, 1 /* txid */);
+
+        Optional<LogRecordWithDLSN> result =
+                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 999L));
+        assertFalse(result.isPresent());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordNotLessThanTxIdWithLessTxId() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
+        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 1, 999L /* txid */);
+
+        Optional<LogRecordWithDLSN> result =
+                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 99L));
+        assertTrue(result.isPresent());
+        assertEquals(999L, result.get().getTransactionId());
+        assertEquals(0L, result.get().getDlsn().getEntryId());
+        assertEquals(0L, result.get().getDlsn().getSlotId());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordNotLessThanTxIdOnSmallSegment() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
+        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 5, 1L /* txid */);
+
+        Optional<LogRecordWithDLSN> result =
+                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 3L));
+        assertTrue(result.isPresent());
+        assertEquals(3L, result.get().getTransactionId());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordNotLessThanTxIdOnLargeSegment() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
+        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 100, 1L /* txid */);
+
+        Optional<LogRecordWithDLSN> result =
+                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 9L));
+        assertTrue(result.isPresent());
+        assertEquals(9L, result.get().getTransactionId());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordGreaterThanTxIdOnLargeSegment() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
+        DLMTestUtil.generateLogSegmentNonPartitioned(bkdlm, 0 /* control recs */, 100, 1L /* txid */, 3L);
+
+        Optional<LogRecordWithDLSN> result =
+                FutureUtils.result(getLogRecordNotLessThanTxId(bkdlm, 0, 23L));
+        assertTrue(result.isPresent());
+        assertEquals(25L, result.get().getTransactionId());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogRecordGreaterThanTxIdOnSameTxId() throws Exception {
+        String streamName = runtime.getMethodName();
+        BKDistributedLogManager bkdlm = createNewDLM(conf, streamName);
+        AsyncLogWriter out = bkdlm.startAsyncLogSegmentNonPartitioned();
+        long txid = 1L;
+        for (int i = 0; i < 10; ++i) {
+            LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
+            Await.result(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));
+        }
+        for (int i = 0; i < 10; ++i) {
+            LogRecord record = DLMTestUtil.getLargeLogRecordInstance(txid);
+            Await.result(out.write(record));
+            txid += 1;
+        }
+        Utils.close(out);
+        Optional<LogRecordWithDLSN> result =
+                FutureUtils.result(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
new file mode 100644
index 0000000..ad5bf8e
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestReader.java
@@ -0,0 +1,206 @@
+/**
+ * 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 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;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.*;
+
+/**
+ * A Reader wraps reading next logic for testing.
+ */
+public class TestReader implements FutureEventListener<LogRecordWithDLSN> {
+
+    static final Logger LOG = LoggerFactory.getLogger(TestReader.class);
+
+    final String readerName;
+    final DistributedLogManager dlm;
+    AsyncLogReader reader;
+    final DLSN startDLSN;
+    DLSN nextDLSN;
+    final boolean simulateErrors;
+    int delayMs;
+    final ScheduledExecutorService executorService;
+
+    // Latches
+
+    // Latch on notifying reader is ready to read
+    final CountDownLatch readyLatch;
+    // Latch no notifying reads are completed or errors are encountered
+    final CountDownLatch completionLatch;
+    // Latch no notifying reads are done.
+    final CountDownLatch countLatch;
+
+    // States
+    final AtomicBoolean errorsFound;
+    final AtomicInteger readCount;
+    final AtomicInteger positionReaderCount;
+
+    public TestReader(String name,
+                      DistributedLogManager dlm,
+                      DLSN startDLSN,
+                      boolean simulateErrors,
+                      int delayMs,
+                      CountDownLatch readyLatch,
+                      CountDownLatch countLatch,
+                      CountDownLatch completionLatch) {
+        this.readerName = name;
+        this.dlm = dlm;
+        this.startDLSN = startDLSN;
+        this.simulateErrors = simulateErrors;
+        this.delayMs = delayMs;
+        this.readyLatch = readyLatch;
+        this.countLatch = countLatch;
+        this.completionLatch = completionLatch;
+        // States
+        this.errorsFound = new AtomicBoolean(false);
+        this.readCount = new AtomicInteger(0);
+        this.positionReaderCount = new AtomicInteger(0);
+        // Executors
+        this.executorService = Executors.newSingleThreadScheduledExecutor();
+    }
+
+    public AtomicInteger getNumReaderPositions() {
+        return this.positionReaderCount;
+    }
+
+    public AtomicInteger getNumReads() {
+        return this.readCount;
+    }
+
+    public boolean areErrorsFound() {
+        return errorsFound.get();
+    }
+
+    private int nextDelayMs() {
+        int newDelayMs = Math.min(delayMs * 2, 500);
+        if (0 == delayMs) {
+            newDelayMs = 10;
+        }
+        delayMs = newDelayMs;
+        return delayMs;
+    }
+
+    private void positionReader(final DLSN dlsn) {
+        positionReaderCount.incrementAndGet();
+        Runnable runnable = new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    AsyncLogReader reader = dlm.getAsyncLogReader(dlsn);
+                    if (simulateErrors) {
+                        ((BKAsyncLogReader) reader).simulateErrors();
+                    }
+                    nextDLSN = dlsn;
+                    LOG.info("Positioned reader {} at {}", readerName, dlsn);
+                    if (null != TestReader.this.reader) {
+                        Utils.close(TestReader.this.reader);
+                    }
+                    TestReader.this.reader = reader;
+                    readNext();
+                    readyLatch.countDown();
+                } catch (IOException exc) {
+                    int nextMs = nextDelayMs();
+                    LOG.info("Encountered exception {} on opening reader {} at {}, retrying in {} ms",
+                            new Object[] { exc, readerName, dlsn, nextMs });
+                    positionReader(dlsn);
+                }
+            }
+        };
+        executorService.schedule(runnable, delayMs, TimeUnit.MILLISECONDS);
+    }
+
+    private void readNext() {
+        Future<LogRecordWithDLSN> record = reader.readNext();
+        record.addEventListener(this);
+    }
+
+    @Override
+    public void onSuccess(LogRecordWithDLSN value) {
+        try {
+            assertTrue(value.getDlsn().compareTo(nextDLSN) >= 0);
+            LOG.info("Received record {} from log {} for reader {}",
+                    new Object[] { value.getDlsn(), dlm.getStreamName(), readerName });
+            assertFalse(value.isControl());
+            assertEquals(0, value.getDlsn().getSlotId());
+            DLMTestUtil.verifyLargeLogRecord(value);
+        } catch (Exception exc) {
+            LOG.error("Exception encountered when verifying received log record {} for reader {} :",
+                    new Object[] { value.getDlsn(), exc, readerName });
+            errorsFound.set(true);
+            completionLatch.countDown();
+            return;
+        }
+        readCount.incrementAndGet();
+        countLatch.countDown();
+        if (countLatch.getCount() <= 0) {
+            LOG.info("Reader {} is completed", readerName);
+            closeReader();
+            completionLatch.countDown();
+        } else {
+            LOG.info("Reader {} : read count becomes {}, latch = {}",
+                    new Object[] { readerName, readCount.get(), countLatch.getCount() });
+            nextDLSN = value.getDlsn().getNextDLSN();
+            readNext();
+        }
+    }
+
+    @Override
+    public void onFailure(Throwable cause) {
+        LOG.error("{} encountered exception on reading next record : ", readerName, cause);
+        closeReader();
+        nextDelayMs();
+        positionReader(nextDLSN);
+    }
+
+    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;
+                }
+            });
+        }
+    }
+
+    public void start() {
+        positionReader(startDLSN);
+    }
+
+    public void stop() {
+        closeReader();
+        executorService.shutdown();
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java
new file mode 100644
index 0000000..9032866
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestRollLogSegments.java
@@ -0,0 +1,429 @@
+/**
+ * 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 java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.CountDownLatch;
+
+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.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;
+import org.junit.Test;
+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 static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.*;
+
+public class TestRollLogSegments extends TestDistributedLogBase {
+    static final Logger logger = LoggerFactory.getLogger(TestRollLogSegments.class);
+
+    private static void ensureOnlyOneInprogressLogSegments(List<LogSegmentMetadata> segments) throws Exception {
+        int numInprogress = 0;
+        for (LogSegmentMetadata segment : segments) {
+            if (segment.isInProgress()) {
+                ++numInprogress;
+            }
+        }
+        assertEquals(1, numInprogress);
+    }
+
+    @Test(timeout = 60000)
+    public void testDisableRollingLogSegments() throws Exception {
+        String name = "distrlog-disable-rolling-log-segments";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setLogSegmentRollingIntervalMinutes(0);
+        confLocal.setMaxLogSegmentBytes(40);
+
+        int numEntries = 100;
+        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+
+        SettableFeature disableLogSegmentRolling =
+                (SettableFeature) dlm.getFeatureProvider()
+                        .getFeature(CoreFeatureKeys.DISABLE_LOGSEGMENT_ROLLING.name().toLowerCase());
+        disableLogSegmentRolling.set(true);
+
+        final CountDownLatch latch = new CountDownLatch(numEntries);
+
+        // send requests in parallel
+        for (int i = 1; i <= numEntries; i++) {
+            final int entryId = i;
+            writer.write(DLMTestUtil.getLogRecordInstance(entryId)).addEventListener(new FutureEventListener<DLSN>() {
+
+                @Override
+                public void onSuccess(DLSN value) {
+                    logger.info("Completed entry {} : {}.", entryId, value);
+                    latch.countDown();
+                }
+
+                @Override
+                public void onFailure(Throwable cause) {
+                    // nope
+                }
+            });
+        }
+        latch.await();
+
+        // make sure all ensure blocks were executed
+        writer.closeAndComplete();
+
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(1, segments.size());
+
+        dlm.close();
+    }
+
+    @Test(timeout = 600000)
+    public void testLastDLSNInRollingLogSegments() throws Exception {
+        final Map<Long, DLSN> lastDLSNs = new HashMap<Long, DLSN>();
+        String name = "distrlog-lastdlsn-in-rolling-log-segments";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setLogSegmentRollingIntervalMinutes(0);
+        confLocal.setMaxLogSegmentBytes(40);
+
+        int numEntries = 100;
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
+
+        final CountDownLatch latch = new CountDownLatch(numEntries);
+
+        // 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>() {
+
+                @Override
+                public void onSuccess(DLSN value) {
+                    logger.info("Completed entry {} : {}.", entryId, value);
+                    synchronized (lastDLSNs) {
+                        DLSN lastDLSN = lastDLSNs.get(value.getLogSegmentSequenceNo());
+                        if (null == lastDLSN || lastDLSN.compareTo(value) < 0) {
+                            lastDLSNs.put(value.getLogSegmentSequenceNo(), value);
+                        }
+                    }
+                    latch.countDown();
+                }
+
+                @Override
+                public void onFailure(Throwable cause) {
+
+                }
+            });
+            if (i == 1) {
+                // wait for first log segment created
+                FutureUtils.result(writeFuture);
+            }
+        }
+        latch.await();
+
+        // make sure all ensure blocks were executed.
+        writer.closeAndComplete();
+
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        logger.info("lastDLSNs after writes {} {}", lastDLSNs.size(), lastDLSNs);
+        logger.info("segments after writes {} {}", segments.size(), segments);
+        assertTrue(segments.size() >= 2);
+        assertTrue(lastDLSNs.size() >= 2);
+        assertEquals(lastDLSNs.size(), segments.size());
+        for (LogSegmentMetadata segment : segments) {
+            DLSN dlsnInMetadata = segment.getLastDLSN();
+            DLSN dlsnSeen = lastDLSNs.get(segment.getLogSegmentSequenceNumber());
+            assertNotNull(dlsnInMetadata);
+            assertNotNull(dlsnSeen);
+            if (dlsnInMetadata.compareTo(dlsnSeen) != 0) {
+                logger.error("Last dlsn recorded in log segment {} is different from the one already seen {}.",
+                             dlsnInMetadata, dlsnSeen);
+            }
+            assertEquals(0, dlsnInMetadata.compareTo(dlsnSeen));
+        }
+
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testUnableToRollLogSegments() throws Exception {
+        String name = "distrlog-unable-to-roll-log-segments";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setLogSegmentRollingIntervalMinutes(0);
+        confLocal.setMaxLogSegmentBytes(1);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
+
+        long txId = 1L;
+
+        // Create Log Segments
+        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId)));
+
+        FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_StartLogSegmentBeforeLedgerCreate,
+                FailpointUtils.FailPointActions.FailPointAction_Throw);
+
+        try {
+            // If we couldn't open new log segment, we should keep using the old one
+            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>() {
+                    @Override
+                    public void onSuccess(DLSN value) {
+                        logger.info("Completed entry : {}.", value);
+                        latch.countDown();
+                    }
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        logger.error("Failed to write entries : ", cause);
+                    }
+                });
+            }
+
+            latch.await();
+
+            writer.close();
+
+            List<LogSegmentMetadata> segments = dlm.getLogSegments();
+            logger.info("LogSegments: {}", segments);
+
+            assertEquals(1, segments.size());
+
+            long expectedTxID = 1L;
+            LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
+            LogRecordWithDLSN record = reader.readNext(false);
+            while (null != record) {
+                DLMTestUtil.verifyLogRecord(record);
+                assertEquals(expectedTxID++, record.getTransactionId());
+                assertEquals(record.getTransactionId() - 1, record.getSequenceId());
+
+                record = reader.readNext(false);
+            }
+
+            assertEquals(12L, expectedTxID);
+
+            reader.close();
+
+            dlm.close();
+        } finally {
+            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_StartLogSegmentBeforeLedgerCreate);
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testRollingLogSegments() throws Exception {
+        logger.info("start testRollingLogSegments");
+        String name = "distrlog-rolling-logsegments-hightraffic";
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setLogSegmentRollingIntervalMinutes(0);
+        confLocal.setMaxLogSegmentBytes(1);
+        confLocal.setLogSegmentRollingConcurrency(Integer.MAX_VALUE);
+
+        int numLogSegments = 10;
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = (BKAsyncLogWriter) dlm.startAsyncLogSegmentNonPartitioned();
+
+        final CountDownLatch latch = new CountDownLatch(numLogSegments);
+        long startTime = System.currentTimeMillis();
+        // 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>() {
+                @Override
+                public void onSuccess(DLSN value) {
+                    logger.info("Completed entry {} : {}.", entryId, value);
+                    latch.countDown();
+                }
+                @Override
+                public void onFailure(Throwable cause) {
+                    logger.error("Failed to write entries : {}", cause);
+                }
+            });
+            if (i == 1) {
+                // wait for first log segment created
+                FutureUtils.result(writeFuture);
+            }
+        }
+        latch.await();
+
+        logger.info("Took {} ms to completed all requests.", System.currentTimeMillis() - startTime);
+
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        logger.info("LogSegments : {}", segments);
+
+        assertTrue(segments.size() >= 2);
+        ensureOnlyOneInprogressLogSegments(segments);
+
+        int numSegmentsAfterAsyncWrites = segments.size();
+
+        // 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)));
+            logger.info("Completed entry {} : {}", numLogSegments + i, newDLSN);
+        }
+
+        segments = dlm.getLogSegments();
+        logger.info("LogSegments : {}", segments);
+
+        assertEquals(numSegmentsAfterAsyncWrites + numLogSegments / 2, segments.size());
+        ensureOnlyOneInprogressLogSegments(segments);
+
+        writer.close();
+        dlm.close();
+    }
+
+    private void checkAndWaitWriterReaderPosition(BKLogSegmentWriter writer, long expectedWriterPosition,
+                                                  BKAsyncLogReader reader, long expectedReaderPosition,
+                                                  LedgerHandle inspector, long expectedLac) throws Exception {
+        while (getLedgerHandle(writer).getLastAddConfirmed() < expectedWriterPosition) {
+            Thread.sleep(1000);
+        }
+        assertEquals(expectedWriterPosition, getLedgerHandle(writer).getLastAddConfirmed());
+        assertEquals(expectedLac, inspector.readLastConfirmed());
+        EntryPosition readPosition = reader.getReadAheadReader().getNextEntryPosition();
+        logger.info("ReadAhead moved read position {} : ", readPosition);
+        while (readPosition.getEntryId() < expectedReaderPosition) {
+            Thread.sleep(1000);
+            readPosition = reader.getReadAheadReader().getNextEntryPosition();
+            logger.info("ReadAhead moved read position {} : ", readPosition);
+        }
+        assertEquals(expectedReaderPosition, readPosition.getEntryId());
+    }
+
+    @FlakyTest
+    @Test(timeout = 60000)
+    @SuppressWarnings("deprecation")
+    public void testCaughtUpReaderOnLogSegmentRolling() throws Exception {
+        String name = "distrlog-caughtup-reader-on-logsegment-rolling";
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setOutputBufferSize(4 * 1024 * 1024);
+        confLocal.setTraceReadAheadMetadataChanges(true);
+        confLocal.setEnsembleSize(1);
+        confLocal.setWriteQuorumSize(1);
+        confLocal.setAckQuorumSize(1);
+        confLocal.setReadLACLongPollTimeout(99999999);
+        confLocal.setReaderIdleWarnThresholdMillis(2 * 99999999 + 1);
+        confLocal.setBKClientReadTimeout(99999999 + 1);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        BKSyncLogWriter writer = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+
+        // 1) writer added 5 entries.
+        final int numEntries = 5;
+        for (int i = 1; i <= numEntries; i++) {
+            writer.write(DLMTestUtil.getLogRecordInstance(i));
+            writer.setReadyToFlush();
+            writer.flushAndSync();
+        }
+
+        BKDistributedLogManager readDLM = (BKDistributedLogManager) createNewDLM(confLocal, name);
+        final BKAsyncLogReader reader = (BKAsyncLogReader) readDLM.getAsyncLogReader(DLSN.InitialDLSN);
+
+        // 2) reader should be able to read 5 entries.
+        for (long i = 1; i <= numEntries; i++) {
+            LogRecordWithDLSN record = Await.result(reader.readNext());
+            DLMTestUtil.verifyLogRecord(record);
+            assertEquals(i, record.getTransactionId());
+            assertEquals(record.getTransactionId() - 1, record.getSequenceId());
+        }
+
+        BKLogSegmentWriter perStreamWriter = writer.segmentWriter;
+        BookKeeperClient bkc = DLMTestUtil.getBookKeeperClient(readDLM);
+        LedgerHandle readLh = bkc.get().openLedgerNoRecovery(getLedgerHandle(perStreamWriter).getId(),
+                BookKeeper.DigestType.CRC32, conf.getBKDigestPW().getBytes(UTF_8));
+
+        // Writer moved to lac = 9, while reader knows lac = 8 and moving to wait on 9
+        checkAndWaitWriterReaderPosition(perStreamWriter, 9, reader, 9, readLh, 8);
+
+        // write 6th record
+        writer.write(DLMTestUtil.getLogRecordInstance(numEntries + 1));
+        writer.setReadyToFlush();
+        // Writer moved to lac = 10, while reader knows lac = 9 and moving to wait on 10
+        checkAndWaitWriterReaderPosition(perStreamWriter, 10, reader, 10, readLh, 9);
+
+        // write records without commit to simulate similar failure cases
+        writer.write(DLMTestUtil.getLogRecordInstance(numEntries + 2));
+        writer.setReadyToFlush();
+        // Writer moved to lac = 11, while reader knows lac = 10 and moving to wait on 11
+        checkAndWaitWriterReaderPosition(perStreamWriter, 11, reader, 11, readLh, 10);
+
+        while (true) {
+            BKLogSegmentEntryReader entryReader =
+                    (BKLogSegmentEntryReader) reader.getReadAheadReader().getCurrentSegmentReader().getEntryReader();
+            if (null != entryReader && null != entryReader.getOutstandingLongPoll()) {
+                break;
+            }
+            Thread.sleep(1000);
+        }
+        logger.info("Waiting for long poll getting interrupted with metadata changed");
+
+        // simulate a recovery without closing ledger causing recording wrong last dlsn
+        BKLogWriteHandler writeHandler = writer.getCachedWriteHandler();
+        writeHandler.completeAndCloseLogSegment(
+                writeHandler.inprogressZNodeName(perStreamWriter.getLogSegmentId(), perStreamWriter.getStartTxId(), perStreamWriter.getLogSegmentSequenceNumber()),
+                perStreamWriter.getLogSegmentSequenceNumber(),
+                perStreamWriter.getLogSegmentId(),
+                perStreamWriter.getStartTxId(), perStreamWriter.getLastTxId(),
+                perStreamWriter.getPositionWithinLogSegment() - 1,
+                9,
+                0);
+
+        BKSyncLogWriter anotherWriter = (BKSyncLogWriter) dlm.startLogSegmentNonPartitioned();
+        anotherWriter.write(DLMTestUtil.getLogRecordInstance(numEntries + 3));
+        anotherWriter.setReadyToFlush();
+        anotherWriter.flushAndSync();
+        anotherWriter.closeAndComplete();
+
+        for (long i = numEntries + 1; i <= numEntries + 3; i++) {
+            LogRecordWithDLSN record = Await.result(reader.readNext());
+            DLMTestUtil.verifyLogRecord(record);
+            assertEquals(i, record.getTransactionId());
+        }
+
+        Utils.close(reader);
+        readDLM.close();
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestSequenceID.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestSequenceID.java
new file mode 100644
index 0000000..9258922
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestSequenceID.java
@@ -0,0 +1,254 @@
+/**
+ * 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 org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion;
+import com.twitter.util.Await;
+import com.twitter.util.FutureEventListener;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.List;
+import java.util.concurrent.LinkedBlockingQueue;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test Cases related to sequence ids.
+ */
+public class TestSequenceID extends TestDistributedLogBase {
+
+    static final Logger logger = LoggerFactory.getLogger(TestSequenceID.class);
+
+    @Test(timeout = 60000)
+    public void testCompleteV4LogSegmentAsV4() throws Exception {
+        completeSingleInprogressSegment(LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value,
+                                        LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value);
+    }
+
+    @Test(timeout = 60000)
+    public void testCompleteV4LogSegmentAsV5() throws Exception {
+        completeSingleInprogressSegment(LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value,
+                                        LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value);
+    }
+
+    @Test(timeout = 60000)
+    public void testCompleteV5LogSegmentAsV4() throws Exception {
+        completeSingleInprogressSegment(LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value,
+                                        LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value);
+    }
+
+    @Test(timeout = 60000)
+    public void testCompleteV5LogSegmentAsV5() throws Exception {
+        completeSingleInprogressSegment(LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value,
+                                        LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value);
+    }
+
+    private void completeSingleInprogressSegment(int writeVersion, int completeVersion) throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setDLLedgerMetadataLayoutVersion(writeVersion);
+
+        String name = "distrlog-complete-single-inprogress-segment-versions-write-"
+                + writeVersion + "-complete-" + completeVersion;
+
+        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocal, name);
+        BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(0L)));
+
+        dlm.close();
+
+        DistributedLogConfiguration confLocal2 = new DistributedLogConfiguration();
+        confLocal2.addConfiguration(confLocal);
+        confLocal2.setDLLedgerMetadataLayoutVersion(completeVersion);
+
+        BKDistributedLogManager dlm2 = (BKDistributedLogManager) createNewDLM(confLocal2, name);
+        dlm2.startAsyncLogSegmentNonPartitioned();
+
+        List<LogSegmentMetadata> segments = dlm2.getLogSegments();
+        assertEquals(1, segments.size());
+
+        if (LogSegmentMetadata.supportsSequenceId(writeVersion)) {
+            if (LogSegmentMetadata.supportsSequenceId(completeVersion)) {
+                // the inprogress log segment is written in v5 and complete log segment in v5,
+                // then it support monotonic sequence id
+                assertEquals(0L, segments.get(0).getStartSequenceId());
+            } else {
+                // the inprogress log segment is written in v5 and complete log segment in v4,
+                // then it doesn't support monotonic sequence id
+                assertTrue(segments.get(0).getStartSequenceId() < 0);
+            }
+        } else {
+            // if the inprogress log segment is created prior to v5, it won't support monotonic sequence id
+            assertTrue(segments.get(0).getStartSequenceId() < 0);
+        }
+
+        dlm2.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testSequenceID() throws Exception {
+        DistributedLogConfiguration confLocalv4 = new DistributedLogConfiguration();
+        confLocalv4.addConfiguration(conf);
+        confLocalv4.setImmediateFlushEnabled(true);
+        confLocalv4.setOutputBufferSize(0);
+        confLocalv4.setDLLedgerMetadataLayoutVersion(LogSegmentMetadataVersion.VERSION_V4_ENVELOPED_ENTRIES.value);
+
+        String name = "distrlog-sequence-id";
+
+        BKDistributedLogManager readDLM = (BKDistributedLogManager) createNewDLM(conf, name);
+        AsyncLogReader reader = null;
+        final LinkedBlockingQueue<LogRecordWithDLSN> readRecords =
+                new LinkedBlockingQueue<LogRecordWithDLSN>();
+
+        BKDistributedLogManager dlm = (BKDistributedLogManager) createNewDLM(confLocalv4, name);
+
+        long txId = 0L;
+
+        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++)));
+
+                if (null == reader) {
+                    reader = readDLM.getAsyncLogReader(DLSN.InitialDLSN);
+                    final AsyncLogReader r = reader;
+                    reader.readNext().addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+                        @Override
+                        public void onSuccess(LogRecordWithDLSN record) {
+                            readRecords.add(record);
+                            r.readNext().addEventListener(this);
+                        }
+
+                        @Override
+                        public void onFailure(Throwable cause) {
+                            logger.error("Encountered exception on reading next : ", cause);
+                        }
+                    });
+                }
+            }
+            writer.closeAndComplete();
+        }
+
+        BKAsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+        Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txId++)));
+
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(4, segments.size());
+        for (int i = 0; i < 3; i++) {
+            assertFalse(segments.get(i).isInProgress());
+            assertTrue(segments.get(i).getStartSequenceId() < 0);
+        }
+        assertTrue(segments.get(3).isInProgress());
+        assertTrue(segments.get(3).getStartSequenceId() < 0);
+
+        dlm.close();
+
+        // simulate upgrading from v4 -> v5
+
+        DistributedLogConfiguration confLocalv5 = new DistributedLogConfiguration();
+        confLocalv5.addConfiguration(conf);
+        confLocalv5.setImmediateFlushEnabled(true);
+        confLocalv5.setOutputBufferSize(0);
+        confLocalv5.setDLLedgerMetadataLayoutVersion(LogSegmentMetadataVersion.VERSION_V5_SEQUENCE_ID.value);
+
+        BKDistributedLogManager dlmv5 = (BKDistributedLogManager) createNewDLM(confLocalv5, name);
+        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++)));
+            }
+            writerv5.closeAndComplete();
+        }
+        BKAsyncLogWriter writerv5 = dlmv5.startAsyncLogSegmentNonPartitioned();
+        Await.result(writerv5.write(DLMTestUtil.getLogRecordInstance(txId++)));
+
+        List<LogSegmentMetadata> segmentsv5 = dlmv5.getLogSegments();
+        assertEquals(8, segmentsv5.size());
+
+        assertFalse(segmentsv5.get(3).isInProgress());
+        assertTrue(segmentsv5.get(3).getStartSequenceId() < 0);
+
+        long startSequenceId = 0L;
+        for (int i = 4; i < 7; i++) {
+            assertFalse(segmentsv5.get(i).isInProgress());
+            assertEquals(startSequenceId, segmentsv5.get(i).getStartSequenceId());
+            startSequenceId += 2L;
+        }
+
+        assertTrue(segmentsv5.get(7).isInProgress());
+        assertEquals(startSequenceId, segmentsv5.get(7).getStartSequenceId());
+
+        dlmv5.close();
+
+        // rollback from v5 to v4
+
+        BKDistributedLogManager dlmv4 = (BKDistributedLogManager) createNewDLM(confLocalv4, name);
+        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++)));
+            }
+            writerv4.closeAndComplete();
+        }
+
+        List<LogSegmentMetadata> segmentsv4 = dlmv4.getLogSegments();
+        assertEquals(11, segmentsv4.size());
+
+        for(int i = 7; i < 11; i++) {
+            assertFalse(segmentsv4.get(i).isInProgress());
+            assertTrue(segmentsv4.get(i).getStartSequenceId() < 0);
+        }
+
+        dlmv4.close();
+
+        // wait until readers read all records
+        while (readRecords.size() < txId) {
+            Thread.sleep(100);
+        }
+
+        assertEquals(txId, readRecords.size());
+        long sequenceId = Long.MIN_VALUE;
+        for (LogRecordWithDLSN record : readRecords) {
+            if (record.getDlsn().getLogSegmentSequenceNo() <= 4) {
+                assertTrue(record.getSequenceId() < 0);
+                assertTrue(record.getSequenceId() > sequenceId);
+                sequenceId = record.getSequenceId();
+            } else if (record.getDlsn().getLogSegmentSequenceNo() <= 7) {
+                if (sequenceId < 0L) {
+                    sequenceId = 0L;
+                }
+                assertEquals(sequenceId, record.getSequenceId());
+                ++sequenceId;
+            } else if (record.getDlsn().getLogSegmentSequenceNo() >= 9) {
+                if (sequenceId > 0) {
+                    sequenceId = Long.MIN_VALUE;
+                }
+                assertTrue(record.getSequenceId() < 0);
+                assertTrue(record.getSequenceId() > sequenceId);
+                sequenceId = record.getSequenceId();
+            }
+        }
+
+        readDLM.close();
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestTruncate.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestTruncate.java
new file mode 100644
index 0000000..5b26a70
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestTruncate.java
@@ -0,0 +1,341 @@
+/**
+ * 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 java.net.URI;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.distributedlog.util.FutureUtils;
+import org.apache.distributedlog.util.Utils;
+import org.apache.commons.lang3.tuple.ImmutablePair;
+import org.apache.commons.lang3.tuple.Pair;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.distributedlog.LogSegmentMetadata.TruncationStatus;
+import com.twitter.util.Await;
+
+import static org.junit.Assert.*;
+
+public class TestTruncate extends TestDistributedLogBase {
+    static final Logger LOG = LoggerFactory.getLogger(TestTruncate.class);
+
+    protected static DistributedLogConfiguration conf =
+            new DistributedLogConfiguration()
+                    .setLockTimeout(10)
+                    .setOutputBufferSize(0)
+                    .setPeriodicFlushFrequencyMilliSeconds(10)
+                    .setSchedulerShutdownTimeoutMs(0)
+                    .setDLLedgerMetadataLayoutVersion(LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO.value);
+
+    static void updateCompletionTime(ZooKeeperClient zkc,
+                                     LogSegmentMetadata l, long completionTime) throws Exception {
+        LogSegmentMetadata newSegment = l.mutator().setCompletionTime(completionTime).build();
+        DLMTestUtil.updateSegmentMetadata(zkc, newSegment);
+    }
+
+    static void setTruncationStatus(ZooKeeperClient zkc,
+                                    LogSegmentMetadata l,
+                                    TruncationStatus status) throws Exception {
+        LogSegmentMetadata newSegment =
+                l.mutator().setTruncationStatus(status).build();
+        DLMTestUtil.updateSegmentMetadata(zkc, newSegment);
+    }
+
+    @Test(timeout = 60000)
+    public void testPurgeLogs() throws Exception {
+        String name = "distrlog-purge-logs";
+        URI uri = createDLMURI("/" + name);
+
+        populateData(new HashMap<Long, DLSN>(), conf, name, 10, 10, false);
+
+        DistributedLogManager distributedLogManager = createNewDLM(conf, name);
+
+        List<LogSegmentMetadata> segments = distributedLogManager.getLogSegments();
+        LOG.info("Segments before modifying completion time : {}", segments);
+
+        ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder(conf)
+                .uri(uri)
+                .build();
+
+        // Update completion time of first 5 segments
+        long newTimeMs = System.currentTimeMillis() - 60*60*1000*2;
+        for (int i = 0; i < 5; i++) {
+            LogSegmentMetadata segment = segments.get(i);
+            updateCompletionTime(zkc, segment, newTimeMs + i);
+        }
+        zkc.close();
+
+        segments = distributedLogManager.getLogSegments();
+        LOG.info("Segments after modifying completion time : {}", segments);
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setRetentionPeriodHours(1);
+        confLocal.setExplicitTruncationByApplication(false);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+        long txid = 1 + 10 * 10;
+        for (int j = 1; j <= 10; j++) {
+            Await.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
+        }
+
+        // to make sure the truncation task is executed
+        DLSN lastDLSN = Await.result(dlm.getLastDLSNAsync());
+        LOG.info("Get last dlsn of stream {} : {}", name, lastDLSN);
+
+        assertEquals(6, distributedLogManager.getLogSegments().size());
+
+        Utils.close(writer);
+        dlm.close();
+
+        distributedLogManager.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testTruncation() throws Exception {
+        String name = "distrlog-truncation";
+
+        long txid = 1;
+        Map<Long, DLSN> txid2DLSN = new HashMap<Long, DLSN>();
+        Pair<DistributedLogManager, AsyncLogWriter> pair =
+                populateData(txid2DLSN, conf, name, 4, 10, true);
+
+        Thread.sleep(1000);
+
+        // delete invalid dlsn
+        assertFalse(Await.result(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)));
+            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)));
+        verifyEntries(name, 1, 41, 10);
+
+        Utils.close(pair.getRight());
+        pair.getLeft().close();
+    }
+
+    @Test(timeout = 60000)
+    public void testExplicitTruncation() throws Exception {
+        String name = "distrlog-truncation-explicit";
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setExplicitTruncationByApplication(true);
+
+        Map<Long, DLSN> txid2DLSN = new HashMap<Long, DLSN>();
+        Pair<DistributedLogManager, AsyncLogWriter> pair =
+                populateData(txid2DLSN, confLocal, name, 4, 10, true);
+
+        Thread.sleep(1000);
+
+        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)));
+            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)));
+        verifyEntries(name, 1, 41, 10);
+
+        Utils.close(pair.getRight());
+        pair.getLeft().close();
+
+        // Try force truncation
+        BKDistributedLogManager dlm = (BKDistributedLogManager)createNewDLM(confLocal, name);
+        BKLogWriteHandler handler = dlm.createWriteHandler(true);
+        FutureUtils.result(handler.purgeLogSegmentsOlderThanTxnId(Integer.MAX_VALUE));
+
+        verifyEntries(name, 1, 41, 10);
+    }
+
+    @Test(timeout = 60000)
+    public void testOnlyPurgeSegmentsBeforeNoneFullyTruncatedSegment() throws Exception {
+        String name = "distrlog-only-purge-segments-before-none-fully-truncated-segment";
+        URI uri = createDLMURI("/" + name);
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setExplicitTruncationByApplication(true);
+
+        // populate data
+        populateData(new HashMap<Long, DLSN>(), confLocal, name, 4, 10, false);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        LOG.info("Segments before modifying segment status : {}", segments);
+
+        ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder(conf)
+                .uri(uri)
+                .build();
+        setTruncationStatus(zkc, segments.get(0), TruncationStatus.PARTIALLY_TRUNCATED);
+        for (int i = 1; i < 4; i++) {
+            LogSegmentMetadata segment = segments.get(i);
+            setTruncationStatus(zkc, segment, TruncationStatus.TRUNCATED);
+        }
+        List<LogSegmentMetadata> segmentsAfterTruncated = dlm.getLogSegments();
+
+        dlm.purgeLogsOlderThan(999999);
+        List<LogSegmentMetadata> newSegments = dlm.getLogSegments();
+        LOG.info("Segments after purge segments older than 999999 : {}", newSegments);
+        assertArrayEquals(segmentsAfterTruncated.toArray(new LogSegmentMetadata[segmentsAfterTruncated.size()]),
+                          newSegments.toArray(new LogSegmentMetadata[newSegments.size()]));
+
+        dlm.close();
+
+        // Update completion time of all 4 segments
+        long newTimeMs = System.currentTimeMillis() - 60 * 60 * 1000 * 10;
+        for (int i = 0; i < 4; i++) {
+            LogSegmentMetadata segment = newSegments.get(i);
+            updateCompletionTime(zkc, segment, newTimeMs + i);
+        }
+
+        DistributedLogConfiguration newConf = new DistributedLogConfiguration();
+        newConf.addConfiguration(confLocal);
+        newConf.setRetentionPeriodHours(1);
+
+        DistributedLogManager newDLM = createNewDLM(newConf, name);
+        AsyncLogWriter newWriter = newDLM.startAsyncLogSegmentNonPartitioned();
+        long txid = 1 + 4 * 10;
+        for (int j = 1; j <= 10; j++) {
+            Await.result(newWriter.write(DLMTestUtil.getLogRecordInstance(txid++)));
+        }
+
+        // to make sure the truncation task is executed
+        DLSN lastDLSN = Await.result(newDLM.getLastDLSNAsync());
+        LOG.info("Get last dlsn of stream {} : {}", name, lastDLSN);
+
+        assertEquals(5, newDLM.getLogSegments().size());
+
+        Utils.close(newWriter);
+        newDLM.close();
+
+        zkc.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testPartiallyTruncateTruncatedSegments() throws Exception {
+        String name = "distrlog-partially-truncate-truncated-segments";
+        URI uri = createDLMURI("/" + name);
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setExplicitTruncationByApplication(true);
+
+        // populate
+        Map<Long, DLSN> dlsnMap = new HashMap<Long, DLSN>();
+        populateData(dlsnMap, confLocal, name, 4, 10, false);
+
+        DistributedLogManager dlm = createNewDLM(confLocal, name);
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        LOG.info("Segments before modifying segment status : {}", segments);
+
+        ZooKeeperClient zkc = TestZooKeeperClientBuilder.newBuilder(conf)
+                .uri(uri)
+                .build();
+        for (int i = 0; i < 4; i++) {
+            LogSegmentMetadata segment = segments.get(i);
+            setTruncationStatus(zkc, segment, TruncationStatus.TRUNCATED);
+        }
+
+        List<LogSegmentMetadata> newSegments = dlm.getLogSegments();
+        LOG.info("Segments after changing truncation status : {}", newSegments);
+
+        dlm.close();
+
+        DistributedLogManager newDLM = createNewDLM(confLocal, name);
+        AsyncLogWriter newWriter = newDLM.startAsyncLogSegmentNonPartitioned();
+        Await.result(newWriter.truncate(dlsnMap.get(15L)));
+
+        List<LogSegmentMetadata> newSegments2 = newDLM.getLogSegments();
+        assertArrayEquals(newSegments.toArray(new LogSegmentMetadata[4]),
+                          newSegments2.toArray(new LogSegmentMetadata[4]));
+
+        Utils.close(newWriter);
+        newDLM.close();
+        zkc.close();
+    }
+
+    private Pair<DistributedLogManager, AsyncLogWriter> populateData(
+            Map<Long, DLSN> txid2DLSN, DistributedLogConfiguration confLocal,
+            String name, int numLogSegments, int numEntriesPerLogSegment,
+            boolean createInprogressLogSegment) throws Exception {
+        long txid = 1;
+        for (long i = 1; i <= numLogSegments; i++) {
+            LOG.info("Writing Log Segment {}.", i);
+            DistributedLogManager dlm = createNewDLM(confLocal, name);
+            AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+            for (int j = 1; j <= numEntriesPerLogSegment; j++) {
+                long curTxId = txid++;
+                DLSN dlsn = Await.result(writer.write(DLMTestUtil.getLogRecordInstance(curTxId)));
+                txid2DLSN.put(curTxId, dlsn);
+            }
+            Utils.close(writer);
+            dlm.close();
+        }
+
+        if (createInprogressLogSegment) {
+            DistributedLogManager dlm = createNewDLM(confLocal, name);
+            AsyncLogWriter writer = dlm.startAsyncLogSegmentNonPartitioned();
+            for (int j = 1; j <= 10; j++) {
+                long curTxId = txid++;
+                DLSN dlsn = Await.result(writer.write(DLMTestUtil.getLogRecordInstance(curTxId)));
+                txid2DLSN.put(curTxId, dlsn);
+            }
+            return new ImmutablePair<DistributedLogManager, AsyncLogWriter>(dlm, writer);
+        } else {
+            return null;
+        }
+    }
+
+    private void verifyEntries(String name, long readFromTxId, long startTxId, int numEntries) throws Exception {
+        DistributedLogManager dlm = createNewDLM(conf, name);
+        LogReader reader = dlm.getInputStream(readFromTxId);
+
+        long txid = startTxId;
+        int numRead = 0;
+        LogRecord r = reader.readNext(false);
+        while (null != r) {
+            DLMTestUtil.verifyLogRecord(r);
+            assertEquals(txid++, r.getTransactionId());
+            ++numRead;
+            r = reader.readNext(false);
+        }
+        assertEquals(numEntries, numRead);
+        reader.close();
+        dlm.close();
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java
new file mode 100644
index 0000000..c28437f
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestWriteLimiter.java
@@ -0,0 +1,205 @@
+/**
+ * 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 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;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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);
+
+    SimplePermitLimiter createPermitLimiter(boolean darkmode, int permits) {
+        return createPermitLimiter(darkmode, permits, new SettableFeature("", 0));
+    }
+
+    SimplePermitLimiter createPermitLimiter(boolean darkmode, int permits, Feature feature) {
+        return new SimplePermitLimiter(darkmode, permits, new NullStatsLogger(), false, feature);
+    }
+
+    @Test(timeout = 60000)
+    public void testGlobalOnly() throws Exception {
+        SimplePermitLimiter streamLimiter = createPermitLimiter(false, Integer.MAX_VALUE);
+        SimplePermitLimiter globalLimiter = createPermitLimiter(false, 1);
+        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
+        limiter.acquire();
+        try {
+            limiter.acquire();
+            fail("should have thrown global limit exception");
+        } catch (OverCapacityException ex) {
+        }
+        assertPermits(streamLimiter, 1, globalLimiter, 1);
+        limiter.release();
+        assertPermits(streamLimiter, 0, globalLimiter, 0);
+    }
+
+    @Test(timeout = 60000)
+    public void testStreamOnly() throws Exception {
+        SimplePermitLimiter streamLimiter = createPermitLimiter(false, 1);
+        SimplePermitLimiter globalLimiter = createPermitLimiter(false, Integer.MAX_VALUE);
+        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
+        limiter.acquire();
+        try {
+            limiter.acquire();
+            fail("should have thrown stream limit exception");
+        } catch (OverCapacityException ex) {
+        }
+        assertPermits(streamLimiter, 1, globalLimiter, 1);
+    }
+
+    @Test(timeout = 60000)
+    public void testDarkmode() throws Exception {
+        SimplePermitLimiter streamLimiter = createPermitLimiter(true, Integer.MAX_VALUE);
+        SimplePermitLimiter globalLimiter = createPermitLimiter(true, 1);
+        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
+        limiter.acquire();
+        limiter.acquire();
+        assertPermits(streamLimiter, 2, globalLimiter, 2);
+    }
+
+    @Test(timeout = 60000)
+    public void testDarkmodeWithDisabledFeature() throws Exception {
+        SettableFeature feature = new SettableFeature("test", 10000);
+        SimplePermitLimiter streamLimiter = createPermitLimiter(true, 1, feature);
+        SimplePermitLimiter globalLimiter = createPermitLimiter(true, Integer.MAX_VALUE, feature);
+        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
+        limiter.acquire();
+        limiter.acquire();
+        assertPermits(streamLimiter, 2, globalLimiter, 2);
+        limiter.release();
+        limiter.release();
+        assertPermits(streamLimiter, 0, globalLimiter, 0);
+    }
+
+    @Test(timeout = 60000)
+    public void testDisabledFeature() throws Exception {
+        // Disable darkmode, but should still ignore limits because of the feature.
+        SettableFeature feature = new SettableFeature("test", 10000);
+        SimplePermitLimiter streamLimiter = createPermitLimiter(false, 1, feature);
+        SimplePermitLimiter globalLimiter = createPermitLimiter(false, Integer.MAX_VALUE, feature);
+        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
+        limiter.acquire();
+        limiter.acquire();
+        assertPermits(streamLimiter, 2, globalLimiter, 2);
+        limiter.release();
+        limiter.release();
+        assertPermits(streamLimiter, 0, globalLimiter, 0);
+    }
+
+    @Test(timeout = 60000)
+    public void testSetDisableFeatureAfterAcquireAndBeforeRelease() throws Exception {
+        SettableFeature feature = new SettableFeature("test", 0);
+        SimplePermitLimiter streamLimiter = createPermitLimiter(false, 2, feature);
+        SimplePermitLimiter globalLimiter = createPermitLimiter(false, Integer.MAX_VALUE, feature);
+        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
+        limiter.acquire();
+        limiter.acquire();
+        assertPermits(streamLimiter, 2, globalLimiter, 2);
+        feature.set(10000);
+        limiter.release();
+        limiter.release();
+        assertPermits(streamLimiter, 0, globalLimiter, 0);
+    }
+
+    @Test(timeout = 60000)
+    public void testUnsetDisableFeatureAfterPermitsExceeded() throws Exception {
+        SettableFeature feature = new SettableFeature("test", 10000);
+        SimplePermitLimiter streamLimiter = createPermitLimiter(false, 1, feature);
+        SimplePermitLimiter globalLimiter = createPermitLimiter(false, Integer.MAX_VALUE, feature);
+        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
+        limiter.acquire();
+        limiter.acquire();
+        limiter.acquire();
+        limiter.acquire();
+        assertPermits(streamLimiter, 4, globalLimiter, 4);
+        feature.set(0);
+        limiter.release();
+        assertPermits(streamLimiter, 3, globalLimiter, 3);
+        try {
+            limiter.acquire();
+            fail("should have thrown stream limit exception");
+        } catch (OverCapacityException ex) {
+        }
+        assertPermits(streamLimiter, 3, globalLimiter, 3);
+        limiter.release();
+        limiter.release();
+        limiter.release();
+        assertPermits(streamLimiter, 0, globalLimiter, 0);
+    }
+
+    @Test(timeout = 60000)
+    public void testUnsetDisableFeatureBeforePermitsExceeded() throws Exception {
+        SettableFeature feature = new SettableFeature("test", 0);
+        SimplePermitLimiter streamLimiter = createPermitLimiter(false, 1, feature);
+        SimplePermitLimiter globalLimiter = createPermitLimiter(false, Integer.MAX_VALUE, feature);
+        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
+        limiter.acquire();
+        try {
+            limiter.acquire();
+            fail("should have thrown stream limit exception");
+        } catch (OverCapacityException ex) {
+        }
+        assertPermits(streamLimiter, 1, globalLimiter, 1);
+        feature.set(10000);
+        limiter.acquire();
+        assertPermits(streamLimiter, 2, globalLimiter, 2);
+    }
+
+    @Test(timeout = 60000)
+    public void testDarkmodeGlobalUnderStreamOver() throws Exception {
+        SimplePermitLimiter streamLimiter = createPermitLimiter(true, 1);
+        SimplePermitLimiter globalLimiter = createPermitLimiter(true, 2);
+        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
+        limiter.acquire();
+        limiter.acquire();
+        assertPermits(streamLimiter, 2, globalLimiter, 2);
+        limiter.release();
+        limiter.release();
+        assertPermits(streamLimiter, 0, globalLimiter, 0);
+    }
+
+    @Test(timeout = 60000)
+    public void testDarkmodeGlobalOverStreamUnder() throws Exception {
+        SimplePermitLimiter streamLimiter = createPermitLimiter(true, 2);
+        SimplePermitLimiter globalLimiter = createPermitLimiter(true, 1);
+        WriteLimiter limiter = new WriteLimiter("test", streamLimiter, globalLimiter);
+        limiter.acquire();
+        limiter.acquire();
+        assertPermits(streamLimiter, 2, globalLimiter, 2);
+        limiter.release();
+        assertPermits(streamLimiter, 1, globalLimiter, 1);
+        limiter.release();
+        assertPermits(streamLimiter, 0, globalLimiter, 0);
+    }
+
+    void assertPermits(SimplePermitLimiter streamLimiter, int streamPermits, SimplePermitLimiter globalLimiter, int globalPermits) {
+        assertEquals(streamPermits, streamLimiter.getPermits());
+        assertEquals(globalPermits, globalLimiter.getPermits());
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java
new file mode 100644
index 0000000..a1c075f
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClient.java
@@ -0,0 +1,449 @@
+/**
+ * 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 org.apache.distributedlog.ZooKeeperClient.Credentials;
+import org.apache.distributedlog.ZooKeeperClient.DigestCredentials;
+import org.apache.distributedlog.annotations.DistributedLogAnnotations;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.zookeeper.BoundExponentialBackoffRetryPolicy;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.Watcher.Event.EventType;
+import org.apache.zookeeper.Watcher.Event.KeeperState;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.lang.reflect.Field;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test Cases for {@link org.apache.distributedlog.ZooKeeperClient}
+ */
+public class TestZooKeeperClient extends ZooKeeperClusterTestCase {
+    static final Logger LOG = LoggerFactory.getLogger(TestZooKeeperClient.class);
+
+    private final static int sessionTimeoutMs = 2000;
+
+    private ZooKeeperClient zkc;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = buildClient();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        zkc.close();
+    }
+
+    private ZooKeeperClientBuilder clientBuilder() throws Exception {
+        return clientBuilder(sessionTimeoutMs);
+    }
+
+    private ZooKeeperClientBuilder clientBuilder(int sessionTimeoutMs)
+            throws Exception {
+        return ZooKeeperClientBuilder.newBuilder()
+                .name("zkc")
+                .uri(DLMTestUtil.createDLMURI(zkPort, "/"))
+                .sessionTimeoutMs(sessionTimeoutMs)
+                .zkServers(zkServers)
+                .retryPolicy(new BoundExponentialBackoffRetryPolicy(100, 200, 2));
+    }
+
+    private ZooKeeperClient buildClient() throws Exception {
+        return clientBuilder().zkAclId(null).build();
+    }
+
+    private ZooKeeperClient buildAuthdClient(String id) throws Exception {
+        return clientBuilder().zkAclId(id).build();
+    }
+
+    private void rmAll(ZooKeeperClient client, String path) throws Exception {
+        List<String> nodes = client.get().getChildren(path, false);
+        for (String node : nodes) {
+            String childPath = path + "/" + node;
+            rmAll(client, childPath);
+        }
+        client.get().delete(path, 0);
+    }
+
+    @Test(timeout = 60000)
+    public void testAclCreatePerms() throws Exception {
+        ZooKeeperClient zkcAuth = buildAuthdClient("test");
+        zkcAuth.get().create("/test", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zkcAuth.get().create("/test/key1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zkcAuth.get().create("/test/key2", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+
+        ZooKeeperClient zkcNoAuth = buildClient();
+        zkcNoAuth.get().create("/test/key1/key1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        try {
+            zkcNoAuth.get().create("/test/key2/key1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+            fail("create should fail on acl protected key");
+        } catch (KeeperException.NoAuthException ex) {
+            LOG.info("caught exception writing to protected key", ex);
+        }
+
+        rmAll(zkcAuth, "/test");
+    }
+
+    @Test(timeout = 60000)
+    public void testAclNullIdDisablesAuth() throws Exception {
+        ZooKeeperClient zkcAuth = buildAuthdClient(null);
+        zkcAuth.get().create("/test", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zkcAuth.get().create("/test/key1", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        try {
+            zkcAuth.get().create("/test/key2", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+            fail("create should fail because we're not authenticated");
+        } catch (KeeperException.InvalidACLException ex) {
+            LOG.info("caught exception writing to protected key", ex);
+        }
+
+        rmAll(zkcAuth, "/test");
+    }
+
+    @Test(timeout = 60000)
+    public void testAclAllowsReadsForNoAuth() throws Exception {
+        ZooKeeperClient zkcAuth = buildAuthdClient("test");
+        zkcAuth.get().create("/test", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+        zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+        zkcAuth.get().create("/test/key1/key2", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+
+        ZooKeeperClient zkcNoAuth = buildClient();
+        List<String> nodes = null;
+        String path = "/test";
+        nodes = zkcNoAuth.get().getChildren(path, false);
+        path = path + "/" + nodes.get(0);
+        nodes = zkcNoAuth.get().getChildren(path, false);
+        assertEquals("key2", nodes.get(0));
+
+        ZooKeeperClient zkcAuth2 = buildAuthdClient("test2");
+        path = "/test";
+        nodes = zkcNoAuth.get().getChildren(path, false);
+        path = path + "/" + nodes.get(0);
+        nodes = zkcNoAuth.get().getChildren(path, false);
+        assertEquals("key2", nodes.get(0));
+
+        rmAll(zkcAuth, "/test");
+    }
+
+    @Test(timeout = 60000)
+    public void testAclDigestCredentialsBasics() throws Exception {
+        ZooKeeperClient zkcAuth = buildClient();
+        zkcAuth.get().create("/test", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+        try {
+            zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+            fail("should have failed");
+        } catch (Exception ex) {
+        }
+
+        Credentials credentials = new DigestCredentials("test", "test");
+        credentials.authenticate(zkcAuth.get());
+
+        // Should not throw now that we're authenticated.
+        zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+
+        rmAll(zkcAuth, "/test");
+    }
+
+    @Test(timeout = 60000)
+    public void testAclNoopCredentialsDoesNothing() throws Exception {
+        Credentials.NONE.authenticate(null);
+    }
+
+    class FailingCredentials implements Credentials {
+        boolean shouldFail = true;
+        @Override
+        public void authenticate(ZooKeeper zooKeeper) {
+            if (shouldFail) {
+                throw new RuntimeException("authfailed");
+            }
+        }
+        public void setShouldFail(boolean shouldFail) {
+            this.shouldFail = shouldFail;
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testAclFailedAuthenticationCanBeRecovered() throws Exception {
+        FailingCredentials credentials = new FailingCredentials();
+        ZooKeeperClient zkc = new ZooKeeperClient("test", 2000, 2000, zkServers,
+                null, NullStatsLogger.INSTANCE, 1, 10000, credentials);
+
+        try {
+            zkc.get();
+            fail("should have failed on auth");
+        } catch (Exception ex) {
+            assertEquals("authfailed", ex.getMessage());
+        }
+
+        // Should recover fine
+        credentials.setShouldFail(false);
+        zkc.get().create("/test", new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+
+        rmAll(zkc, "/test");
+    }
+
+    private void expireZooKeeperSession(ZooKeeper zk, int timeout)
+            throws IOException, InterruptedException, KeeperException {
+        final CountDownLatch latch = new CountDownLatch(1);
+
+        ZooKeeper newZk = new ZooKeeper(zkServers, timeout, new Watcher() {
+            @Override
+            public void process(WatchedEvent event) {
+                if (event.getType() == EventType.None && event.getState() == KeeperState.SyncConnected) {
+                    latch.countDown();
+                }
+            }},
+            zk.getSessionId(),
+            zk.getSessionPasswd());
+
+        if (!latch.await(timeout, TimeUnit.MILLISECONDS)) {
+            throw KeeperException.create(KeeperException.Code.CONNECTIONLOSS);
+        }
+
+        newZk.close();
+    }
+
+    private CountDownLatch awaitConnectionEvent(final KeeperState state, final ZooKeeperClient zkc) {
+        final CountDownLatch connected = new CountDownLatch(1);
+        Watcher watcher = new Watcher() {
+            @Override
+            public void process(WatchedEvent event) {
+                if (event.getType() == EventType.None && event.getState() == state) {
+                    connected.countDown();
+                }
+            }
+        };
+        zkc.register(watcher);
+        return connected;
+    }
+
+    /**
+     * {@link https://issues.apache.org/jira/browse/DL-34}
+     */
+    @DistributedLogAnnotations.FlakyTest
+    @Ignore
+    @Test(timeout = 60000)
+    public void testAclAuthSpansExpiration() throws Exception {
+        ZooKeeperClient zkcAuth = buildAuthdClient("test");
+        zkcAuth.get().create("/test", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+
+        CountDownLatch expired = awaitConnectionEvent(KeeperState.Expired, zkcAuth);
+        CountDownLatch connected = awaitConnectionEvent(KeeperState.SyncConnected, zkcAuth);
+
+        expireZooKeeperSession(zkcAuth.get(), 2000);
+
+        expired.await(2, TimeUnit.SECONDS);
+        connected.await(2, TimeUnit.SECONDS);
+
+        zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+
+        rmAll(zkcAuth, "/test");
+    }
+
+    /**
+     * {@link https://issues.apache.org/jira/browse/DL-34}
+     */
+    @DistributedLogAnnotations.FlakyTest
+    @Ignore
+    @Test(timeout = 60000)
+    public void testAclAuthSpansExpirationNonRetryableClient() throws Exception {
+        ZooKeeperClient zkcAuth = clientBuilder().retryPolicy(null).zkAclId("test").build();
+        zkcAuth.get().create("/test", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+
+        CountDownLatch expired = awaitConnectionEvent(KeeperState.Expired, zkcAuth);
+        CountDownLatch connected = awaitConnectionEvent(KeeperState.SyncConnected, zkcAuth);
+
+        expireZooKeeperSession(zkcAuth.get(), 2000);
+
+        expired.await(2, TimeUnit.SECONDS);
+        connected.await(2, TimeUnit.SECONDS);
+
+        zkcAuth.get().create("/test/key1", new byte[0], DistributedLogConstants.EVERYONE_READ_CREATOR_ALL, CreateMode.PERSISTENT);
+
+        rmAll(zkcAuth, "/test");
+    }
+
+    static class TestWatcher implements Watcher {
+
+        final List<WatchedEvent> receivedEvents = new ArrayList<WatchedEvent>();
+        CountDownLatch latch = new CountDownLatch(0);
+
+        public TestWatcher setLatch(CountDownLatch latch) {
+            this.latch = latch;
+            return this;
+        }
+
+        @Override
+        public void process(WatchedEvent event) {
+            if (event.getType() == Event.EventType.NodeDataChanged) {
+                synchronized (receivedEvents) {
+                    receivedEvents.add(event);
+                }
+                latch.countDown();
+            }
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testRegisterUnregisterWatchers() throws Exception {
+        TestWatcher w1 = new TestWatcher();
+        TestWatcher w2 = new TestWatcher();
+
+        final CountDownLatch latch = new CountDownLatch(2);
+        w1.setLatch(latch);
+        w2.setLatch(latch);
+
+        zkc.register(w1);
+        zkc.register(w2);
+
+        assertEquals(2, zkc.watchers.size());
+
+        final String zkPath = "/test-register-unregister-watchers";
+
+        zkc.get().create(zkPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zkc.get().getData(zkPath, true, new Stat());
+
+        zkc.get().setData(zkPath, "first-set".getBytes(), -1);
+        latch.await();
+        assertEquals(1, w1.receivedEvents.size());
+        assertEquals(zkPath, w1.receivedEvents.get(0).getPath());
+        assertEquals(Watcher.Event.EventType.NodeDataChanged, w1.receivedEvents.get(0).getType());
+        assertEquals(1, w2.receivedEvents.size());
+        assertEquals(zkPath, w2.receivedEvents.get(0).getPath());
+        assertEquals(Watcher.Event.EventType.NodeDataChanged, w2.receivedEvents.get(0).getType());
+
+        final CountDownLatch latch1 = new CountDownLatch(1);
+        final CountDownLatch latch2 = new CountDownLatch(1);
+        w1.setLatch(latch1);
+        w2.setLatch(latch2);
+
+        zkc.unregister(w2);
+
+        assertEquals(1, zkc.watchers.size());
+        zkc.get().getData(zkPath, true, new Stat());
+        zkc.get().setData(zkPath, "second-set".getBytes(), -1);
+        latch1.await();
+        assertEquals(2, w1.receivedEvents.size());
+        assertEquals(zkPath, w1.receivedEvents.get(1).getPath());
+        assertEquals(Watcher.Event.EventType.NodeDataChanged, w1.receivedEvents.get(1).getType());
+        assertFalse(latch2.await(2, TimeUnit.SECONDS));
+        assertEquals(1, w2.receivedEvents.size());
+    }
+
+    @Test(timeout = 60000)
+    public void testExceptionOnWatchers() throws Exception {
+        TestWatcher w1 = new TestWatcher();
+        TestWatcher w2 = new TestWatcher();
+
+        final CountDownLatch latch = new CountDownLatch(2);
+        w1.setLatch(latch);
+        w2.setLatch(latch);
+
+        zkc.register(w1);
+        zkc.register(w2);
+        // register bad watcher
+        zkc.register(new Watcher() {
+            @Override
+            public void process(WatchedEvent event) {
+                throw new NullPointerException("bad watcher returning null");
+            }
+        });
+
+        assertEquals(3, zkc.watchers.size());
+
+        final String zkPath = "/test-exception-on-watchers";
+
+        zkc.get().create(zkPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        zkc.get().getData(zkPath, true, new Stat());
+
+        zkc.get().setData(zkPath, "first-set".getBytes(), -1);
+        latch.await();
+        assertEquals(1, w1.receivedEvents.size());
+        assertEquals(zkPath, w1.receivedEvents.get(0).getPath());
+        assertEquals(Watcher.Event.EventType.NodeDataChanged, w1.receivedEvents.get(0).getType());
+        assertEquals(1, w2.receivedEvents.size());
+        assertEquals(zkPath, w2.receivedEvents.get(0).getPath());
+        assertEquals(Watcher.Event.EventType.NodeDataChanged, w2.receivedEvents.get(0).getType());
+    }
+
+    @Test(timeout = 60000)
+    public void testZooKeeperReconnection() throws Exception {
+        int sessionTimeoutMs = 100;
+        ZooKeeperClient zkc = clientBuilder(sessionTimeoutMs).zkAclId(null).build();
+        ZooKeeper zk = zkc.get();
+        long sessionId = zk.getSessionId();
+        ZooKeeperClientUtils.expireSession(zkc, zkServers, 2 * sessionTimeoutMs);
+        ZooKeeper newZk = zkc.get();
+        while (!ZooKeeper.States.CONNECTED.equals(newZk.getState())) {
+            TimeUnit.MILLISECONDS.sleep(sessionTimeoutMs / 2);
+        }
+        long newSessionId = newZk.getSessionId();
+        assertTrue(newZk == zk);
+        assertFalse(sessionId == newSessionId);
+    }
+
+    @Test(timeout = 60000)
+    public void testZooKeeperReconnectionBlockingRetryThread() throws Exception {
+        int sessionTimeoutMs = 100;
+        ZooKeeperClient zkc = clientBuilder(sessionTimeoutMs).zkAclId(null).build();
+        ZooKeeper zk = zkc.get();
+        assertTrue(zk instanceof org.apache.bookkeeper.zookeeper.ZooKeeperClient);
+        org.apache.bookkeeper.zookeeper.ZooKeeperClient bkZkc =
+                (org.apache.bookkeeper.zookeeper.ZooKeeperClient) zk;
+        // get the connect executor
+        Field connectExecutorField = bkZkc.getClass().getDeclaredField("connectExecutor");
+        connectExecutorField.setAccessible(true);
+        ExecutorService connectExecutor = (ExecutorService) connectExecutorField.get(bkZkc);
+        final CountDownLatch latch = new CountDownLatch(1);
+        // block retry thread in the zookeeper client
+        connectExecutor.submit(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    latch.await();
+                } catch (InterruptedException e) {
+                }
+            }
+        });
+        ZooKeeperClientUtils.expireSession(zkc, zkServers, 2 * sessionTimeoutMs);
+        ZooKeeper newZk;
+        while ((newZk = zkc.get()) == zk) {
+            TimeUnit.MILLISECONDS.sleep(sessionTimeoutMs / 2);
+        }
+        assertEquals(ZooKeeper.States.CONNECTED, newZk.getState());
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClientBuilder.java b/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClientBuilder.java
new file mode 100644
index 0000000..9fa0279
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/TestZooKeeperClientBuilder.java
@@ -0,0 +1,57 @@
+/**
+ * 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 org.apache.distributedlog.util.RetryPolicyUtils;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+
+/**
+ * The zookeeper client builder used for testing.
+ */
+public class TestZooKeeperClientBuilder {
+
+    /**
+     * Return a zookeeper client builder for testing.
+     *
+     * @return a zookeeper client builder
+     */
+    public static ZooKeeperClientBuilder newBuilder() {
+        return ZooKeeperClientBuilder.newBuilder()
+                .retryPolicy(RetryPolicyUtils.DEFAULT_INFINITE_RETRY_POLICY)
+                .connectionTimeoutMs(10000)
+                .sessionTimeoutMs(60000)
+                .zkAclId(null)
+                .statsLogger(NullStatsLogger.INSTANCE);
+    }
+
+    /**
+     * Create a zookeeper client builder with provided <i>conf</i> for testing.
+     *
+     * @param conf distributedlog configuration
+     * @return zookeeper client builder
+     */
+    public static ZooKeeperClientBuilder newBuilder(DistributedLogConfiguration conf) {
+        return ZooKeeperClientBuilder.newBuilder()
+                .retryPolicy(RetryPolicyUtils.DEFAULT_INFINITE_RETRY_POLICY)
+                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                .zkAclId(conf.getZkAclId())
+                .retryThreadCount(conf.getZKClientNumberRetryThreads())
+                .requestRateLimit(conf.getZKRequestRateLimit())
+                .statsLogger(NullStatsLogger.INSTANCE);
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.java
new file mode 100644
index 0000000..cafdcd6
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClientUtils.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;
+
+import com.google.common.base.Stopwatch;
+
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.*;
+
+/**
+ * Utilities of {@link org.apache.distributedlog.ZooKeeperClient}
+ */
+public class ZooKeeperClientUtils {
+
+    static final Logger logger = LoggerFactory.getLogger(ZooKeeperClientUtils.class);
+
+    /**
+     * Expire given zookeeper client's session.
+     *
+     * @param zkc
+     *          zookeeper client
+     * @param zkServers
+     *          zookeeper servers
+     * @param timeout
+     *          timeout
+     * @throws Exception
+     */
+    public static void expireSession(ZooKeeperClient zkc, String zkServers, int timeout)
+            throws Exception {
+        final CountDownLatch expireLatch = new CountDownLatch(1);
+        final CountDownLatch latch = new CountDownLatch(1);
+        ZooKeeper oldZk = zkc.get();
+        oldZk.exists("/", new Watcher() {
+            @Override
+            public void process(WatchedEvent event) {
+                logger.debug("Receive event : {}", event);
+                if (event.getType() == Event.EventType.None &&
+                        event.getState() == Event.KeeperState.Expired) {
+                    expireLatch.countDown();
+                }
+            }
+        });
+        ZooKeeper newZk = new ZooKeeper(zkServers, timeout, new Watcher() {
+            @Override
+            public void process(WatchedEvent event) {
+                if (Event.EventType.None == event.getType() &&
+                        Event.KeeperState.SyncConnected == event.getState()) {
+                    latch.countDown();
+                }
+            }
+        }, oldZk.getSessionId(), oldZk.getSessionPasswd());
+        if (!latch.await(timeout, TimeUnit.MILLISECONDS)) {
+            throw KeeperException.create(KeeperException.Code.CONNECTIONLOSS);
+        }
+        newZk.close();
+
+        boolean done = false;
+        Stopwatch expireWait = Stopwatch.createStarted();
+        while (!done && expireWait.elapsed(TimeUnit.MILLISECONDS) < timeout*2) {
+            try {
+                zkc.get().exists("/", false);
+                done = true;
+            } catch (KeeperException ke) {
+                done = (ke.code() == KeeperException.Code.SESSIONEXPIRED);
+            }
+        }
+
+        assertTrue("Client should receive session expired event.",
+                   expireLatch.await(timeout, TimeUnit.MILLISECONDS));
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClusterTestCase.java b/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClusterTestCase.java
new file mode 100644
index 0000000..3eeb870
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/ZooKeeperClusterTestCase.java
@@ -0,0 +1,53 @@
+/**
+ * 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 org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
+import org.apache.bookkeeper.util.IOUtils;
+import org.apache.bookkeeper.util.LocalBookKeeper;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+
+import java.io.File;
+
+public class ZooKeeperClusterTestCase {
+
+    protected static File zkDir;
+    protected static ZooKeeperServerShim zks;
+    protected static String zkServers;
+    protected static int zkPort;
+
+    @BeforeClass
+    public static void setupZooKeeper() throws Exception {
+        zkDir = IOUtils.createTempDir("zookeeper", ZooKeeperClusterTestCase.class.getName());
+        Pair<ZooKeeperServerShim, Integer> serverAndPort = LocalDLMEmulator.runZookeeperOnAnyPort(zkDir);
+        zks = serverAndPort.getLeft();
+        zkPort = serverAndPort.getRight();
+        zkServers = "127.0.0.1:" + zkPort;
+    }
+
+    @AfterClass
+    public static void shutdownZooKeeper() throws Exception {
+        zks.stop();
+        if (null != zkDir) {
+            FileUtils.deleteDirectory(zkDir);
+        }
+    }
+}
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
new file mode 100644
index 0000000..8d88a37
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControl.java
@@ -0,0 +1,154 @@
+/**
+ * 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.acl;
+
+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.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.*;
+
+public class TestZKAccessControl extends ZooKeeperClusterTestCase {
+
+    private ZooKeeperClient zkc;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .uri(createURI("/"))
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        zkc.close();
+    }
+
+    private URI createURI(String path) {
+        return URI.create("distributedlog://127.0.0.1:" + zkPort + path);
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateZKAccessControl() throws Exception {
+        AccessControlEntry ace = new AccessControlEntry();
+        ace.setDenyWrite(true);
+        String zkPath = "/create-zk-access-control";
+        ZKAccessControl zkac = new ZKAccessControl(ace, zkPath);
+        Await.result(zkac.create(zkc));
+
+        ZKAccessControl readZKAC = Await.result(ZKAccessControl.read(zkc, zkPath, null));
+        assertEquals(zkac, readZKAC);
+
+        ZKAccessControl another = new ZKAccessControl(ace, zkPath);
+        try {
+            Await.result(another.create(zkc));
+        } catch (KeeperException.NodeExistsException ke) {
+            // expected
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testDeleteZKAccessControl() throws Exception {
+        String zkPath = "/delete-zk-access-control";
+
+        AccessControlEntry ace = new AccessControlEntry();
+        ace.setDenyDelete(true);
+
+        ZKAccessControl zkac = new ZKAccessControl(ace, zkPath);
+        Await.result(zkac.create(zkc));
+
+        ZKAccessControl readZKAC = Await.result(ZKAccessControl.read(zkc, zkPath, null));
+        assertEquals(zkac, readZKAC);
+
+        Await.result(ZKAccessControl.delete(zkc, zkPath));
+
+        try {
+            Await.result(ZKAccessControl.read(zkc, zkPath, null));
+        } catch (KeeperException.NoNodeException nne) {
+            // expected.
+        }
+        Await.result(ZKAccessControl.delete(zkc, zkPath));
+    }
+
+    @Test(timeout = 60000)
+    public void testEmptyZKAccessControl() throws Exception {
+        String zkPath = "/empty-access-control";
+
+        zkc.get().create(zkPath, new byte[0], zkc.getDefaultACL(), CreateMode.PERSISTENT);
+
+        ZKAccessControl readZKAC = Await.result(ZKAccessControl.read(zkc, zkPath, null));
+
+        assertEquals(zkPath, readZKAC.getZKPath());
+        assertEquals(ZKAccessControl.DEFAULT_ACCESS_CONTROL_ENTRY, readZKAC.getAccessControlEntry());
+        assertTrue(ZKAccessControl.DEFAULT_ACCESS_CONTROL_ENTRY == readZKAC.getAccessControlEntry());
+    }
+
+    @Test(timeout = 60000)
+    public void testCorruptedZKAccessControl() throws Exception {
+        String zkPath = "/corrupted-zk-access-control";
+
+        zkc.get().create(zkPath, "corrupted-data".getBytes(UTF_8), zkc.getDefaultACL(), CreateMode.PERSISTENT);
+
+        try {
+            Await.result(ZKAccessControl.read(zkc, zkPath, null));
+        } catch (ZKAccessControl.CorruptedAccessControlException cace) {
+            // expected
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testUpdateZKAccessControl() throws Exception {
+        String zkPath = "/update-zk-access-control";
+
+        AccessControlEntry ace = new AccessControlEntry();
+        ace.setDenyDelete(true);
+
+        ZKAccessControl zkac = new ZKAccessControl(ace, zkPath);
+        Await.result(zkac.create(zkc));
+
+        ZKAccessControl readZKAC = Await.result(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));
+        assertEquals(newZKAC, readZKAC2);
+
+        try {
+            Await.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));
+        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
new file mode 100644
index 0000000..19c301b
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/acl/TestZKAccessControlManager.java
@@ -0,0 +1,178 @@
+/**
+ * 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.acl;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.TestZooKeeperClientBuilder;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientUtils;
+import org.apache.distributedlog.ZooKeeperClusterTestCase;
+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.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+
+import static org.junit.Assert.*;
+
+public class TestZKAccessControlManager extends ZooKeeperClusterTestCase {
+
+    static final Logger logger = LoggerFactory.getLogger(TestZKAccessControlManager.class);
+
+    private DistributedLogConfiguration conf;
+    private ZooKeeperClient zkc;
+    private ScheduledExecutorService executorService;
+
+    private URI createURI(String path) {
+        return URI.create("distributedlog://127.0.0.1:" + zkPort + path);
+    }
+
+    @Before
+    public void setup() throws Exception {
+        executorService = Executors.newSingleThreadScheduledExecutor();
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .uri(createURI("/"))
+                .build();
+        conf = new DistributedLogConfiguration();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        zkc.close();
+        executorService.shutdown();
+    }
+
+    void setACL(ZKAccessControl accessControl) throws Exception {
+        String zkPath = accessControl.getZKPath();
+        if (null == zkc.get().exists(zkPath, false)) {
+            accessControl.create(zkc);
+        } else {
+            accessControl.update(zkc);
+        }
+    }
+
+    static void verifyStreamPermissions(ZKAccessControlManager zkcm,
+                                        String stream,
+                                        boolean allowWrite,
+                                        boolean allowTruncate,
+                                        boolean allowRelease,
+                                        boolean allowDelete,
+                                        boolean allowAcquire) throws Exception {
+        assertEquals(allowWrite, zkcm.allowWrite(stream));
+        assertEquals(allowTruncate, zkcm.allowTruncate(stream));
+        assertEquals(allowRelease, zkcm.allowRelease(stream));
+        assertEquals(allowDelete, zkcm.allowDelete(stream));
+        assertEquals(allowAcquire, zkcm.allowAcquire(stream));
+    }
+
+    @Test(timeout = 60000)
+    public void testZKAccessControlManager() throws Exception {
+        String zkRootPath = "/test-zk-access-control-manager";
+        String stream1 = "test-acm-1";
+        String stream2 = "test-acm-2";
+        logger.info("Creating ACL Manager for {}", zkRootPath);
+        ZKAccessControlManager zkcm = new ZKAccessControlManager(conf, zkc, zkRootPath, executorService);
+        logger.info("Created ACL Manager for {}", zkRootPath);
+        try {
+            verifyStreamPermissions(zkcm, stream1, true, true, true, true, true);
+
+            // create stream1 (denyDelete = true)
+            String zkPath1 = zkRootPath + "/" + stream1;
+            AccessControlEntry ace1 = new AccessControlEntry();
+            ace1.setDenyDelete(true);
+            ZKAccessControl accessControl1 = new ZKAccessControl(ace1, zkPath1);
+            setACL(accessControl1);
+            logger.info("Create ACL for stream {} : {}", stream1, accessControl1);
+            while (zkcm.allowDelete(stream1)) {
+                Thread.sleep(100);
+            }
+            verifyStreamPermissions(zkcm, stream1, true, true, true, false, true);
+
+            // update stream1 (denyDelete = false, denyWrite = true)
+            ace1 = new AccessControlEntry();
+            ace1.setDenyWrite(true);
+            accessControl1 = new ZKAccessControl(ace1, zkPath1);
+            setACL(accessControl1);
+            logger.info("Update ACL for stream {} : {}", stream1, accessControl1);
+
+            // create stream2 (denyTruncate = true)
+            String zkPath2 = zkRootPath + "/" + stream2;
+            AccessControlEntry ace2 = new AccessControlEntry();
+            ace2.setDenyTruncate(true);
+            ZKAccessControl accessControl2 = new ZKAccessControl(ace2, zkPath2);
+            setACL(accessControl2);
+            logger.info("Create ACL for stream {} : {}", stream2, accessControl2);
+            while (zkcm.allowWrite(stream1)) {
+                Thread.sleep(100);
+            }
+            while (zkcm.allowTruncate(stream2)) {
+                Thread.sleep(100);
+            }
+
+            verifyStreamPermissions(zkcm, stream1, false, true, true, true, true);
+            verifyStreamPermissions(zkcm, stream2, true, false, true, true, true);
+
+            // delete stream2
+            Await.result(ZKAccessControl.delete(zkc, zkPath2));
+            logger.info("Delete ACL for stream {}", stream2);
+            while (!zkcm.allowTruncate(stream2)) {
+                Thread.sleep(100);
+            }
+
+            verifyStreamPermissions(zkcm, stream1, false, true, true, true, true);
+            verifyStreamPermissions(zkcm, stream2, true, true, true, true, true);
+
+            // expire session
+            ZooKeeperClientUtils.expireSession(zkc, zkServers, 1000);
+
+            // update stream1 (denyDelete = false, denyWrite = true)
+            ace1 = new AccessControlEntry();
+            ace1.setDenyRelease(true);
+            accessControl1 = new ZKAccessControl(ace1, zkPath1);
+            setACL(accessControl1);
+            logger.info("Update ACL for stream {} : {}", stream1, accessControl1);
+
+            // create stream2 (denyTruncate = true)
+            ace2 = new AccessControlEntry();
+            ace2.setDenyAcquire(true);
+            accessControl2 = new ZKAccessControl(ace2, zkPath2);
+            setACL(accessControl2);
+            logger.info("Created ACL for stream {} again : {}", stream2, accessControl2);
+
+            while (zkcm.allowRelease(stream1)) {
+                Thread.sleep(100);
+            }
+            while (zkcm.allowAcquire(stream2)) {
+                Thread.sleep(100);
+            }
+
+            verifyStreamPermissions(zkcm, stream1, true, true, false, true, true);
+            verifyStreamPermissions(zkcm, stream2, true, true, true, true, false);
+        } finally {
+            zkcm.close();
+        }
+    }
+}
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
new file mode 100644
index 0000000..4f968b6
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDLCK.java
@@ -0,0 +1,164 @@
+/**
+ * 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.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.LogSegmentMetadata;
+import org.apache.distributedlog.TestDistributedLogBase;
+import org.apache.distributedlog.TestZooKeeperClientBuilder;
+import org.apache.distributedlog.ZooKeeperClient;
+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.util.OrderedScheduler;
+import org.apache.distributedlog.util.SchedulerUtils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+public class TestDLCK extends TestDistributedLogBase {
+
+    static final Logger LOG = LoggerFactory.getLogger(TestDLCK.class);
+
+    protected static DistributedLogConfiguration conf =
+            new DistributedLogConfiguration().setLockTimeout(10)
+                .setEnableLedgerAllocatorPool(true).setLedgerAllocatorPoolName("test");
+
+    private ZooKeeperClient zkc;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = TestZooKeeperClientBuilder
+            .newBuilder()
+            .uri(createDLMURI("/"))
+            .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        zkc.close();
+    }
+
+    static Map<Long, LogSegmentMetadata> getLogSegments(DistributedLogManager dlm) throws Exception {
+        Map<Long, LogSegmentMetadata> logSegmentMap =
+                new HashMap<Long, LogSegmentMetadata>();
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        for (LogSegmentMetadata segment : segments) {
+            logSegmentMap.put(segment.getLogSegmentSequenceNumber(), segment);
+        }
+        return logSegmentMap;
+    }
+
+    static void verifyLogSegment(Map<Long, LogSegmentMetadata> segments,
+                                 DLSN lastDLSN, long logSegmentSequenceNumber,
+                                 int recordCount, long lastTxId) {
+        LogSegmentMetadata segment = segments.get(logSegmentSequenceNumber);
+        assertNotNull(segment);
+        assertEquals(lastDLSN, segment.getLastDLSN());
+        assertEquals(recordCount, segment.getRecordCount());
+        assertEquals(lastTxId, segment.getLastTxId());
+    }
+
+    @Test(timeout = 60000)
+    @SuppressWarnings("deprecation")
+    public void testCheckAndRepairDLNamespace() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.loadConf(conf);
+        confLocal.setImmediateFlushEnabled(true);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setLogSegmentSequenceNumberValidationEnabled(false);
+        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()
+                .conf(confLocal)
+                .uri(uri)
+                .build();
+        OrderedScheduler scheduler = OrderedScheduler.newBuilder()
+                .name("dlck-tool")
+                .corePoolSize(1)
+                .build();
+        ExecutorService executorService = Executors.newCachedThreadPool();
+
+        String streamName = "check-and-repair-dl-namespace";
+
+        // Create completed log segments
+        DistributedLogManager dlm = namespace.openLog(streamName);
+        DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 1L, 1L, 10, false);
+        DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 2L, 11L, 10, true);
+        DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 3L, 21L, 10, false);
+        DLMTestUtil.injectLogSegmentWithLastDLSN(dlm, confLocal, 4L, 31L, 10, true);
+
+        // dryrun
+        DistributedLogAdmin.checkAndRepairDLNamespace(
+                uri,
+                namespace,
+                new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(namespace)),
+                scheduler,
+                false,
+                false);
+
+        Map<Long, LogSegmentMetadata> segments = getLogSegments(dlm);
+        LOG.info("segments after drynrun {}", segments);
+        verifyLogSegment(segments, new DLSN(1L, 18L, 0L), 1L, 10, 10L);
+        verifyLogSegment(segments, new DLSN(2L, 16L, 0L), 2L, 9, 19L);
+        verifyLogSegment(segments, new DLSN(3L, 18L, 0L), 3L, 10, 30L);
+        verifyLogSegment(segments, new DLSN(4L, 16L, 0L), 4L, 9, 39L);
+
+        // check and repair
+        DistributedLogAdmin.checkAndRepairDLNamespace(
+                uri,
+                namespace,
+                LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(namespace)),
+                scheduler,
+                false,
+                false);
+
+        segments = getLogSegments(dlm);
+        LOG.info("segments after repair {}", segments);
+        verifyLogSegment(segments, new DLSN(1L, 18L, 0L), 1L, 10, 10L);
+        verifyLogSegment(segments, new DLSN(2L, 18L, 0L), 2L, 10, 20L);
+        verifyLogSegment(segments, new DLSN(3L, 18L, 0L), 3L, 10, 30L);
+        verifyLogSegment(segments, new DLSN(4L, 18L, 0L), 4L, 10, 40L);
+
+        dlm.close();
+        SchedulerUtils.shutdownScheduler(executorService, 5, TimeUnit.MINUTES);
+        namespace.close();
+    }
+
+}
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
new file mode 100644
index 0000000..f911f15
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/admin/TestDistributedLogAdmin.java
@@ -0,0 +1,194 @@
+/**
+ * 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.admin;
+
+import java.net.URI;
+import java.util.concurrent.TimeUnit;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.TestZooKeeperClientBuilder;
+import org.apache.distributedlog.annotations.DistributedLogAnnotations;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.distributedlog.util.Utils;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import org.apache.distributedlog.AsyncLogReader;
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.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.*;
+
+public class TestDistributedLogAdmin extends TestDistributedLogBase {
+
+    static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogAdmin.class);
+
+    private ZooKeeperClient zooKeeperClient;
+
+    @Before
+    public void setup() throws Exception {
+        zooKeeperClient = TestZooKeeperClientBuilder
+            .newBuilder()
+            .uri(createDLMURI("/"))
+            .build();
+        conf.setTraceReadAheadMetadataChanges(true);
+    }
+
+    @After
+    public void teardown() throws Exception {
+        zooKeeperClient.close();
+    }
+
+    /**
+     * {@link https://issues.apache.org/jira/browse/DL-44}
+     */
+    @DistributedLogAnnotations.FlakyTest
+    @Ignore
+    @Test(timeout = 60000)
+    @SuppressWarnings("deprecation")
+    public void testChangeSequenceNumber() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setLogSegmentSequenceNumberValidationEnabled(false);
+        confLocal.setLogSegmentCacheEnabled(false);
+
+        DistributedLogConfiguration readConf = new DistributedLogConfiguration();
+        readConf.addConfiguration(conf);
+        readConf.setLogSegmentCacheEnabled(false);
+        readConf.setLogSegmentSequenceNumberValidationEnabled(true);
+
+        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()
+                .conf(confLocal)
+                .uri(uri)
+                .build();
+        DistributedLogNamespace readNamespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(readConf)
+                .uri(uri)
+                .build();
+
+        String streamName = "change-sequence-number";
+
+        // create completed log segments
+        DistributedLogManager dlm = namespace.openLog(streamName);
+        DLMTestUtil.generateCompletedLogSegments(dlm, confLocal, 4, 10);
+        DLMTestUtil.injectLogSegmentWithGivenLogSegmentSeqNo(dlm, confLocal, 5, 41, false, 10, true);
+        dlm.close();
+
+        // create a reader
+        DistributedLogManager readDLM = readNamespace.openLog(streamName);
+        AsyncLogReader reader = readDLM.getAsyncLogReader(DLSN.InitialDLSN);
+
+        // read the records
+        long expectedTxId = 1L;
+        DLSN lastDLSN = DLSN.InitialDLSN;
+        for (int i = 0; i < 4 * 10; i++) {
+            LogRecordWithDLSN record = Await.result(reader.readNext());
+            assertNotNull(record);
+            DLMTestUtil.verifyLogRecord(record);
+            assertEquals(expectedTxId, record.getTransactionId());
+            expectedTxId++;
+            lastDLSN = record.getDlsn();
+        }
+
+        LOG.info("Injecting bad log segment '3'");
+
+        dlm = namespace.openLog(streamName);
+        DLMTestUtil.injectLogSegmentWithGivenLogSegmentSeqNo(dlm, confLocal, 3L, 5 * 10 + 1, true, 10, false);
+
+        LOG.info("Injected bad log segment '3'");
+
+        // there isn't records should be read
+        Future<LogRecordWithDLSN> readFuture = reader.readNext();
+        try {
+            LogRecordWithDLSN record = Await.result(readFuture);
+            fail("Should fail reading next record "
+                    + record
+                    + " when there is a corrupted log segment");
+        } catch (UnexpectedException ue) {
+            // expected
+        }
+
+        LOG.info("Dryrun fix inprogress segment that has lower sequence number");
+
+        // Dryrun
+        DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(namespace,
+                new DryrunLogSegmentMetadataStoreUpdater(confLocal, getLogSegmentMetadataStore(namespace)), streamName, false, false);
+
+        try {
+            reader = readDLM.getAsyncLogReader(lastDLSN);
+            Await.result(reader.readNext());
+            fail("Should fail reading next when there is a corrupted log segment");
+        } catch (UnexpectedException ue) {
+            // expected
+        }
+
+        LOG.info("Actual run fix inprogress segment that has lower sequence number");
+
+        // Actual run
+        DistributedLogAdmin.fixInprogressSegmentWithLowerSequenceNumber(namespace,
+                LogSegmentMetadataStoreUpdater.createMetadataUpdater(confLocal, getLogSegmentMetadataStore(namespace)), streamName, false, false);
+
+        // be able to read more after fix
+        reader = readDLM.getAsyncLogReader(lastDLSN);
+        // skip the first record
+        Await.result(reader.readNext());
+        readFuture = reader.readNext();
+
+        expectedTxId = 51L;
+        LogRecord record = Await.result(readFuture);
+        assertNotNull(record);
+        DLMTestUtil.verifyLogRecord(record);
+        assertEquals(expectedTxId, record.getTransactionId());
+        expectedTxId++;
+
+        for (int i = 1; i < 10; i++) {
+            record = Await.result(reader.readNext());
+            assertNotNull(record);
+            DLMTestUtil.verifyLogRecord(record);
+            assertEquals(expectedTxId, record.getTransactionId());
+            expectedTxId++;
+        }
+
+        Utils.close(reader);
+        readDLM.close();
+
+        dlm.close();
+        namespace.close();
+        readNamespace.close();
+    }
+}
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
new file mode 100644
index 0000000..2492c06
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocator.java
@@ -0,0 +1,389 @@
+/**
+ * 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.bk;
+
+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.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;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.Stat;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.Enumeration;
+import java.util.HashSet;
+import java.util.Set;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestLedgerAllocator extends TestDistributedLogBase {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestLedgerAllocator.class);
+
+    private static final String ledgersPath = "/ledgers";
+    private static final OpListener<LedgerHandle> NULL_LISTENER = new OpListener<LedgerHandle>() {
+        @Override
+        public void onCommit(LedgerHandle r) {
+            // no-op
+        }
+
+        @Override
+        public void onAbort(Throwable t) {
+            // no-op
+        }
+    };
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    private ZooKeeperClient zkc;
+    private BookKeeperClient bkc;
+    private DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
+
+    private URI createURI(String path) {
+        return URI.create("distributedlog://" + zkServers + path);
+    }
+
+    @Before
+    public void setup() throws Exception {
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .uri(createURI("/"))
+                .zkServers(zkServers)
+                .build();
+        bkc = BookKeeperClientBuilder.newBuilder().name("bkc")
+                .dlConfig(dlConf).ledgersPath(ledgersPath).zkc(zkc).build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        bkc.close();
+        zkc.close();
+    }
+
+    private QuorumConfigProvider newQuorumConfigProvider(DistributedLogConfiguration conf) {
+        return new ImmutableQuorumConfigProvider(conf.getQuorumConfig());
+    }
+
+    private ZKTransaction newTxn() {
+        return new ZKTransaction(zkc);
+    }
+
+    private SimpleLedgerAllocator createAllocator(String allocationPath) throws IOException {
+        return createAllocator(allocationPath, dlConf);
+    }
+
+    private SimpleLedgerAllocator createAllocator(String allocationPath,
+                                                  DistributedLogConfiguration conf) throws IOException {
+        return FutureUtils.result(SimpleLedgerAllocator.of(allocationPath, null, newQuorumConfigProvider(conf), zkc, bkc));
+    }
+
+    /**
+     * {@link https://issues.apache.org/jira/browse/DL-43}
+     */
+    @DistributedLogAnnotations.FlakyTest
+    @Ignore
+    @Test(timeout = 60000)
+    public void testAllocation() throws Exception {
+        String allocationPath = "/allocation1";
+        SimpleLedgerAllocator allocator = createAllocator(allocationPath);
+        allocator.allocate();
+        ZKTransaction txn = newTxn();
+        LedgerHandle lh = FutureUtils.result(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());
+            fail("Should fail the transaction when setting unexisted path");
+        } catch (ZKException ke) {
+            // expected
+            logger.info("Should fail on executing transaction when setting unexisted path", ke);
+        }
+        data = zkc.get().getData(allocationPath, false, null);
+        assertEquals((Long) lh.getId(), Long.valueOf(new String(data, UTF_8)));
+
+        // 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));
+        assertEquals(lh.getId(), newLh.getId());
+        FutureUtils.result(txn.execute());
+        data = zkc.get().getData(allocationPath, false, null);
+        assertEquals(0, data.length);
+        Utils.close(allocator);
+    }
+
+    @Test(timeout = 60000)
+    public void testBadVersionOnTwoAllocators() throws Exception {
+        String allocationPath = "/allocation-bad-version";
+        zkc.get().create(allocationPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        Stat stat = new Stat();
+        byte[] data = zkc.get().getData(allocationPath, false, stat);
+        Versioned<byte[]> allocationData = new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
+
+        SimpleLedgerAllocator allocator1 =
+                new SimpleLedgerAllocator(allocationPath, allocationData, newQuorumConfigProvider(dlConf), zkc, bkc);
+        SimpleLedgerAllocator allocator2 =
+                new SimpleLedgerAllocator(allocationPath, allocationData, newQuorumConfigProvider(dlConf), zkc, bkc);
+        allocator1.allocate();
+        // wait until allocated
+        ZKTransaction txn1 = newTxn();
+        LedgerHandle lh = FutureUtils.result(allocator1.tryObtain(txn1, NULL_LISTENER));
+        allocator2.allocate();
+        ZKTransaction txn2 = newTxn();
+        try {
+            FutureUtils.result(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());
+        }
+        FutureUtils.result(txn1.execute());
+        Utils.close(allocator1);
+        Utils.close(allocator2);
+
+        long eid = lh.addEntry("hello world".getBytes());
+        lh.close();
+        LedgerHandle readLh = bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes());
+        Enumeration<LedgerEntry> entries = readLh.readEntries(eid, eid);
+        int i = 0;
+        while (entries.hasMoreElements()) {
+            LedgerEntry entry = entries.nextElement();
+            assertEquals("hello world", new String(entry.getEntry(), UTF_8));
+            ++i;
+        }
+        assertEquals(1, i);
+    }
+
+    @Test(timeout = 60000)
+    public void testAllocatorWithoutEnoughBookies() throws Exception {
+        String allocationPath = "/allocator-without-enough-bookies";
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setEnsembleSize(numBookies * 2);
+        confLocal.setWriteQuorumSize(numBookies * 2);
+
+        SimpleLedgerAllocator allocator1 = createAllocator(allocationPath, confLocal);
+        allocator1.allocate();
+        ZKTransaction txn1 = newTxn();
+
+        try {
+            FutureUtils.result(allocator1.tryObtain(txn1, NULL_LISTENER));
+            fail("Should fail allocating ledger if there aren't enough bookies");
+        } catch (AllocationException ioe) {
+            // expected
+            assertEquals(Phase.ERROR, ioe.getPhase());
+        }
+        byte[] data = zkc.get().getData(allocationPath, false, null);
+        assertEquals(0, data.length);
+    }
+
+    @Test(timeout = 60000)
+    public void testSuccessAllocatorShouldDeleteUnusedledger() throws Exception {
+        String allocationPath = "/allocation-delete-unused-ledger";
+        zkc.get().create(allocationPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        Stat stat = new Stat();
+        byte[] data = zkc.get().getData(allocationPath, false, stat);
+
+        Versioned<byte[]> allocationData = new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
+
+        SimpleLedgerAllocator allocator1 =
+                new SimpleLedgerAllocator(allocationPath, allocationData, newQuorumConfigProvider(dlConf), zkc, bkc);
+        allocator1.allocate();
+        // wait until allocated
+        ZKTransaction txn1 = newTxn();
+        LedgerHandle lh1 = FutureUtils.result(allocator1.tryObtain(txn1, NULL_LISTENER));
+
+        // Second allocator kicks in
+        stat = new Stat();
+        data = zkc.get().getData(allocationPath, false, stat);
+        allocationData = new Versioned<byte[]>(data, new ZkVersion(stat.getVersion()));
+        SimpleLedgerAllocator allocator2 =
+                new SimpleLedgerAllocator(allocationPath, allocationData, newQuorumConfigProvider(dlConf), zkc, bkc);
+        allocator2.allocate();
+        // wait until allocated
+        ZKTransaction txn2 = newTxn();
+        LedgerHandle lh2 = FutureUtils.result(allocator2.tryObtain(txn2, NULL_LISTENER));
+
+        // should fail to commit txn1 as version is changed by second allocator
+        try {
+            FutureUtils.result(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.close(allocator1);
+        Utils.close(allocator2);
+
+        // ledger handle should be deleted
+        try {
+            lh1.close();
+            fail("LedgerHandle allocated by allocator1 should be deleted.");
+        } catch (BKException bke) {
+            // as expected
+        }
+        try {
+            bkc.get().openLedger(lh1.getId(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes());
+            fail("LedgerHandle allocated by allocator1 should be deleted.");
+        } catch (BKException.BKNoSuchLedgerExistsException nslee) {
+            // as expected
+        }
+        long eid = lh2.addEntry("hello world".getBytes());
+        lh2.close();
+        LedgerHandle readLh = bkc.get().openLedger(lh2.getId(), BookKeeper.DigestType.CRC32, dlConf.getBKDigestPW().getBytes());
+        Enumeration<LedgerEntry> entries = readLh.readEntries(eid, eid);
+        int i = 0;
+        while (entries.hasMoreElements()) {
+            LedgerEntry entry = entries.nextElement();
+            assertEquals("hello world", new String(entry.getEntry(), UTF_8));
+            ++i;
+        }
+        assertEquals(1, i);
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseAllocatorDuringObtaining() throws Exception {
+        String allocationPath = "/allocation2";
+        SimpleLedgerAllocator allocator = createAllocator(allocationPath);
+        allocator.allocate();
+        ZKTransaction txn = newTxn();
+        // close during obtaining ledger.
+        LedgerHandle lh = FutureUtils.result(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)));
+        // the ledger is not deleted
+        bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32,
+                dlConf.getBKDigestPW().getBytes(UTF_8));
+    }
+
+    /**
+     * {@link https://issues.apache.org/jira/browse/DL-26}
+     */
+    @DistributedLogAnnotations.FlakyTest
+    @Ignore
+    @Test(timeout = 60000)
+    public void testCloseAllocatorAfterConfirm() throws Exception {
+        String allocationPath = "/allocation2";
+        SimpleLedgerAllocator allocator = createAllocator(allocationPath);
+        allocator.allocate();
+        ZKTransaction txn = newTxn();
+        // close during obtaining ledger.
+        LedgerHandle lh = FutureUtils.result(allocator.tryObtain(txn, NULL_LISTENER));
+        FutureUtils.result(txn.execute());
+        Utils.close(allocator);
+        byte[] data = zkc.get().getData(allocationPath, false, null);
+        assertEquals(0, data.length);
+        // the ledger is not deleted.
+        bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32,
+                dlConf.getBKDigestPW().getBytes(UTF_8));
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseAllocatorAfterAbort() throws Exception {
+        String allocationPath = "/allocation3";
+        SimpleLedgerAllocator allocator = createAllocator(allocationPath);
+        allocator.allocate();
+        ZKTransaction txn = newTxn();
+        // close during obtaining ledger.
+        LedgerHandle lh = FutureUtils.result(allocator.tryObtain(txn, NULL_LISTENER));
+        txn.addOp(DefaultZKOp.of(Op.setData("/unexistedpath", "data".getBytes(UTF_8), -1), null));
+        try {
+            FutureUtils.result(txn.execute());
+            fail("Should fail the transaction when setting unexisted path");
+        } catch (ZKException ke) {
+            // expected
+        }
+        Utils.close(allocator);
+        byte[] data = zkc.get().getData(allocationPath, false, null);
+        assertEquals((Long) lh.getId(), Long.valueOf(new String(data, UTF_8)));
+        // the ledger is not deleted.
+        bkc.get().openLedger(lh.getId(), BookKeeper.DigestType.CRC32,
+                dlConf.getBKDigestPW().getBytes(UTF_8));
+    }
+
+    @Test(timeout = 60000)
+    public void testConcurrentAllocation() throws Exception {
+        String allcationPath = "/" + runtime.getMethodName();
+        SimpleLedgerAllocator allocator = createAllocator(allcationPath);
+        allocator.allocate();
+        ZKTransaction txn1 = newTxn();
+        Future<LedgerHandle> obtainFuture1 = allocator.tryObtain(txn1, NULL_LISTENER);
+        ZKTransaction txn2 = newTxn();
+        Future<LedgerHandle> obtainFuture2 = allocator.tryObtain(txn2, NULL_LISTENER);
+        assertTrue(obtainFuture2.isDefined());
+        assertTrue(obtainFuture2.isThrow());
+        try {
+            FutureUtils.result(obtainFuture2);
+            fail("Should fail the concurrent obtain since there is already a transaction obtaining the ledger handle");
+        } catch (SimpleLedgerAllocator.ConcurrentObtainException cbe) {
+            // expected
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testObtainMultipleLedgers() throws Exception {
+        String allocationPath = "/" + runtime.getMethodName();
+        SimpleLedgerAllocator allocator = createAllocator(allocationPath);
+        int numLedgers = 10;
+        Set<LedgerHandle> allocatedLedgers = new HashSet<LedgerHandle>();
+        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());
+            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
new file mode 100644
index 0000000..e1aaa0b
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/bk/TestLedgerAllocatorPool.java
@@ -0,0 +1,311 @@
+/**
+ * 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.bk;
+
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.BookKeeperClient;
+import org.apache.distributedlog.BookKeeperClientBuilder;
+import org.apache.distributedlog.DistributedLogConfiguration;
+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;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.zookeeper.data.Stat;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.*;
+
+public class TestLedgerAllocatorPool extends TestDistributedLogBase {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestLedgerAllocatorPool.class);
+
+    private static final String ledgersPath = "/ledgers";
+    private static final OpListener<LedgerHandle> NULL_LISTENER = new OpListener<LedgerHandle>() {
+        @Override
+        public void onCommit(LedgerHandle r) {
+            // no-op
+        }
+
+        @Override
+        public void onAbort(Throwable t) {
+            // no-op
+        }
+    };
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    private ZooKeeperClient zkc;
+    private BookKeeperClient bkc;
+    private DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
+    private ScheduledExecutorService allocationExecutor;
+
+    private URI createURI(String path) {
+        return URI.create("distributedlog://" + zkServers + path);
+    }
+
+    @Before
+    public void setup() throws Exception {
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .uri(createURI("/"))
+                .build();
+        bkc = BookKeeperClientBuilder.newBuilder().name("bkc")
+                .dlConfig(dlConf).ledgersPath(ledgersPath).zkc(zkc).build();
+        allocationExecutor = Executors.newSingleThreadScheduledExecutor();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        bkc.close();
+        zkc.close();
+        allocationExecutor.shutdown();
+    }
+
+    private ZKTransaction newTxn() {
+        return new ZKTransaction(zkc);
+    }
+
+    private void validatePoolSize(LedgerAllocatorPool pool,
+                                  int pendingSize,
+                                  int allocatingSize,
+                                  int obtainingSize,
+                                  int rescueSize) {
+        assertEquals(pendingSize, pool.pendingListSize());
+        assertEquals(allocatingSize, pool.allocatingListSize());
+        assertEquals(obtainingSize, pool.obtainMapSize());
+        assertEquals(rescueSize, pool.rescueSize());
+    }
+
+    @Test(timeout = 60000)
+    public void testNonAvailableAllocator() throws Exception {
+        String allocationPath = "/nonAvailableAllocator";
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(dlConf);
+        confLocal.setEnsembleSize(2 * numBookies);
+        confLocal.setWriteQuorumSize(2 * numBookies);
+
+        int numAllocators = 3;
+        LedgerAllocatorPool pool =
+                new LedgerAllocatorPool(allocationPath, numAllocators, confLocal, zkc, bkc, allocationExecutor);
+        for (int i = 0; i < numAllocators; i++) {
+            try {
+                pool.allocate();
+                FutureUtils.result(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());
+            }
+        }
+        for (int i = 0; i < numAllocators; i++) {
+            try {
+                pool.allocate();
+                FutureUtils.result(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());
+            } catch (IOException ioe) {
+                // expected
+            }
+        }
+        Utils.close(pool);
+    }
+
+    @Test(timeout = 60000)
+    public void testRescueAllocators() throws Exception {
+        String allocationPath = "/rescueAllocators";
+
+        int numAllocators = 3;
+        LedgerAllocatorPool pool =
+                new LedgerAllocatorPool(allocationPath, numAllocators, dlConf, zkc, bkc, allocationExecutor);
+        List<ZKTransaction> pendingTxns = Lists.newArrayListWithExpectedSize(numAllocators);
+        List<String> allocatePaths = Lists.newArrayListWithExpectedSize(numAllocators);
+        for (int i = 0; i < numAllocators; i++) {
+            ZKTransaction txn = newTxn();
+            pool.allocate();
+            LedgerHandle lh = FutureUtils.result(pool.tryObtain(txn, NULL_LISTENER));
+
+            // get the corresponding ledger allocator
+            SimpleLedgerAllocator sla = pool.getLedgerAllocator(lh);
+            String slaPath = sla.allocatePath;
+
+            logger.info("Allocated ledger {} from path {}", lh.getId(), slaPath);
+
+            pendingTxns.add(txn);
+            allocatePaths.add(slaPath);
+        }
+
+        for (int i = 0; i < numAllocators; i++) {
+            ZKTransaction txn = pendingTxns.get(i);
+            String slaPath = allocatePaths.get(i);
+
+            // execute the transaction to confirm/abort obtain
+            FutureUtils.result(txn.execute());
+
+            // introduce error to individual ledger allocator
+            byte[] data = zkc.get().getData(slaPath, false, new Stat());
+            zkc.get().setData(slaPath, data, -1);
+        }
+        int numSuccess = 0;
+        Set<String> allocatedPathSet = new HashSet<String>();
+        while (numSuccess < 2 * numAllocators) {
+            try {
+                pool.allocate();
+                ZKTransaction txn = newTxn();
+                LedgerHandle lh = FutureUtils.result(pool.tryObtain(txn, NULL_LISTENER));
+
+                // get the corresponding ledger allocator
+                SimpleLedgerAllocator sla = pool.getLedgerAllocator(lh);
+                String slaPath = sla.allocatePath;
+
+                logger.info("Allocated ledger {} from path {}", lh.getId(), slaPath);
+                allocatedPathSet.add(slaPath);
+
+                FutureUtils.result(txn.execute());
+                ++numSuccess;
+            } catch (IOException ioe) {
+                // continue
+            }
+        }
+        assertEquals(2 * numAllocators, numSuccess);
+        assertEquals(numAllocators, allocatedPathSet.size());
+        Utils.close(pool);
+    }
+
+    @Test(timeout = 60000)
+    public void testAllocateWhenNoAllocator() throws Exception {
+        String allocationPath = "/allocateWhenNoAllocator";
+        LedgerAllocatorPool pool = new LedgerAllocatorPool(allocationPath, 0, dlConf, zkc, bkc, allocationExecutor);
+        try {
+            pool.allocate();
+            fail("Should fail to allocate ledger if there isn't allocator.");
+        } catch (SimpleLedgerAllocator.AllocationException ae) {
+            fail("Should fail to allocate ledger if there isn't allocator.");
+        } catch (IOException ioe) {
+            // expected
+        }
+        Utils.close(pool);
+    }
+
+    @Test(timeout = 60000)
+    public void testObtainWhenNoAllocator() throws Exception {
+        String allocationPath = "/obtainWhenNoAllocator";
+        LedgerAllocatorPool pool = new LedgerAllocatorPool(allocationPath, 0, dlConf, zkc, bkc, allocationExecutor);
+        ZKTransaction txn = newTxn();
+        try {
+            FutureUtils.result(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.");
+        } catch (IOException ioe) {
+            // expected.
+        }
+
+        Utils.close(pool);
+    }
+
+    @Test(timeout = 60000)
+    public void testAllocateMultipleLedgers() throws Exception {
+        String allocationPath = "/" + runtime.getMethodName();
+        int numAllocators = 5;
+        final LedgerAllocatorPool pool =
+                new LedgerAllocatorPool(allocationPath, numAllocators, dlConf, zkc, bkc, allocationExecutor);
+        int numLedgers = 20;
+        Set<LedgerHandle> allocatedLedgers = new HashSet<LedgerHandle>();
+        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());
+            allocatedLedgers.add(lh);
+        }
+        assertEquals(numLedgers, allocatedLedgers.size());
+    }
+
+    @Test(timeout = 60000)
+    public void testConcurrentAllocation() throws Exception {
+        final int numAllocators = 5;
+        String allocationPath = "/concurrentAllocation";
+        final LedgerAllocatorPool pool =
+                new LedgerAllocatorPool(allocationPath, numAllocators, dlConf, zkc, bkc, allocationExecutor);
+        final ConcurrentMap<Long, LedgerHandle> allocatedLedgers =
+                new ConcurrentHashMap<Long, LedgerHandle>();
+        final AtomicInteger numFailures = new AtomicInteger(0);
+        Thread[] allocationThreads = new Thread[numAllocators];
+        for (int i = 0; i < numAllocators; i++) {
+            final int tid = i;
+            allocationThreads[i] = new Thread() {
+
+                int numLedgers = 50;
+
+                @Override
+                public void run() {
+                    try {
+                        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());
+                            lh.close();
+                            allocatedLedgers.putIfAbsent(lh.getId(), lh);
+                            logger.info("[thread {}] allocate {}th ledger {}",
+                                    new Object[] { tid, i, lh.getId() });
+                        }
+                    } catch (Exception ioe) {
+                        numFailures.incrementAndGet();
+                    }
+                }
+            };
+        }
+
+        for (Thread t : allocationThreads) {
+            t.start();
+        }
+
+        for (Thread t : allocationThreads) {
+            t.join();
+        }
+
+        assertEquals(0, numFailures.get());
+        assertEquals(50 * numAllocators, allocatedLedgers.size());
+
+        Utils.close(pool);
+    }
+
+}
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
new file mode 100644
index 0000000..f371007
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/config/PropertiesWriter.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.config;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.util.Properties;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+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-core/src/test/java/org/apache/distributedlog/config/TestConcurrentBaseConfiguration.java
new file mode 100644
index 0000000..9563511
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestConcurrentBaseConfiguration.java
@@ -0,0 +1,46 @@
+/**
+ * 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.config;
+
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+public class TestConcurrentBaseConfiguration {
+    static final Logger LOG = LoggerFactory.getLogger(TestConcurrentBaseConfiguration.class);
+
+    @Test(timeout = 20000)
+    public void testBasicOperations() throws Exception {
+        ConcurrentBaseConfiguration conf = new ConcurrentBaseConfiguration();
+        conf.setProperty("prop1", "1");
+        assertEquals(1, conf.getInt("prop1"));
+        conf.setProperty("prop1", "2");
+        assertEquals(2, conf.getInt("prop1"));
+        conf.clearProperty("prop1");
+        assertEquals(null, conf.getInteger("prop1", null));
+        conf.setProperty("prop1", "1");
+        conf.setProperty("prop2", "2");
+        assertEquals(1, conf.getInt("prop1"));
+        assertEquals(2, conf.getInt("prop2"));
+        conf.clearProperty("prop1");
+        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-core/src/test/java/org/apache/distributedlog/config/TestConfigurationSubscription.java
new file mode 100644
index 0000000..8420a97
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestConfigurationSubscription.java
@@ -0,0 +1,171 @@
+/**
+ * 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.config;
+
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.commons.configuration.event.ConfigurationEvent;
+import org.apache.commons.configuration.event.ConfigurationListener;
+import org.jmock.lib.concurrent.DeterministicScheduler;
+import org.junit.Test;
+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
+ * get picked up.
+ */
+public class TestConfigurationSubscription {
+    static final Logger LOG = LoggerFactory.getLogger(TestConfigurationSubscription.class);
+
+    /**
+     * Give FileChangedReloadingStrategy some time to start reloading
+     * Make sure now!=lastChecked
+     * {@link org.apache.commons.configuration.reloading.FileChangedReloadingStrategy#reloadingRequired()}
+     */
+    private void ensureConfigReloaded() throws InterruptedException {
+        // sleep 1 ms so that System.currentTimeMillis() !=
+        // lastChecked (the time we construct FileChangedReloadingStrategy
+        Thread.sleep(1);
+    }
+
+    @Test(timeout = 60000)
+    public void testReloadConfiguration() throws Exception {
+        PropertiesWriter writer = new PropertiesWriter();
+        FileConfigurationBuilder builder = new PropertiesConfigurationBuilder(writer.getFile().toURI().toURL());
+        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
+        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() {
+            @Override
+            public void onReload(ConcurrentBaseConfiguration conf) {
+                confHolder.set(conf);
+            }
+        });
+        assertEquals(null, conf.getProperty("prop1"));
+
+        // add
+        writer.setProperty("prop1", "1");
+        writer.save();
+        // ensure the file change reloading event can be triggered
+        ensureConfigReloaded();
+        // reload the config
+        confSub.reload();
+        assertNotNull(confHolder.get());
+        assertTrue(conf == confHolder.get());
+        assertEquals("1", conf.getProperty("prop1"));
+    }
+
+    @Test(timeout = 60000)
+    public void testAddReloadBasicsConfig() throws Exception {
+        PropertiesWriter writer = new PropertiesWriter();
+        DeterministicScheduler mockScheduler = new DeterministicScheduler();
+        FileConfigurationBuilder builder = new PropertiesConfigurationBuilder(writer.getFile().toURI().toURL());
+        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
+        List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(builder);
+        ConfigurationSubscription confSub =
+                new ConfigurationSubscription(conf, fileConfigBuilders, mockScheduler, 100, TimeUnit.MILLISECONDS);
+        assertEquals(null, conf.getProperty("prop1"));
+
+        // add
+        writer.setProperty("prop1", "1");
+        writer.save();
+        // ensure the file change reloading event can be triggered
+        ensureConfigReloaded();
+        mockScheduler.tick(100, TimeUnit.MILLISECONDS);
+        assertEquals("1", conf.getProperty("prop1"));
+
+    }
+
+    @Test(timeout = 60000)
+    public void testInitialConfigLoad() throws Exception {
+        PropertiesWriter writer = new PropertiesWriter();
+        writer.setProperty("prop1", "1");
+        writer.setProperty("prop2", "abc");
+        writer.setProperty("prop3", "123.0");
+        writer.setProperty("prop4", "11132");
+        writer.setProperty("prop5", "true");
+        writer.save();
+
+        ScheduledExecutorService mockScheduler = new DeterministicScheduler();
+        FileConfigurationBuilder builder = new PropertiesConfigurationBuilder(writer.getFile().toURI().toURL());
+        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
+        List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(builder);
+        ConfigurationSubscription confSub =
+                new ConfigurationSubscription(conf, fileConfigBuilders, mockScheduler, 100, TimeUnit.MILLISECONDS);
+        assertEquals(1, conf.getInt("prop1"));
+        assertEquals("abc", conf.getString("prop2"));
+        assertEquals(123.0, conf.getFloat("prop3"), 0);
+        assertEquals(11132, conf.getInt("prop4"));
+        assertEquals(true, conf.getBoolean("prop5"));
+    }
+
+    @Test(timeout = 60000)
+    public void testExceptionInConfigLoad() throws Exception {
+        PropertiesWriter writer = new PropertiesWriter();
+        writer.setProperty("prop1", "1");
+        writer.save();
+
+        DeterministicScheduler mockScheduler = new DeterministicScheduler();
+        FileConfigurationBuilder builder = new PropertiesConfigurationBuilder(writer.getFile().toURI().toURL());
+        ConcurrentConstConfiguration conf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
+        List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(builder);
+        ConfigurationSubscription confSub =
+                new ConfigurationSubscription(conf, fileConfigBuilders, mockScheduler, 100, TimeUnit.MILLISECONDS);
+
+        final AtomicInteger count = new AtomicInteger(1);
+        conf.addConfigurationListener(new ConfigurationListener() {
+            @Override
+            public void configurationChanged(ConfigurationEvent event) {
+                LOG.info("config changed {}", event);
+                // Throw after so we actually see the update anyway.
+                if (!event.isBeforeUpdate()) {
+                    count.getAndIncrement();
+                    throw new RuntimeException("config listener threw and exception");
+                }
+            }
+        });
+
+        int i = 0;
+        int initial = 0;
+        while (count.get() == initial) {
+            writer.setProperty("prop1", Integer.toString(i++));
+            writer.save();
+            mockScheduler.tick(100, TimeUnit.MILLISECONDS);
+        }
+
+        initial = count.get();
+        while (count.get() == initial) {
+            writer.setProperty("prop1", Integer.toString(i++));
+            writer.save();
+            mockScheduler.tick(100, TimeUnit.MILLISECONDS);
+        }
+    }
+}
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
new file mode 100644
index 0000000..b5d6300
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicConfigurationFactory.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.config;
+
+import com.google.common.base.Objects;
+import com.google.common.base.Optional;
+
+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.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+public class TestDynamicConfigurationFactory {
+    static final Logger LOG = LoggerFactory.getLogger(TestDynamicConfigurationFactory.class);
+
+    private void waitForConfig(DynamicDistributedLogConfiguration conf, int value) throws Exception {
+        while (!Objects.equal(conf.getRetentionPeriodHours(), value)) {
+            Thread.sleep(100);
+        }
+    }
+
+    private DynamicConfigurationFactory getConfigFactory(File configFile) {
+        String streamConfigPath = configFile.getParent();
+        ScheduledExecutorService executorService = new ScheduledThreadPoolExecutor(1);
+        ConcurrentBaseConfiguration defaultConf = new ConcurrentConstConfiguration(new DistributedLogConfiguration());
+        return new DynamicConfigurationFactory(executorService, 100, TimeUnit.MILLISECONDS);
+    }
+
+    private String getNamePart(File configFile) {
+        String propsFilename = configFile.getName();
+        return propsFilename.substring(0, propsFilename.indexOf(".conf"));
+    }
+
+    @Test(timeout = 60000)
+    public void testGetDynamicConfigBasics() throws Exception {
+        PropertiesWriter writer = new PropertiesWriter();
+        DynamicConfigurationFactory factory = getConfigFactory(writer.getFile());
+        Optional<DynamicDistributedLogConfiguration> conf = factory.getDynamicConfiguration(writer.getFile().getPath());
+        assertEquals(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT, conf.get().getRetentionPeriodHours());
+        writer.setProperty(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS, "1");
+        writer.save();
+        waitForConfig(conf.get(), 1);
+        assertEquals(1, conf.get().getRetentionPeriodHours());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetDynamicConfigIsSingleton() throws Exception {
+        PropertiesWriter writer = new PropertiesWriter();
+        DynamicConfigurationFactory factory = getConfigFactory(writer.getFile());
+        String configPath = writer.getFile().getPath();
+        Optional<DynamicDistributedLogConfiguration> conf1 = factory.getDynamicConfiguration(configPath);
+        Optional<DynamicDistributedLogConfiguration> conf2 = factory.getDynamicConfiguration(configPath);
+        assertEquals(conf1, conf2);
+    }
+
+    /**
+     * If the file is missing, get-config should not fail, and the file should be picked up if its added.
+     * If the file is removed externally same should apply.
+     */
+    @Test(timeout = 60000)
+    public void testMissingConfig() throws Exception {
+        PropertiesWriter writer = new PropertiesWriter();
+        DynamicConfigurationFactory factory = getConfigFactory(writer.getFile());
+        Optional<DynamicDistributedLogConfiguration> conf = factory.getDynamicConfiguration(writer.getFile().getPath());
+        writer.setProperty(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS, "1");
+        writer.save();
+        waitForConfig(conf.get(), 1);
+        File configFile = writer.getFile();
+        configFile.delete();
+        Thread.sleep(1000);
+        PropertiesWriter writer2 = new PropertiesWriter(writer.getFile());
+        writer2.setProperty(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS, "2");
+        writer2.save();
+        waitForConfig(conf.get(), 2);
+    }
+}
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
new file mode 100644
index 0000000..c1ac98a
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/config/TestDynamicDistributedLogConfiguration.java
@@ -0,0 +1,261 @@
+/**
+ * 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.config;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+
+import org.apache.distributedlog.bk.QuorumConfig;
+import org.junit.Test;
+
+import static org.apache.distributedlog.DistributedLogConfiguration.*;
+import static org.junit.Assert.*;
+
+public class TestDynamicDistributedLogConfiguration {
+
+    @Test(timeout = 20000)
+    public void testDefaults() throws Exception {
+        // Default config defines retention period plus two other params, but eaves ack quorum unspecified
+        DistributedLogConfiguration underlyingConfig = new DistributedLogConfiguration();
+        underlyingConfig.setRetentionPeriodHours(99);
+        underlyingConfig.setProperty("rpsHardWriteLimit", 99);
+
+        ConcurrentConstConfiguration defaultConfig = new ConcurrentConstConfiguration(underlyingConfig);
+        DynamicDistributedLogConfiguration config = new DynamicDistributedLogConfiguration(defaultConfig);
+        assertEquals(99, config.getRetentionPeriodHours());
+        assertEquals(99, config.getRpsHardWriteLimit());
+        config.setProperty(DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS, 5);
+
+        // Config checks primary then secondary then const defaults
+        assertEquals(5, config.getRetentionPeriodHours());
+        assertEquals(99, config.getRpsHardWriteLimit());
+    }
+
+    @Test(timeout = 20000)
+    public void testGetRetentionPeriodHours() {
+        ConcurrentBaseConfiguration defaultConfig = new ConcurrentBaseConfiguration();
+        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(defaultConfig);
+        // get default value
+        assertEquals(BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT, dynConf.getRetentionPeriodHours());
+        // get value from old key of default config
+        defaultConfig.setProperty(BKDL_RETENTION_PERIOD_IN_HOURS_OLD, BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 1);
+        assertEquals(BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 1, dynConf.getRetentionPeriodHours());
+        // get value from new key of default config
+        defaultConfig.setProperty(BKDL_RETENTION_PERIOD_IN_HOURS, BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT + 2);
+        assertEquals(BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 2, dynConf.getRetentionPeriodHours());
+        // get value from old key of dynamic config
+        dynConf.setProperty(BKDL_RETENTION_PERIOD_IN_HOURS_OLD, BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 3);
+        assertEquals(BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 3, dynConf.getRetentionPeriodHours());
+        // get value from new key of default config
+        dynConf.setProperty(BKDL_RETENTION_PERIOD_IN_HOURS, BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 4);
+        assertEquals(BKDL_RETENTION_PERIOD_IN_HOURS_DEFAULT  + 4, dynConf.getRetentionPeriodHours());
+    }
+
+    @Test(timeout = 20000)
+    public void testGetOutputBufferSize() {
+        ConcurrentBaseConfiguration defaultConfig = new ConcurrentBaseConfiguration();
+        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(defaultConfig);
+        // get default value
+        assertEquals(BKDL_OUTPUT_BUFFER_SIZE_DEFAULT, dynConf.getOutputBufferSize());
+        // get value from old key of default config
+        defaultConfig.setProperty(BKDL_OUTPUT_BUFFER_SIZE_OLD, BKDL_OUTPUT_BUFFER_SIZE_DEFAULT + 1);
+        assertEquals(BKDL_OUTPUT_BUFFER_SIZE_DEFAULT  + 1, dynConf.getOutputBufferSize());
+        // get value from new key of default config
+        defaultConfig.setProperty(BKDL_OUTPUT_BUFFER_SIZE, BKDL_OUTPUT_BUFFER_SIZE_DEFAULT + 2);
+        assertEquals(BKDL_OUTPUT_BUFFER_SIZE_DEFAULT  + 2, dynConf.getOutputBufferSize());
+        // get value from old key of dynamic config
+        dynConf.setProperty(BKDL_OUTPUT_BUFFER_SIZE_OLD, BKDL_OUTPUT_BUFFER_SIZE_DEFAULT  + 3);
+        assertEquals(BKDL_OUTPUT_BUFFER_SIZE_DEFAULT  + 3, dynConf.getOutputBufferSize());
+        // get value from new key of default config
+        dynConf.setProperty(BKDL_OUTPUT_BUFFER_SIZE, BKDL_OUTPUT_BUFFER_SIZE_DEFAULT  + 4);
+        assertEquals(BKDL_OUTPUT_BUFFER_SIZE_DEFAULT  + 4, dynConf.getOutputBufferSize());
+    }
+
+    @Test(timeout = 20000)
+    public void testGetReadAheadBatchSize() {
+        ConcurrentBaseConfiguration defaultConfig = new ConcurrentBaseConfiguration();
+        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(defaultConfig);
+        // get default value
+        assertEquals(BKDL_READAHEAD_BATCHSIZE_DEFAULT, dynConf.getReadAheadBatchSize());
+        // get value from old key of default config
+        defaultConfig.setProperty(BKDL_READAHEAD_BATCHSIZE_OLD, BKDL_READAHEAD_BATCHSIZE_DEFAULT + 1);
+        assertEquals(BKDL_READAHEAD_BATCHSIZE_DEFAULT  + 1, dynConf.getReadAheadBatchSize());
+        // get value from new key of default config
+        defaultConfig.setProperty(BKDL_READAHEAD_BATCHSIZE, BKDL_READAHEAD_BATCHSIZE_DEFAULT + 2);
+        assertEquals(BKDL_READAHEAD_BATCHSIZE_DEFAULT  + 2, dynConf.getReadAheadBatchSize());
+        // get value from old key of dynamic config
+        dynConf.setProperty(BKDL_READAHEAD_BATCHSIZE_OLD, BKDL_READAHEAD_BATCHSIZE_DEFAULT  + 3);
+        assertEquals(BKDL_READAHEAD_BATCHSIZE_DEFAULT  + 3, dynConf.getReadAheadBatchSize());
+        // get value from new key of default config
+        dynConf.setProperty(BKDL_READAHEAD_BATCHSIZE, BKDL_READAHEAD_BATCHSIZE_DEFAULT  + 4);
+        assertEquals(BKDL_READAHEAD_BATCHSIZE_DEFAULT  + 4, dynConf.getReadAheadBatchSize());
+    }
+
+    @Test(timeout = 20000)
+    public void testGetReadAheadMaxRecords() {
+        ConcurrentBaseConfiguration defaultConfig = new ConcurrentBaseConfiguration();
+        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(defaultConfig);
+        // get default value
+        assertEquals(BKDL_READAHEAD_MAX_RECORDS_DEFAULT, dynConf.getReadAheadMaxRecords());
+        // get value from old key of default config
+        defaultConfig.setProperty(BKDL_READAHEAD_MAX_RECORDS_OLD, BKDL_READAHEAD_MAX_RECORDS_DEFAULT + 1);
+        assertEquals(BKDL_READAHEAD_MAX_RECORDS_DEFAULT  + 1, dynConf.getReadAheadMaxRecords());
+        // get value from new key of default config
+        defaultConfig.setProperty(BKDL_READAHEAD_MAX_RECORDS, BKDL_READAHEAD_MAX_RECORDS_DEFAULT + 2);
+        assertEquals(BKDL_READAHEAD_MAX_RECORDS_DEFAULT  + 2, dynConf.getReadAheadMaxRecords());
+        // get value from old key of dynamic config
+        dynConf.setProperty(BKDL_READAHEAD_MAX_RECORDS_OLD, BKDL_READAHEAD_MAX_RECORDS_DEFAULT  + 3);
+        assertEquals(BKDL_READAHEAD_MAX_RECORDS_DEFAULT  + 3, dynConf.getReadAheadMaxRecords());
+        // get value from new key of default config
+        dynConf.setProperty(BKDL_READAHEAD_MAX_RECORDS, BKDL_READAHEAD_MAX_RECORDS_DEFAULT  + 4);
+        assertEquals(BKDL_READAHEAD_MAX_RECORDS_DEFAULT  + 4, dynConf.getReadAheadMaxRecords());
+    }
+
+    void assertQuorumConfig(QuorumConfig config,
+                            int expectedEnsembleSize,
+                            int expectedWriteQuorumSize,
+                            int expectedAckQuorumSize) {
+        assertEquals(expectedEnsembleSize, config.getEnsembleSize());
+        assertEquals(expectedWriteQuorumSize, config.getWriteQuorumSize());
+        assertEquals(expectedAckQuorumSize, config.getAckQuorumSize());
+    }
+
+    @Test(timeout = 20000)
+    public void testGetQuorumConfig() {
+        ConcurrentBaseConfiguration defaultConfig = new ConcurrentBaseConfiguration();
+        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(defaultConfig);
+        // get default value
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
+
+        // Test Ensemble Size
+
+        // get value from old key of default config
+        defaultConfig.setProperty(BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD, BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 1);
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 1,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
+        // get value from new key of default config
+        defaultConfig.setProperty(BKDL_BOOKKEEPER_ENSEMBLE_SIZE, BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 2);
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 2,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
+        // get value from old key of dynamic config
+        dynConf.setProperty(BKDL_BOOKKEEPER_ENSEMBLE_SIZE_OLD, BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 3);
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 3,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
+        // get value from new key of dynamic config
+        dynConf.setProperty(BKDL_BOOKKEEPER_ENSEMBLE_SIZE, BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4);
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
+
+        // Test Write Quorum Size
+
+        // get value from old key of default config
+        defaultConfig.setProperty(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 1);
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 1,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
+        // get value from new key of default config
+        defaultConfig.setProperty(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 2);
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 2,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
+        // get value from old key of dynamic config
+        dynConf.setProperty(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_OLD,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 3);
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 3,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
+        // get value from new key of dynamic config
+        dynConf.setProperty(BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 4);
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT);
+
+        // Test Ack Quorum Size
+
+        // get value from old key of default config
+        defaultConfig.setProperty(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 1);
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 1);
+        // get value from new key of default config
+        defaultConfig.setProperty(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 2);
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 2);
+        // get value from old key of dynamic config
+        dynConf.setProperty(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_OLD,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 3);
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 3);
+        // get value from new key of dynamic config
+        dynConf.setProperty(BKDL_BOOKKEEPER_ACK_QUORUM_SIZE,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 4);
+        assertQuorumConfig(
+                dynConf.getQuorumConfig(),
+                BKDL_BOOKKEEPER_ENSEMBLE_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_WRITE_QUORUM_SIZE_DEFAULT + 4,
+                BKDL_BOOKKEEPER_ACK_QUORUM_SIZE_DEFAULT + 4);
+    }
+
+    @Test(timeout = 20000)
+    public void testIsDurableWriteEnabled() {
+        ConcurrentBaseConfiguration defaultConfig = new ConcurrentBaseConfiguration();
+        DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(defaultConfig);
+
+        assertTrue(dynConf.isDurableWriteEnabled());
+        defaultConfig.setProperty(BKDL_IS_DURABLE_WRITE_ENABLED, false);
+        assertFalse(dynConf.isDurableWriteEnabled());
+        dynConf.setProperty(BKDL_IS_DURABLE_WRITE_ENABLED, true);
+        assertTrue(dynConf.isDurableWriteEnabled());
+    }
+}
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
new file mode 100644
index 0000000..3ce4952
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestConfigurationFeatureProvider.java
@@ -0,0 +1,65 @@
+/**
+ * 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.feature;
+
+import org.apache.distributedlog.config.ConcurrentBaseConfiguration;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.SettableFeature;
+import org.junit.Test;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test case for configuration based feature provider
+ */
+public class TestConfigurationFeatureProvider {
+
+    @Test(timeout = 60000)
+    public void testConfigurationFeatureProvider() throws Exception {
+        String rootScope = "dl";
+        ConcurrentBaseConfiguration featureConf =
+                new ConcurrentBaseConfiguration();
+        ConcurrentMap<String, SettableFeature> features =
+                new ConcurrentHashMap<String, SettableFeature>();
+        ConfigurationFeatureProvider featureProvider =
+                new ConfigurationFeatureProvider(rootScope, featureConf, features);
+
+        String featureName1 = "feature1";
+        String fullFeatureName1 = rootScope + "." + featureName1;
+        int availability1 = 1234;
+        featureConf.setProperty(fullFeatureName1, availability1);
+        Feature feature1 = featureProvider.getFeature(featureName1);
+        assertEquals(availability1, feature1.availability());
+        assertTrue(features.containsKey(fullFeatureName1));
+        assertTrue(feature1 == features.get(fullFeatureName1));
+
+        String subScope = "subscope";
+        String featureName2 = "feature2";
+        String fullFeatureName2 = rootScope + "." + subScope + "." + featureName2;
+        int availability2 = 4321;
+        featureConf.setProperty(fullFeatureName2, availability2);
+        Feature feature2 = featureProvider.scope(subScope).getFeature(featureName2);
+        assertEquals(availability2, feature2.availability());
+        assertTrue(features.containsKey(fullFeatureName2));
+        assertTrue(feature2 == features.get(fullFeatureName2));
+    }
+
+}
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
new file mode 100644
index 0000000..5d4472d
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/feature/TestDynamicConfigurationFeatureProvider.java
@@ -0,0 +1,183 @@
+/**
+ * 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.feature;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.annotations.DistributedLogAnnotations;
+import org.apache.distributedlog.config.PropertiesWriter;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.junit.Ignore;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test case for dynamic configuration based feature provider
+ */
+public class TestDynamicConfigurationFeatureProvider {
+
+    /**
+     * Make sure config is reloaded
+     *
+     * Give FileChangedReloadingStrategy some time to allow reloading
+     * Make sure now!=lastChecked
+     * {@link org.apache.commons.configuration.reloading.FileChangedReloadingStrategy#reloadingRequired()}
+     */
+    private void ensureConfigReloaded() throws InterruptedException {
+        // sleep 1 ms so that System.currentTimeMillis() !=
+        // lastChecked (the time we construct FileChangedReloadingStrategy
+        Thread.sleep(1);
+    }
+
+    @Test(timeout = 60000)
+    public void testLoadFeaturesFromBase() throws Exception {
+        PropertiesWriter writer = new PropertiesWriter();
+        writer.setProperty("feature_1", "10000");
+        writer.setProperty("feature_2", "5000");
+        writer.save();
+
+        DistributedLogConfiguration conf = new DistributedLogConfiguration()
+                .setDynamicConfigReloadIntervalSec(Integer.MAX_VALUE)
+                .setFileFeatureProviderBaseConfigPath(writer.getFile().toURI().toURL().getPath());
+        DynamicConfigurationFeatureProvider provider =
+                new DynamicConfigurationFeatureProvider("", conf, NullStatsLogger.INSTANCE);
+        provider.start();
+        ensureConfigReloaded();
+
+        Feature feature1 = provider.getFeature("feature_1");
+        assertTrue(feature1.isAvailable());
+        assertEquals(10000, feature1.availability());
+        Feature feature2 = provider.getFeature("feature_2");
+        assertTrue(feature2.isAvailable());
+        assertEquals(5000, feature2.availability());
+        Feature feature3 = provider.getFeature("feature_3");
+        assertFalse(feature3.isAvailable());
+        assertEquals(0, feature3.availability());
+        Feature feature4 = provider.getFeature("unknown_feature");
+        assertFalse(feature4.isAvailable());
+        assertEquals(0, feature4.availability());
+
+        provider.stop();
+    }
+
+    /**
+     * {@link https://issues.apache.org/jira/browse/DL-40}
+     */
+    @DistributedLogAnnotations.FlakyTest
+    @Ignore
+    @Test(timeout = 60000)
+    public void testLoadFeaturesFromOverlay() throws Exception {
+        PropertiesWriter writer = new PropertiesWriter();
+        writer.setProperty("feature_1", "10000");
+        writer.setProperty("feature_2", "5000");
+        writer.save();
+
+        PropertiesWriter overlayWriter = new PropertiesWriter();
+        overlayWriter.setProperty("feature_2", "6000");
+        overlayWriter.setProperty("feature_4", "6000");
+        overlayWriter.save();
+
+        DistributedLogConfiguration conf = new DistributedLogConfiguration()
+                .setDynamicConfigReloadIntervalSec(Integer.MAX_VALUE)
+                .setFileFeatureProviderBaseConfigPath(writer.getFile().toURI().toURL().getPath())
+                .setFileFeatureProviderOverlayConfigPath(overlayWriter.getFile().toURI().toURL().getPath());
+        DynamicConfigurationFeatureProvider provider =
+                new DynamicConfigurationFeatureProvider("", conf, NullStatsLogger.INSTANCE);
+        provider.start();
+        ensureConfigReloaded();
+
+        Feature feature1 = provider.getFeature("feature_1");
+        assertTrue(feature1.isAvailable());
+        assertEquals(10000, feature1.availability());
+        Feature feature2 = provider.getFeature("feature_2");
+        assertTrue(feature2.isAvailable());
+        assertEquals(6000, feature2.availability());
+        Feature feature3 = provider.getFeature("feature_3");
+        assertFalse(feature3.isAvailable());
+        assertEquals(0, feature3.availability());
+        Feature feature4 = provider.getFeature("feature_4");
+        assertTrue(feature4.isAvailable());
+        assertEquals(6000, feature4.availability());
+        Feature feature5 = provider.getFeature("unknown_feature");
+        assertFalse(feature5.isAvailable());
+        assertEquals(0, feature5.availability());
+
+        provider.stop();
+    }
+
+    @Test(timeout = 60000)
+    public void testReloadFeaturesFromOverlay() throws Exception {
+        PropertiesWriter writer = new PropertiesWriter();
+        writer.setProperty("feature_1", "10000");
+        writer.setProperty("feature_2", "5000");
+        writer.save();
+
+        PropertiesWriter overlayWriter = new PropertiesWriter();
+        overlayWriter.setProperty("feature_2", "6000");
+        overlayWriter.setProperty("feature_4", "6000");
+        overlayWriter.save();
+
+        DistributedLogConfiguration conf = new DistributedLogConfiguration()
+                .setDynamicConfigReloadIntervalSec(Integer.MAX_VALUE)
+                .setFileFeatureProviderBaseConfigPath(writer.getFile().toURI().toURL().getPath())
+                .setFileFeatureProviderOverlayConfigPath(overlayWriter.getFile().toURI().toURL().getPath());
+        DynamicConfigurationFeatureProvider provider =
+                new DynamicConfigurationFeatureProvider("", conf, NullStatsLogger.INSTANCE);
+        provider.start();
+        ensureConfigReloaded();
+
+        Feature feature1 = provider.getFeature("feature_1");
+        assertTrue(feature1.isAvailable());
+        assertEquals(10000, feature1.availability());
+        Feature feature2 = provider.getFeature("feature_2");
+        assertTrue(feature2.isAvailable());
+        assertEquals(6000, feature2.availability());
+        Feature feature3 = provider.getFeature("feature_3");
+        assertFalse(feature3.isAvailable());
+        assertEquals(0, feature3.availability());
+        Feature feature4 = provider.getFeature("feature_4");
+        assertTrue(feature4.isAvailable());
+        assertEquals(6000, feature4.availability());
+        Feature feature5 = provider.getFeature("unknown_feature");
+        assertFalse(feature5.isAvailable());
+        assertEquals(0, feature5.availability());
+
+        // dynamic load config
+        provider.getFeatureConf().setProperty("feature_1", 3000);
+        provider.getFeatureConf().setProperty("feature_2", 7000);
+        provider.getFeatureConf().setProperty("feature_3", 8000);
+        provider.getFeatureConf().setProperty("feature_4", 9000);
+        provider.onReload(provider.getFeatureConf());
+        feature1 = provider.getFeature("feature_1");
+        assertTrue(feature1.isAvailable());
+        assertEquals(3000, feature1.availability());
+        feature2 = provider.getFeature("feature_2");
+        assertTrue(feature2.isAvailable());
+        assertEquals(7000, feature2.availability());
+        feature3 = provider.getFeature("feature_3");
+        assertTrue(feature3.isAvailable());
+        assertEquals(8000, feature3.availability());
+        feature4 = provider.getFeature("feature_4");
+        assertTrue(feature4.isAvailable());
+        assertEquals(9000, feature4.availability());
+
+        provider.stop();
+    }
+
+}
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
new file mode 100644
index 0000000..b2fcbf6
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogMetadataStore.java
@@ -0,0 +1,114 @@
+/**
+ * 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.impl;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.DistributedLogConfiguration;
+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;
+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 java.net.URI;
+import java.util.Set;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test ZK based metadata store.
+ */
+public class TestZKLogMetadataStore extends TestDistributedLogBase {
+
+    private final static int zkSessionTimeoutMs = 2000;
+
+    @Rule
+    public TestName runtime = new TestName();
+    protected final DistributedLogConfiguration baseConf =
+            new DistributedLogConfiguration();
+    protected ZooKeeperClient zkc;
+    protected ZKLogMetadataStore metadataStore;
+    protected OrderedScheduler scheduler;
+    protected URI uri;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .uri(createDLMURI("/"))
+                .sessionTimeoutMs(zkSessionTimeoutMs)
+                .build();
+        scheduler = OrderedScheduler.newBuilder()
+                .name("test-zk-logmetadata-store")
+                .corePoolSize(1)
+                .build();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.addConfiguration(baseConf);
+        this.uri = createDLMURI("/" + runtime.getMethodName());
+        metadataStore = new ZKLogMetadataStore(conf, uri, zkc, scheduler);
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (null != zkc) {
+            zkc.close();
+        }
+        if (null != scheduler) {
+            scheduler.shutdown();
+        }
+    }
+
+    private void createLogInNamespace(URI uri, String logName) throws Exception {
+        String logPath = uri.getPath() + "/" + logName;
+        Utils.zkCreateFullPathOptimistic(zkc, logPath, new byte[0],
+                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLog() throws Exception {
+        assertEquals(uri, FutureUtils.result(metadataStore.createLog("test")));
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogLocation() throws Exception {
+        Optional<URI> uriOptional = FutureUtils.result(metadataStore.getLogLocation("test"));
+        assertTrue(uriOptional.isPresent());
+        assertEquals(uri, uriOptional.get());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogs() throws Exception {
+        Set<String> logs = Sets.newHashSet();
+        for (int i = 0; i < 10; i++) {
+            String logName = "test-" + i;
+            logs.add(logName);
+            createLogInNamespace(uri, logName);
+        }
+        Set<String> result = Sets.newHashSet(FutureUtils.result(metadataStore.getLogs()));
+        assertEquals(10, result.size());
+        assertTrue(Sets.difference(logs, result).isEmpty());
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java
new file mode 100644
index 0000000..29a9add
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentFilters.java
@@ -0,0 +1,74 @@
+/**
+ * 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.impl;
+
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import static org.apache.distributedlog.impl.ZKLogSegmentFilters.*;
+import static org.junit.Assert.*;
+
+public class TestZKLogSegmentFilters {
+
+    static final Logger LOG = LoggerFactory.getLogger(TestZKLogSegmentFilters.class);
+
+    @Test(timeout = 60000)
+    public void testWriteFilter() {
+        Set<String> expectedFilteredSegments = new HashSet<String>();
+        List<String> segments = new ArrayList<String>();
+        for (int i = 1; i <= 5; i++) {
+            segments.add(DLMTestUtil.completedLedgerZNodeNameWithVersion(i, (i - 1) * 100, i * 100 - 1, i));
+        }
+        for (int i = 6; i <= 10; i++) {
+            String segmentName = DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i);
+            segments.add(segmentName);
+            if (i == 10) {
+                expectedFilteredSegments.add(segmentName);
+            }
+        }
+        for (int i = 11; i <= 15; i++) {
+            String segmentName = DLMTestUtil.completedLedgerZNodeNameWithTxID((i - 1) * 100, i * 100 - 1);
+            segments.add(segmentName);
+            expectedFilteredSegments.add(segmentName);
+        }
+        segments.add("");
+        segments.add("unknown");
+        segments.add(DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX + "_1234_5678_9");
+        expectedFilteredSegments.add(DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX + "_1234_5678_9");
+        segments.add(DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX + "_1_2_3_4_5_6_7_8_9");
+        expectedFilteredSegments.add(DistributedLogConstants.COMPLETED_LOGSEGMENT_PREFIX + "_1_2_3_4_5_6_7_8_9");
+
+        Collection<String> filteredCollection = WRITE_HANDLE_FILTER.filter(segments);
+        LOG.info("Filter log segments {} to {}.", segments, filteredCollection);
+        assertEquals(expectedFilteredSegments.size(), filteredCollection.size());
+
+        Set<String> filteredSegments = Sets.newHashSet(filteredCollection);
+        Sets.SetView<String> diff = Sets.difference(filteredSegments, expectedFilteredSegments);
+        assertTrue(diff.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
new file mode 100644
index 0000000..eb81900
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKLogSegmentMetadataStore.java
@@ -0,0 +1,831 @@
+/**
+ * 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.impl;
+
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.TestDistributedLogBase;
+import org.apache.distributedlog.TestZooKeeperClientBuilder;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientUtils;
+import org.apache.distributedlog.callback.LogSegmentNamesListener;
+import org.apache.distributedlog.exceptions.ZKException;
+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.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.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.data.Stat;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+/**
+ * Test ZK based log segment metadata store.
+ */
+public class TestZKLogSegmentMetadataStore extends TestDistributedLogBase {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestZKLogSegmentMetadataStore.class);
+
+    private final static int zkSessionTimeoutMs = 2000;
+
+    private LogSegmentMetadata createLogSegment(
+            long logSegmentSequenceNumber) {
+        return createLogSegment(logSegmentSequenceNumber, 99L);
+    }
+
+    private LogSegmentMetadata createLogSegment(
+        long logSegmentSequenceNumber,
+        long lastEntryId) {
+        return DLMTestUtil.completedLogSegment(
+                "/" + runtime.getMethodName(),
+                logSegmentSequenceNumber,
+                logSegmentSequenceNumber,
+                1L,
+                100,
+                logSegmentSequenceNumber,
+                lastEntryId,
+                0L,
+                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
+    }
+
+    @Rule
+    public TestName runtime = new TestName();
+    protected final DistributedLogConfiguration baseConf =
+            new DistributedLogConfiguration();
+    protected ZooKeeperClient zkc;
+    protected ZKLogSegmentMetadataStore lsmStore;
+    protected OrderedScheduler scheduler;
+    protected URI uri;
+    protected String rootZkPath;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .uri(createDLMURI("/"))
+                .sessionTimeoutMs(zkSessionTimeoutMs)
+                .build();
+        scheduler = OrderedScheduler.newBuilder()
+                .name("test-zk-logsegment-metadata-store")
+                .corePoolSize(1)
+                .build();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.addConfiguration(baseConf);
+        this.uri = createDLMURI("/" + runtime.getMethodName());
+        lsmStore = new ZKLogSegmentMetadataStore(conf, zkc, scheduler);
+        zkc.get().create(
+                "/" + runtime.getMethodName(),
+                new byte[0],
+                ZooDefs.Ids.OPEN_ACL_UNSAFE,
+                CreateMode.PERSISTENT);
+        this.rootZkPath = "/" + runtime.getMethodName();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (null != zkc) {
+            zkc.close();
+        }
+        if (null != scheduler) {
+            scheduler.shutdown();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogSegment() throws Exception {
+        LogSegmentMetadata segment = createLogSegment(1L);
+        Transaction<Object> createTxn = lsmStore.transaction();
+        lsmStore.createLogSegment(createTxn, segment, null);
+        FutureUtils.result(createTxn.execute());
+        // the log segment should be created
+        assertNotNull("LogSegment " + segment + " should be created",
+                zkc.get().exists(segment.getZkPath(), false));
+        LogSegmentMetadata segment2 = createLogSegment(1L);
+        Transaction<Object> createTxn2 = lsmStore.transaction();
+        lsmStore.createLogSegment(createTxn2, segment2, null);
+        try {
+            FutureUtils.result(createTxn2.execute());
+            fail("Should fail if log segment exists");
+        } catch (Throwable t) {
+            // expected
+            assertTrue("Should throw NodeExistsException if log segment exists",
+                    t instanceof ZKException);
+            ZKException zke = (ZKException) t;
+            assertEquals("Should throw NodeExistsException if log segment exists",
+                    KeeperException.Code.NODEEXISTS, zke.getKeeperExceptionCode());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testDeleteLogSegment() throws Exception {
+        LogSegmentMetadata segment = createLogSegment(1L);
+        Transaction<Object> createTxn = lsmStore.transaction();
+        lsmStore.createLogSegment(createTxn, segment, null);
+        FutureUtils.result(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());
+        assertNull("LogSegment " + segment + " should be deleted",
+                zkc.get().exists(segment.getZkPath(), false));
+    }
+
+    @Test(timeout = 60000)
+    public void testDeleteNonExistentLogSegment() throws Exception {
+        LogSegmentMetadata segment = createLogSegment(1L);
+        Transaction<Object> deleteTxn = lsmStore.transaction();
+        lsmStore.deleteLogSegment(deleteTxn, segment, null);
+        try {
+            FutureUtils.result(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",
+                    t instanceof ZKException);
+            ZKException zke = (ZKException) t;
+            assertEquals("Should throw NoNodeException if log segment doesn't exist",
+                    KeeperException.Code.NONODE, zke.getKeeperExceptionCode());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testUpdateNonExistentLogSegment() throws Exception {
+        LogSegmentMetadata segment = createLogSegment(1L);
+        Transaction<Object> updateTxn = lsmStore.transaction();
+        lsmStore.updateLogSegment(updateTxn, segment);
+        try {
+            FutureUtils.result(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",
+                    t instanceof ZKException);
+            ZKException zke = (ZKException) t;
+            assertEquals("Should throw NoNodeException if log segment doesn't exist",
+                    KeeperException.Code.NONODE, zke.getKeeperExceptionCode());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testUpdateLogSegment() throws Exception {
+        LogSegmentMetadata segment = createLogSegment(1L, 99L);
+        Transaction<Object> createTxn = lsmStore.transaction();
+        lsmStore.createLogSegment(createTxn, segment, null);
+        FutureUtils.result(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());
+        // the log segment should be updated
+        LogSegmentMetadata readSegment =
+                FutureUtils.result(LogSegmentMetadata.read(zkc, segment.getZkPath(), true));
+        assertEquals("Last entry id should be changed from 99L to 999L",
+                999L, readSegment.getLastEntryId());
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateDeleteLogSegmentSuccess() throws Exception {
+        LogSegmentMetadata segment1 = createLogSegment(1L);
+        LogSegmentMetadata segment2 = createLogSegment(2L);
+        // create log segment 1
+        Transaction<Object> createTxn = lsmStore.transaction();
+        lsmStore.createLogSegment(createTxn, segment1, null);
+        FutureUtils.result(createTxn.execute());
+        // the log segment should be created
+        assertNotNull("LogSegment " + segment1 + " should be created",
+                zkc.get().exists(segment1.getZkPath(), false));
+        // delete log segment 1 and create log segment 2
+        Transaction<Object> createDeleteTxn = lsmStore.transaction();
+        lsmStore.createLogSegment(createDeleteTxn, segment2, null);
+        lsmStore.deleteLogSegment(createDeleteTxn, segment1, null);
+        FutureUtils.result(createDeleteTxn.execute());
+        // segment 1 should be deleted, segment 2 should be created
+        assertNull("LogSegment " + segment1 + " should be deleted",
+                zkc.get().exists(segment1.getZkPath(), false));
+        assertNotNull("LogSegment " + segment2 + " should be created",
+                zkc.get().exists(segment2.getZkPath(), false));
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateDeleteLogSegmentFailure() throws Exception {
+        LogSegmentMetadata segment1 = createLogSegment(1L);
+        LogSegmentMetadata segment2 = createLogSegment(2L);
+        LogSegmentMetadata segment3 = createLogSegment(3L);
+        // create log segment 1
+        Transaction<Object> createTxn = lsmStore.transaction();
+        lsmStore.createLogSegment(createTxn, segment1, null);
+        FutureUtils.result(createTxn.execute());
+        // the log segment should be created
+        assertNotNull("LogSegment " + segment1 + " should be created",
+                zkc.get().exists(segment1.getZkPath(), false));
+        // delete log segment 1 and delete log segment 2
+        Transaction<Object> createDeleteTxn = lsmStore.transaction();
+        lsmStore.deleteLogSegment(createDeleteTxn, segment1, null);
+        lsmStore.deleteLogSegment(createDeleteTxn, segment2, null);
+        lsmStore.createLogSegment(createDeleteTxn, segment3, null);
+        try {
+            FutureUtils.result(createDeleteTxn.execute());
+            fail("Should fail transaction if one operation failed");
+        } catch (Throwable t) {
+            assertTrue("Transaction is aborted",
+                    t instanceof ZKException);
+            ZKException zke = (ZKException) t;
+            assertEquals("Transaction is aborted",
+                    KeeperException.Code.NONODE, zke.getKeeperExceptionCode());
+        }
+        // segment 1 should not be deleted
+        assertNotNull("LogSegment " + segment1 + " should not be deleted",
+                zkc.get().exists(segment1.getZkPath(), false));
+        // segment 3 should not be created
+        assertNull("LogSegment " + segment3 + " should be created",
+                zkc.get().exists(segment3.getZkPath(), false));
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogSegment() throws Exception {
+        LogSegmentMetadata segment = createLogSegment(1L, 99L);
+        Transaction<Object> createTxn = lsmStore.transaction();
+        lsmStore.createLogSegment(createTxn, segment, null);
+        FutureUtils.result(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()));
+        assertEquals("Log segment should match",
+                segment, readSegment);
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogSegmentNames() throws Exception {
+        Transaction<Object> createTxn = lsmStore.transaction();
+        List<LogSegmentMetadata> createdSegments = Lists.newArrayListWithExpectedSize(10);
+        for (int i = 0; i < 10; i++) {
+            LogSegmentMetadata segment = createLogSegment(i);
+            createdSegments.add(segment);
+            lsmStore.createLogSegment(createTxn, segment, null);
+        }
+        FutureUtils.result(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();
+        Collections.sort(logSegmentNames);
+        assertEquals("Should find 10 log segments",
+                10, logSegmentNames.size());
+        assertEquals(children, logSegmentNames);
+        List<Future<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));
+        for (int i = 0; i < 10; i++) {
+            assertEquals(createdSegments.get(i), segments.get(i));
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testRegisterListenerAfterLSMStoreClosed() throws Exception {
+        lsmStore.close();
+        LogSegmentMetadata segment = createLogSegment(1L);
+        lsmStore.getLogSegmentNames(segment.getZkPath(), new LogSegmentNamesListener() {
+            @Override
+            public void onSegmentsUpdated(Versioned<List<String>> segments) {
+                // no-op;
+            }
+            @Override
+            public void onLogStreamDeleted() {
+                // no-op;
+            }
+        });
+        assertTrue("No listener is registered",
+                lsmStore.listeners.isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testLogSegmentNamesListener() throws Exception {
+        int numSegments = 3;
+        Transaction<Object> createTxn = lsmStore.transaction();
+        for (int i = 0; i < numSegments; i++) {
+            LogSegmentMetadata segment = createLogSegment(i);
+            lsmStore.createLogSegment(createTxn, segment, null);
+        }
+        FutureUtils.result(createTxn.execute());
+        String rootPath = "/" + runtime.getMethodName();
+        List<String> children = zkc.get().getChildren(rootPath, false);
+        Collections.sort(children);
+
+        final AtomicInteger numNotifications = new AtomicInteger(0);
+        final List<List<String>> segmentLists = Lists.newArrayListWithExpectedSize(2);
+        LogSegmentNamesListener listener = new LogSegmentNamesListener() {
+            @Override
+            public void onSegmentsUpdated(Versioned<List<String>> segments) {
+                logger.info("Received segments : {}", segments);
+                segmentLists.add(segments.getValue());
+                numNotifications.incrementAndGet();
+            }
+            @Override
+            public void onLogStreamDeleted() {
+                // no-op;
+            }
+        };
+        lsmStore.getLogSegmentNames(rootPath, listener);
+        assertEquals(1, lsmStore.listeners.size());
+        assertTrue("Should contain listener", lsmStore.listeners.containsKey(rootPath));
+        assertTrue("Should contain listener", lsmStore.listeners.get(rootPath).containsKey(listener));
+        while (numNotifications.get() < 1) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals("Should receive one segment list update",
+                1, numNotifications.get());
+        List<String> firstSegmentList = segmentLists.get(0);
+        Collections.sort(firstSegmentList);
+        assertEquals("List of segments should be same",
+                children, firstSegmentList);
+
+        logger.info("Create another {} segments.", numSegments);
+
+        // create another log segment, it should trigger segment list updated
+        Transaction<Object> anotherCreateTxn = lsmStore.transaction();
+        for (int i = numSegments; i < 2 * numSegments; i++) {
+            LogSegmentMetadata segment = createLogSegment(i);
+            lsmStore.createLogSegment(anotherCreateTxn, segment, null);
+        }
+        FutureUtils.result(anotherCreateTxn.execute());
+        List<String> newChildren = zkc.get().getChildren(rootPath, false);
+        Collections.sort(newChildren);
+        logger.info("All log segments become {}", newChildren);
+        while (numNotifications.get() < 2) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals("Should receive second segment list update",
+                2, numNotifications.get());
+        List<String> secondSegmentList = segmentLists.get(1);
+        Collections.sort(secondSegmentList);
+        assertEquals("List of segments should be updated",
+                2 * numSegments, secondSegmentList.size());
+        assertEquals("List of segments should be updated",
+                newChildren, secondSegmentList);
+    }
+
+    @Test(timeout = 60000)
+    public void testLogSegmentNamesListenerOnDeletion() throws Exception {
+        int numSegments = 3;
+        Transaction<Object> createTxn = lsmStore.transaction();
+        for (int i = 0; i < numSegments; i++) {
+            LogSegmentMetadata segment = createLogSegment(i);
+            lsmStore.createLogSegment(createTxn, segment, null);
+        }
+        FutureUtils.result(createTxn.execute());
+        String rootPath = "/" + runtime.getMethodName();
+        List<String> children = zkc.get().getChildren(rootPath, false);
+        Collections.sort(children);
+
+        final AtomicInteger numNotifications = new AtomicInteger(0);
+        final List<List<String>> segmentLists = Lists.newArrayListWithExpectedSize(2);
+        LogSegmentNamesListener listener = new LogSegmentNamesListener() {
+            @Override
+            public void onSegmentsUpdated(Versioned<List<String>> segments) {
+                logger.info("Received segments : {}", segments);
+                segmentLists.add(segments.getValue());
+                numNotifications.incrementAndGet();
+            }
+
+            @Override
+            public void onLogStreamDeleted() {
+                // no-op;
+            }
+        };
+        lsmStore.getLogSegmentNames(rootPath, listener);
+        assertEquals(1, lsmStore.listeners.size());
+        assertTrue("Should contain listener", lsmStore.listeners.containsKey(rootPath));
+        assertTrue("Should contain listener", lsmStore.listeners.get(rootPath).containsKey(listener));
+        while (numNotifications.get() < 1) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals("Should receive one segment list update",
+                1, numNotifications.get());
+        List<String> firstSegmentList = segmentLists.get(0);
+        Collections.sort(firstSegmentList);
+        assertEquals("List of segments should be same",
+                children, firstSegmentList);
+
+        // delete all log segments, it should trigger segment list updated
+        Transaction<Object> deleteTxn = lsmStore.transaction();
+        for (int i = 0; i < numSegments; i++) {
+            LogSegmentMetadata segment = createLogSegment(i);
+            lsmStore.deleteLogSegment(deleteTxn, segment, null);
+        }
+        FutureUtils.result(deleteTxn.execute());
+        List<String> newChildren = zkc.get().getChildren(rootPath, false);
+        Collections.sort(newChildren);
+        while (numNotifications.get() < 2) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals("Should receive second segment list update",
+                2, numNotifications.get());
+        List<String> secondSegmentList = segmentLists.get(1);
+        Collections.sort(secondSegmentList);
+        assertEquals("List of segments should be updated",
+                0, secondSegmentList.size());
+        assertEquals("List of segments should be updated",
+                newChildren, secondSegmentList);
+
+        // delete the root path
+        zkc.get().delete(rootPath, -1);
+        while (!lsmStore.listeners.isEmpty()) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertTrue("listener should be removed after root path is deleted",
+                lsmStore.listeners.isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testLogSegmentNamesListenerOnSessionExpired() throws Exception {
+        int numSegments = 3;
+        Transaction<Object> createTxn = lsmStore.transaction();
+        for (int i = 0; i < numSegments; i++) {
+            LogSegmentMetadata segment = createLogSegment(i);
+            lsmStore.createLogSegment(createTxn, segment, null);
+        }
+        FutureUtils.result(createTxn.execute());
+        String rootPath = "/" + runtime.getMethodName();
+        List<String> children = zkc.get().getChildren(rootPath, false);
+        Collections.sort(children);
+
+        final AtomicInteger numNotifications = new AtomicInteger(0);
+        final List<List<String>> segmentLists = Lists.newArrayListWithExpectedSize(2);
+        LogSegmentNamesListener listener = new LogSegmentNamesListener() {
+            @Override
+            public void onSegmentsUpdated(Versioned<List<String>> segments) {
+                logger.info("Received segments : {}", segments);
+                segmentLists.add(segments.getValue());
+                numNotifications.incrementAndGet();
+            }
+
+            @Override
+            public void onLogStreamDeleted() {
+                // no-op;
+            }
+        };
+        lsmStore.getLogSegmentNames(rootPath, listener);
+        assertEquals(1, lsmStore.listeners.size());
+        assertTrue("Should contain listener", lsmStore.listeners.containsKey(rootPath));
+        assertTrue("Should contain listener", lsmStore.listeners.get(rootPath).containsKey(listener));
+        while (numNotifications.get() < 1) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals("Should receive one segment list update",
+                1, numNotifications.get());
+        List<String> firstSegmentList = segmentLists.get(0);
+        Collections.sort(firstSegmentList);
+        assertEquals("List of segments should be same",
+                children, firstSegmentList);
+
+        ZooKeeperClientUtils.expireSession(zkc,
+                BKNamespaceDriver.getZKServersFromDLUri(uri), conf.getZKSessionTimeoutMilliseconds());
+
+        logger.info("Create another {} segments.", numSegments);
+
+        // create another log segment, it should trigger segment list updated
+        Transaction<Object> anotherCreateTxn = lsmStore.transaction();
+        for (int i = numSegments; i < 2 * numSegments; i++) {
+            LogSegmentMetadata segment = createLogSegment(i);
+            lsmStore.createLogSegment(anotherCreateTxn, segment, null);
+        }
+        FutureUtils.result(anotherCreateTxn.execute());
+        List<String> newChildren = zkc.get().getChildren(rootPath, false);
+        Collections.sort(newChildren);
+        logger.info("All log segments become {}", newChildren);
+        while (numNotifications.get() < 2) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals("Should receive third segment list update",
+                2, numNotifications.get());
+        List<String> thirdSegmentList = segmentLists.get(1);
+        Collections.sort(thirdSegmentList);
+        assertEquals("List of segments should be updated",
+                2 * numSegments, thirdSegmentList.size());
+        assertEquals("List of segments should be updated",
+                newChildren, thirdSegmentList);
+    }
+
+    @Test(timeout = 60000)
+    public void testLogSegmentNamesListenerOnDeletingLogStream() throws Exception {
+        int numSegments = 3;
+        Transaction<Object> createTxn = lsmStore.transaction();
+        for (int i = 0; i < numSegments; i++) {
+            LogSegmentMetadata segment = createLogSegment(i);
+            lsmStore.createLogSegment(createTxn, segment, null);
+        }
+        FutureUtils.result(createTxn.execute());
+        String rootPath = "/" + runtime.getMethodName();
+        List<String> children = zkc.get().getChildren(rootPath, false);
+        Collections.sort(children);
+
+        final AtomicInteger numNotifications = new AtomicInteger(0);
+        final List<List<String>> segmentLists = Lists.newArrayListWithExpectedSize(2);
+        final CountDownLatch deleteLatch = new CountDownLatch(1);
+        LogSegmentNamesListener listener = new LogSegmentNamesListener() {
+            @Override
+            public void onSegmentsUpdated(Versioned<List<String>> segments) {
+                logger.info("Received segments : {}", segments);
+                segmentLists.add(segments.getValue());
+                numNotifications.incrementAndGet();
+            }
+
+            @Override
+            public void onLogStreamDeleted() {
+                deleteLatch.countDown();
+            }
+        };
+        lsmStore.getLogSegmentNames(rootPath, listener);
+        assertEquals(1, lsmStore.listeners.size());
+        assertTrue("Should contain listener", lsmStore.listeners.containsKey(rootPath));
+        assertTrue("Should contain listener", lsmStore.listeners.get(rootPath).containsKey(listener));
+        while (numNotifications.get() < 1) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals("Should receive one segment list update",
+                1, numNotifications.get());
+        List<String> firstSegmentList = segmentLists.get(0);
+        Collections.sort(firstSegmentList);
+        assertEquals("List of segments should be same",
+                children, firstSegmentList);
+
+        // delete all log segments, it should trigger segment list updated
+        Transaction<Object> deleteTxn = lsmStore.transaction();
+        for (int i = 0; i < numSegments; i++) {
+            LogSegmentMetadata segment = createLogSegment(i);
+            lsmStore.deleteLogSegment(deleteTxn, segment, null);
+        }
+        FutureUtils.result(deleteTxn.execute());
+        List<String> newChildren = zkc.get().getChildren(rootPath, false);
+        Collections.sort(newChildren);
+        while (numNotifications.get() < 2) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals("Should receive second segment list update",
+                2, numNotifications.get());
+        List<String> secondSegmentList = segmentLists.get(1);
+        Collections.sort(secondSegmentList);
+        assertEquals("List of segments should be updated",
+                0, secondSegmentList.size());
+        assertEquals("List of segments should be updated",
+                newChildren, secondSegmentList);
+
+        // delete the root path
+        zkc.get().delete(rootPath, -1);
+        while (!lsmStore.listeners.isEmpty()) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertTrue("listener should be removed after root path is deleted",
+                lsmStore.listeners.isEmpty());
+        deleteLatch.await();
+    }
+
+    @Test(timeout = 60000)
+    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>();
+        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);
+            }
+
+            @Override
+            public void onAbort(Throwable t) {
+                result.setException(t);
+            }
+        });
+        FutureUtils.result(updateTxn.execute());
+        assertEquals(1, ((ZkVersion) FutureUtils.result(result)).getZnodeVersion());
+        Stat stat = new Stat();
+        byte[] data = zkc.get().getData(rootZkPath, false, stat);
+        assertEquals(999L, DLUtils.deserializeLogSegmentSequenceNumber(data));
+        assertEquals(1, stat.getVersion());
+    }
+
+    @Test(timeout = 60000)
+    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>();
+        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);
+                    }
+
+                    @Override
+                    public void onAbort(Throwable t) {
+                        result.setException(t);
+                    }
+                });
+        try {
+            FutureUtils.result(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);
+            fail("Should fail on storing log segment sequence number if providing bad version");
+        } catch (KeeperException ke) {
+            assertEquals(KeeperException.Code.BADVERSION, ke.code());
+        }
+        Stat stat = new Stat();
+        byte[] data = zkc.get().getData(rootZkPath, false, stat);
+        assertEquals(0, stat.getVersion());
+        assertEquals(0, data.length);
+    }
+
+    @Test(timeout = 60000)
+    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>();
+        String nonExistentPath = rootZkPath + "/non-existent";
+        LogMetadata metadata = mock(LogMetadata.class);
+        when(metadata.getLogSegmentsPath()).thenReturn(nonExistentPath);
+        lsmStore.storeMaxLogSegmentSequenceNumber(updateTxn, metadata, value,
+                new Transaction.OpListener<Version>() {
+                    @Override
+                    public void onCommit(Version r) {
+                        result.setValue(r);
+                    }
+
+                    @Override
+                    public void onAbort(Throwable t) {
+                        result.setException(t);
+                    }
+                });
+        try {
+            FutureUtils.result(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);
+            fail("Should fail on storing log segment sequence number if path doesn't exist");
+        } catch (KeeperException ke) {
+            assertEquals(KeeperException.Code.NONODE, ke.code());
+        }
+    }
+
+    @Test(timeout = 60000)
+    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>();
+        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);
+            }
+
+            @Override
+            public void onAbort(Throwable t) {
+                result.setException(t);
+            }
+        });
+        FutureUtils.result(updateTxn.execute());
+        assertEquals(1, ((ZkVersion) FutureUtils.result(result)).getZnodeVersion());
+        Stat stat = new Stat();
+        byte[] data = zkc.get().getData(rootZkPath, false, stat);
+        assertEquals(999L, DLUtils.deserializeTransactionId(data));
+        assertEquals(1, stat.getVersion());
+    }
+
+    @Test(timeout = 60000)
+    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>();
+        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);
+                    }
+
+                    @Override
+                    public void onAbort(Throwable t) {
+                        result.setException(t);
+                    }
+                });
+        try {
+            FutureUtils.result(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);
+            fail("Should fail on storing log record transaction id if providing bad version");
+        } catch (KeeperException ke) {
+            assertEquals(KeeperException.Code.BADVERSION, ke.code());
+        }
+        Stat stat = new Stat();
+        byte[] data = zkc.get().getData(rootZkPath, false, stat);
+        assertEquals(0, stat.getVersion());
+        assertEquals(0, data.length);
+    }
+
+    @Test(timeout = 60000)
+    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>();
+        String nonExistentPath = rootZkPath + "/non-existent";
+        LogMetadataForWriter metadata = mock(LogMetadataForWriter.class);
+        when(metadata.getMaxTxIdPath()).thenReturn(nonExistentPath);
+        lsmStore.storeMaxTxnId(updateTxn, metadata, value,
+                new Transaction.OpListener<Version>() {
+                    @Override
+                    public void onCommit(Version r) {
+                        result.setValue(r);
+                    }
+
+                    @Override
+                    public void onAbort(Throwable t) {
+                        result.setException(t);
+                    }
+                });
+        try {
+            FutureUtils.result(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);
+            fail("Should fail on storing log record transaction id if path doesn't exist");
+        } catch (KeeperException ke) {
+            assertEquals(KeeperException.Code.NONODE, ke.code());
+        }
+    }
+
+}
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
new file mode 100644
index 0000000..3c6e77c
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/TestZKNamespaceWatcher.java
@@ -0,0 +1,185 @@
+/**
+ * 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.impl;
+
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.TestDistributedLogBase;
+import org.apache.distributedlog.TestZooKeeperClientBuilder;
+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;
+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 java.net.URI;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test ZK Namespace Watcher.
+ */
+public class TestZKNamespaceWatcher extends TestDistributedLogBase {
+
+    private final static int zkSessionTimeoutMs = 2000;
+
+    @Rule
+    public TestName runtime = new TestName();
+    protected final DistributedLogConfiguration baseConf =
+            new DistributedLogConfiguration();
+    protected ZooKeeperClient zkc;
+    protected OrderedScheduler scheduler;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .uri(createDLMURI("/"))
+                .sessionTimeoutMs(zkSessionTimeoutMs)
+                .build();
+        scheduler = OrderedScheduler.newBuilder()
+                .name("test-zk-namespace-watcher")
+                .corePoolSize(1)
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (null != zkc) {
+            zkc.close();
+        }
+        if (null != scheduler) {
+            scheduler.shutdown();
+        }
+    }
+
+    private void createLogInNamespace(URI uri, String logName) throws Exception {
+        String logPath = uri.getPath() + "/" + logName;
+        Utils.zkCreateFullPathOptimistic(zkc, logPath, new byte[0],
+                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+    }
+
+    private void deleteLogInNamespace(URI uri, String logName) throws Exception {
+        String logPath = uri.getPath() + "/" + logName;
+        zkc.get().delete(logPath, -1);
+    }
+
+    @Test(timeout = 60000)
+    public void testNamespaceListener() throws Exception {
+        URI uri = createDLMURI("/" + runtime.getMethodName());
+        zkc.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.addConfiguration(baseConf);
+        ZKNamespaceWatcher watcher = new ZKNamespaceWatcher(conf, uri, zkc, scheduler);
+        final CountDownLatch[] latches = new CountDownLatch[10];
+        for (int i = 0; i < 10; i++) {
+            latches[i] = new CountDownLatch(1);
+        }
+        final AtomicInteger numUpdates = new AtomicInteger(0);
+        final AtomicReference<Set<String>> receivedLogs = new AtomicReference<Set<String>>(null);
+        watcher.registerListener(new NamespaceListener() {
+            @Override
+            public void onStreamsChanged(Iterator<String> streams) {
+                Set<String> streamSet = Sets.newHashSet(streams);
+                int updates = numUpdates.incrementAndGet();
+                receivedLogs.set(streamSet);
+                latches[updates - 1].countDown();
+            }
+        });
+        // first update
+        final Set<String> expectedLogs = Sets.newHashSet();
+        latches[0].await();
+        validateReceivedLogs(expectedLogs, receivedLogs.get());
+
+        // create test1
+        expectedLogs.add("test1");
+        createLogInNamespace(uri, "test1");
+        latches[1].await();
+        validateReceivedLogs(expectedLogs, receivedLogs.get());
+
+        // create invalid log
+        createLogInNamespace(uri, ".test1");
+        latches[2].await();
+        validateReceivedLogs(expectedLogs, receivedLogs.get());
+
+        // create test2
+        expectedLogs.add("test2");
+        createLogInNamespace(uri, "test2");
+        latches[3].await();
+        validateReceivedLogs(expectedLogs, receivedLogs.get());
+
+        // delete test1
+        expectedLogs.remove("test1");
+        deleteLogInNamespace(uri, "test1");
+        latches[4].await();
+        validateReceivedLogs(expectedLogs, receivedLogs.get());
+    }
+
+    private void validateReceivedLogs(Set<String> expectedLogs, Set<String> receivedLogs) {
+        assertTrue(Sets.difference(expectedLogs, receivedLogs).isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testSessionExpired() throws Exception {
+        URI uri = createDLMURI("/" + runtime.getMethodName());
+        zkc.get().create(uri.getPath(), new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.addConfiguration(baseConf);
+        ZKNamespaceWatcher watcher = new ZKNamespaceWatcher(conf, uri, zkc, scheduler);
+        final CountDownLatch[] latches = new CountDownLatch[10];
+        for (int i = 0; i < 10; i++) {
+            latches[i] = new CountDownLatch(1);
+        }
+        final AtomicInteger numUpdates = new AtomicInteger(0);
+        final AtomicReference<Set<String>> receivedLogs = new AtomicReference<Set<String>>(null);
+        watcher.registerListener(new NamespaceListener() {
+            @Override
+            public void onStreamsChanged(Iterator<String> streams) {
+                Set<String> streamSet = Sets.newHashSet(streams);
+                int updates = numUpdates.incrementAndGet();
+                receivedLogs.set(streamSet);
+                latches[updates - 1].countDown();
+            }
+        });
+        latches[0].await();
+        createLogInNamespace(uri, "test1");
+        latches[1].await();
+        createLogInNamespace(uri, "test2");
+        latches[2].await();
+        assertEquals(2, receivedLogs.get().size());
+        ZooKeeperClientUtils.expireSession(zkc, BKNamespaceDriver.getZKServersFromDLUri(uri), zkSessionTimeoutMs);
+        latches[3].await();
+        assertEquals(2, receivedLogs.get().size());
+        createLogInNamespace(uri, "test3");
+        latches[4].await();
+        assertEquals(3, receivedLogs.get().size());
+    }
+
+}
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
new file mode 100644
index 0000000..9c46d96
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/federated/TestFederatedZKLogMetadataStore.java
@@ -0,0 +1,446 @@
+/**
+ * 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.impl.federated;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Iterators;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.DistributedLogConfiguration;
+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.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import java.net.URI;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Test ZK based metadata store.
+ */
+public class TestFederatedZKLogMetadataStore extends TestDistributedLogBase {
+
+    private final static int zkSessionTimeoutMs = 2000;
+    private final static int maxLogsPerSubnamespace = 10;
+
+    static class TestNamespaceListener implements NamespaceListener {
+
+        final CountDownLatch doneLatch = new CountDownLatch(1);
+        final AtomicReference<Iterator<String>> resultHolder = new AtomicReference<Iterator<String>>();
+
+        @Override
+        public void onStreamsChanged(Iterator<String> streams) {
+            resultHolder.set(streams);
+            if (streams.hasNext()) {
+                doneLatch.countDown();
+            }
+        }
+
+        Iterator<String> getResult() {
+            return resultHolder.get();
+        }
+
+        void waitForDone() throws InterruptedException {
+            doneLatch.await();
+        }
+    }
+
+    static class TestNamespaceListenerWithExpectedSize implements NamespaceListener {
+
+        final int expectedSize;
+        final CountDownLatch doneLatch = new CountDownLatch(1);
+        final AtomicReference<Set<String>> resultHolder = new AtomicReference<Set<String>>();
+
+        TestNamespaceListenerWithExpectedSize(int expectedSize) {
+            this.expectedSize = expectedSize;
+        }
+
+        Set<String> getResult() {
+            return resultHolder.get();
+        }
+
+        @Override
+        public void onStreamsChanged(Iterator<String> logsIter) {
+            List<String> logList = Lists.newArrayList(logsIter);
+            if (logList.size() < expectedSize) {
+                return;
+            }
+            resultHolder.set(Sets.newTreeSet(logList));
+            doneLatch.countDown();
+        }
+
+        void waitForDone() throws InterruptedException {
+            doneLatch.await();
+        }
+    }
+
+    @Rule
+    public TestName runtime = new TestName();
+    protected final DistributedLogConfiguration baseConf =
+            new DistributedLogConfiguration()
+                    .setFederatedMaxLogsPerSubnamespace(maxLogsPerSubnamespace);
+    protected ZooKeeperClient zkc;
+    protected FederatedZKLogMetadataStore metadataStore;
+    protected OrderedScheduler scheduler;
+    protected URI uri;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .uri(createDLMURI("/"))
+                .sessionTimeoutMs(zkSessionTimeoutMs)
+                .build();
+        scheduler = OrderedScheduler.newBuilder()
+                .name("test-zk-logmetadata-store")
+                .corePoolSize(2)
+                .build();
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.addConfiguration(baseConf);
+        this.uri = createDLMURI("/" + runtime.getMethodName());
+        FederatedZKLogMetadataStore.createFederatedNamespace(uri, zkc);
+        metadataStore = new FederatedZKLogMetadataStore(conf, uri, zkc, scheduler);
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (null != zkc) {
+            zkc.close();
+        }
+        if (null != scheduler) {
+            scheduler.shutdown();
+        }
+    }
+
+    private void deleteLog(String logName) throws Exception {
+        Optional<URI> logUriOptional = FutureUtils.result(metadataStore.getLogLocation(logName));
+        assertTrue(logUriOptional.isPresent());
+        URI logUri = logUriOptional.get();
+        zkc.get().delete(logUri.getPath() + "/" + logName, -1);
+    }
+
+    @Test(timeout = 60000)
+    public void testBasicOperations() throws Exception {
+        TestNamespaceListener listener = new TestNamespaceListener();
+        metadataStore.registerNamespaceListener(listener);
+        String logName = "test-log-1";
+        URI logUri = FutureUtils.result(metadataStore.createLog(logName));
+        assertEquals(uri, logUri);
+        Optional<URI> logLocation = FutureUtils.result(metadataStore.getLogLocation(logName));
+        assertTrue(logLocation.isPresent());
+        assertEquals(uri, logLocation.get());
+        Optional<URI> notExistLogLocation = FutureUtils.result(metadataStore.getLogLocation("non-existent-log"));
+        assertFalse(notExistLogLocation.isPresent());
+        // listener should receive notification
+        listener.waitForDone();
+        Iterator<String> logsIter = listener.getResult();
+        assertTrue(logsIter.hasNext());
+        assertEquals(logName, logsIter.next());
+        assertFalse(logsIter.hasNext());
+        // get logs should return the log
+        Iterator<String> newLogsIter = FutureUtils.result(metadataStore.getLogs());
+        assertTrue(newLogsIter.hasNext());
+        assertEquals(logName, newLogsIter.next());
+        assertFalse(newLogsIter.hasNext());
+    }
+
+    @Test(timeout = 60000)
+    public void testMultipleListeners() throws Exception {
+        TestNamespaceListener listener1 = new TestNamespaceListener();
+        TestNamespaceListener listener2 = new TestNamespaceListener();
+        metadataStore.registerNamespaceListener(listener1);
+        metadataStore.registerNamespaceListener(listener2);
+        String logName = "test-multiple-listeners";
+        URI logUri = FutureUtils.result(metadataStore.createLog(logName));
+        assertEquals(uri, logUri);
+        listener1.waitForDone();
+        listener2.waitForDone();
+        Iterator<String> logsIter1 = listener1.getResult();
+        Iterator<String> logsIter2 = listener2.getResult();
+        assertTrue(Iterators.elementsEqual(logsIter1, logsIter2));
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLog() throws Exception {
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.addConfiguration(baseConf);
+        ZooKeeperClient anotherZkc = TestZooKeeperClientBuilder.newBuilder()
+                .uri(uri)
+                .sessionTimeoutMs(zkSessionTimeoutMs)
+                .build();
+        FederatedZKLogMetadataStore anotherMetadataStore =
+                new FederatedZKLogMetadataStore(conf, uri, anotherZkc, scheduler);
+        for (int i = 0; i < 2 * maxLogsPerSubnamespace; i++) {
+            LogMetadataStore createStore, checkStore;
+            if (i % 2 == 0) {
+                createStore = metadataStore;
+                checkStore = anotherMetadataStore;
+            } else {
+                createStore = anotherMetadataStore;
+                checkStore = metadataStore;
+            }
+            String logName = "test-create-log-" + i;
+            URI logUri = FutureUtils.result(createStore.createLog(logName));
+            Optional<URI> logLocation = FutureUtils.result(checkStore.getLogLocation(logName));
+            assertTrue("Log " + logName + " doesn't exist", logLocation.isPresent());
+            assertEquals("Different log location " + logLocation.get() + " is found",
+                    logUri, logLocation.get());
+        }
+        assertEquals(2, metadataStore.getSubnamespaces().size());
+        assertEquals(2, anotherMetadataStore.getSubnamespaces().size());
+    }
+
+    @Test(timeout = 60000)
+    public void testDuplicatedLogs() throws Exception {
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.addConfiguration(baseConf);
+
+        String logName = "test-log";
+        FutureUtils.result(metadataStore.createLog(logName));
+
+        URI subNs1 = FutureUtils.result(metadataStore.createSubNamespace());
+        URI subNs2 = FutureUtils.result(metadataStore.createSubNamespace());
+
+        String duplicatedLogName = "test-duplicated-logs";
+        // Create same log in different sub namespaces
+        metadataStore.createLogInNamespaceSync(subNs1, duplicatedLogName);
+        metadataStore.createLogInNamespaceSync(subNs2, duplicatedLogName);
+
+        try {
+            FutureUtils.result(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));
+            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"));
+            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));
+            fail("should throw exception when duplicated log found");
+        } catch (UnexpectedException ue) {
+            // should throw unexpected exception
+            assertTrue(metadataStore.duplicatedLogFound.get());
+        }
+        try {
+            FutureUtils.result(metadataStore.getLogs());
+            fail("should throw exception when duplicated log found");
+        } catch (UnexpectedException ue) {
+            // should throw unexpected exception
+            assertTrue(metadataStore.duplicatedLogFound.get());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogLocationWhenCacheMissed() throws Exception {
+        String logName = "test-get-location-when-cache-missed";
+        URI logUri = FutureUtils.result(metadataStore.createLog(logName));
+        assertEquals(uri, logUri);
+        metadataStore.removeLogFromCache(logName);
+        Optional<URI> logLocation = FutureUtils.result(metadataStore.getLogLocation(logName));
+        assertTrue(logLocation.isPresent());
+        assertEquals(logUri, logLocation.get());
+    }
+
+    @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));
+        assertEquals(uri, logUri);
+        metadataStore.removeLogFromCache(logName);
+        FutureUtils.result(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));
+        assertEquals(uri, logUri);
+        FutureUtils.result(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));
+            expectedLogs.add(logName);
+        }
+        return expectedLogs;
+    }
+
+    private Set<String> createLogs(URI uri, int numLogs, String prefix) throws Exception {
+        Set<String> expectedLogs = Sets.newTreeSet();
+        for (int i = 0; i < numLogs; i++) {
+            String logName = prefix + i;
+            metadataStore.createLogInNamespaceSync(uri, logName);
+            expectedLogs.add(logName);
+        }
+        return expectedLogs;
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLogs() throws Exception {
+        int numLogs = 3 * maxLogsPerSubnamespace;
+        Set<String> expectedLogs = createLogs(numLogs, "test-get-logs");
+        Set<String> receivedLogs;
+        do {
+            TimeUnit.MILLISECONDS.sleep(20);
+            receivedLogs = new TreeSet<String>();
+            Iterator<String> logs = FutureUtils.result(metadataStore.getLogs());
+            receivedLogs.addAll(Lists.newArrayList(logs));
+        } while (receivedLogs.size() < numLogs);
+        assertEquals(numLogs, receivedLogs.size());
+        assertTrue(Sets.difference(expectedLogs, receivedLogs).isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testNamespaceListener() throws Exception {
+        int numLogs = 3 * maxLogsPerSubnamespace;
+        TestNamespaceListenerWithExpectedSize listener = new TestNamespaceListenerWithExpectedSize(numLogs);
+        metadataStore.registerNamespaceListener(listener);
+        Set<String> expectedLogs = createLogs(numLogs, "test-namespace-listener");
+        listener.waitForDone();
+        Set<String> receivedLogs = listener.getResult();
+        assertEquals(numLogs, receivedLogs.size());
+        assertTrue(Sets.difference(expectedLogs, receivedLogs).isEmpty());
+
+        Random r = new Random(System.currentTimeMillis());
+        int logId = r.nextInt(numLogs);
+        String logName = "test-namespace-listener" + logId;
+        TestNamespaceListener deleteListener = new TestNamespaceListener();
+        metadataStore.registerNamespaceListener(deleteListener);
+        deleteLog(logName);
+        deleteListener.waitForDone();
+        Set<String> logsAfterDeleted = Sets.newTreeSet(Lists.newArrayList(deleteListener.getResult()));
+        assertEquals(numLogs - 1, logsAfterDeleted.size());
+        expectedLogs.remove(logName);
+        assertTrue(Sets.difference(expectedLogs, receivedLogs).isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogPickingFirstAvailableSubNamespace() throws Exception {
+        URI subNs1 = FutureUtils.result(metadataStore.createSubNamespace());
+        URI subNs2 = FutureUtils.result(metadataStore.createSubNamespace());
+
+        Set<String> logs0 = createLogs(uri, maxLogsPerSubnamespace - 1, "test-ns0-");
+        Set<String> logs1 = createLogs(subNs1, maxLogsPerSubnamespace, "test-ns1-");
+        Set<String> logs2 = createLogs(subNs2, maxLogsPerSubnamespace, "test-ns2-");
+        Set<String> allLogs = Sets.newTreeSet();
+        allLogs.addAll(logs0);
+        allLogs.addAll(logs1);
+        allLogs.addAll(logs2);
+
+        // make sure the metadata store saw all 29 logs
+        Set<String> receivedLogs;
+        do {
+            TimeUnit.MILLISECONDS.sleep(20);
+            receivedLogs = new TreeSet<String>();
+            Iterator<String> logs = FutureUtils.result(metadataStore.getLogs());
+            receivedLogs.addAll(Lists.newArrayList(logs));
+        } while (receivedLogs.size() < 3 * maxLogsPerSubnamespace - 1);
+
+        TestNamespaceListenerWithExpectedSize listener =
+                new TestNamespaceListenerWithExpectedSize(3 * maxLogsPerSubnamespace + 1);
+        metadataStore.registerNamespaceListener(listener);
+
+        Set<URI> uris = FutureUtils.result(metadataStore.fetchSubNamespaces(null));
+        assertEquals(3, uris.size());
+        String testLogName = "test-pick-first-available-ns";
+        URI createdURI = FutureUtils.result(metadataStore.createLog(testLogName));
+        allLogs.add(testLogName);
+        assertEquals(uri, createdURI);
+        uris = FutureUtils.result(metadataStore.fetchSubNamespaces(null));
+        assertEquals(3, uris.size());
+        testLogName = "test-create-new-ns";
+        URI newURI = FutureUtils.result(metadataStore.createLog(testLogName));
+        allLogs.add(testLogName);
+        assertFalse(uris.contains(newURI));
+        uris = FutureUtils.result(metadataStore.fetchSubNamespaces(null));
+        assertEquals(4, uris.size());
+
+        listener.waitForDone();
+        receivedLogs = listener.getResult();
+        assertEquals(3 * maxLogsPerSubnamespace + 1, receivedLogs.size());
+        assertEquals(allLogs, receivedLogs);
+    }
+
+    @Test(timeout = 60000)
+    public void testZooKeeperSessionExpired() throws Exception {
+        Set<String> allLogs = createLogs(2 * maxLogsPerSubnamespace, "test-zookeeper-session-expired-");
+        TestNamespaceListenerWithExpectedSize listener =
+                new TestNamespaceListenerWithExpectedSize(2 * maxLogsPerSubnamespace + 1);
+        metadataStore.registerNamespaceListener(listener);
+        ZooKeeperClientUtils.expireSession(zkc, BKNamespaceDriver.getZKServersFromDLUri(uri), zkSessionTimeoutMs);
+        String testLogName = "test-log-name";
+        allLogs.add(testLogName);
+
+        DistributedLogConfiguration anotherConf = new DistributedLogConfiguration();
+        anotherConf.addConfiguration(baseConf);
+        ZooKeeperClient anotherZkc = TestZooKeeperClientBuilder.newBuilder()
+                .uri(uri)
+                .sessionTimeoutMs(zkSessionTimeoutMs)
+                .build();
+        FederatedZKLogMetadataStore anotherMetadataStore =
+                new FederatedZKLogMetadataStore(anotherConf, uri, anotherZkc, scheduler);
+        FutureUtils.result(anotherMetadataStore.createLog(testLogName));
+
+        listener.waitForDone();
+        Set<String> receivedLogs = listener.getResult();
+        assertEquals(2 * maxLogsPerSubnamespace + 1, receivedLogs.size());
+        assertEquals(allLogs, receivedLogs);
+    }
+
+}
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
new file mode 100644
index 0000000..5505259
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/logsegment/TestBKLogSegmentEntryReader.java
@@ -0,0 +1,561 @@
+/**
+ * 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.impl.logsegment;
+
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.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.Entry;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.TestDistributedLogBase;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.exceptions.EndOfLogSegmentException;
+import org.apache.distributedlog.exceptions.ReadCancelledException;
+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;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test Case for {@link BKLogSegmentEntryReader}
+ */
+public class TestBKLogSegmentEntryReader extends TestDistributedLogBase {
+
+    @Rule
+    public TestName runtime = new TestName();
+    private OrderedScheduler scheduler;
+    private BookKeeperClient bkc;
+    private ZooKeeperClient zkc;
+
+    @Before
+    public void setup() throws Exception {
+        super.setup();
+        zkc = ZooKeeperClientBuilder.newBuilder()
+                .name("test-zk")
+                .zkServers(bkutil.getZkServers())
+                .sessionTimeoutMs(conf.getZKSessionTimeoutMilliseconds())
+                .zkAclId(conf.getZkAclId())
+                .build();
+        bkc = BookKeeperClientBuilder.newBuilder()
+                .name("test-bk")
+                .dlConfig(conf)
+                .ledgersPath("/ledgers")
+                .zkServers(bkutil.getZkServers())
+                .build();
+        scheduler = OrderedScheduler.newBuilder()
+                .name("test-bk-logsegment-entry-reader")
+                .corePoolSize(1)
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (null != bkc) {
+            bkc.close();
+        }
+        if (null != scheduler) {
+            scheduler.shutdown();
+        }
+        if (null != zkc) {
+            zkc.close();
+        }
+        super.teardown();
+    }
+
+    BKLogSegmentEntryReader createEntryReader(LogSegmentMetadata segment,
+                                              long startEntryId,
+                                              DistributedLogConfiguration conf)
+            throws Exception {
+        LogSegmentEntryStore store = new BKLogSegmentEntryStore(
+                conf,
+                ConfUtils.getConstDynConf(conf),
+                zkc,
+                bkc,
+                scheduler,
+                null,
+                NullStatsLogger.INSTANCE,
+                AsyncFailureInjector.NULL);
+        return (BKLogSegmentEntryReader) FutureUtils.result(store.openReader(segment, startEntryId));
+    }
+
+    void generateCompletedLogSegments(DistributedLogManager dlm,
+                                      DistributedLogConfiguration conf,
+                                      long numCompletedSegments,
+                                      long segmentSize) throws Exception {
+        long txid = 1L;
+        for (long i = 0; i < numCompletedSegments; i++) {
+            AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+            for (long j = 1; j <= segmentSize; j++) {
+                FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txid++)));
+                LogRecord ctrlRecord = DLMTestUtil.getLogRecordInstance(txid);
+                ctrlRecord.setControl();
+                FutureUtils.result(writer.write(ctrlRecord));
+            }
+            Utils.close(writer);
+        }
+    }
+
+    AsyncLogWriter createInprogressLogSegment(DistributedLogManager dlm,
+                                              DistributedLogConfiguration conf,
+                                              long segmentSize) throws Exception {
+        AsyncLogWriter writer = FutureUtils.result(dlm.openAsyncLogWriter());
+        for (long i = 1L; i <= segmentSize; i++) {
+            FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(i)));
+            LogRecord ctrlRecord = DLMTestUtil.getLogRecordInstance(i);
+            ctrlRecord.setControl();
+            FutureUtils.result(writer.write(ctrlRecord));
+        }
+        return writer;
+    }
+
+    @Test(timeout = 60000)
+    public void testReadEntriesFromCompleteLogSegment() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setNumPrefetchEntriesPerLogSegment(10);
+        confLocal.setMaxPrefetchEntriesPerLogSegment(10);
+        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
+        generateCompletedLogSegments(dlm, confLocal, 1, 20);
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(segments.size() + " log segments found, expected to be only one",
+                1, segments.size());
+
+        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
+        reader.start();
+        boolean done = false;
+        long txId = 1L;
+        long entryId = 0L;
+        while (!done) {
+            Entry.Reader entryReader;
+            try {
+                entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+            } catch (EndOfLogSegmentException eol) {
+                done = true;
+                continue;
+            }
+            LogRecordWithDLSN record = entryReader.nextRecord();
+            while (null != record) {
+                if (!record.isControl()) {
+                    DLMTestUtil.verifyLogRecord(record);
+                    assertEquals(txId, record.getTransactionId());
+                    ++txId;
+                }
+                DLSN dlsn = record.getDlsn();
+                assertEquals(1L, dlsn.getLogSegmentSequenceNo());
+                assertEquals(entryId, dlsn.getEntryId());
+                record = entryReader.nextRecord();
+            }
+            ++entryId;
+        }
+        assertEquals(21, txId);
+        assertFalse(reader.hasCaughtUpOnInprogress());
+        Utils.close(reader);
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseReaderToCancelPendingReads() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setNumPrefetchEntriesPerLogSegment(10);
+        confLocal.setMaxPrefetchEntriesPerLogSegment(10);
+        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
+        DLMTestUtil.generateCompletedLogSegments(dlm, confLocal, 1, 20);
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(segments.size() + " log segments found, expected to be only one",
+                1, segments.size());
+
+        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
+        List<Future<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) {
+            try {
+                FutureUtils.result(future);
+                fail("The read request should be cancelled");
+            } catch (ReadCancelledException rce) {
+                // expected
+            }
+        }
+        assertFalse(reader.hasCaughtUpOnInprogress());
+        assertTrue("Reader should be closed yet", reader.isClosed());
+    }
+
+    @Test(timeout = 60000)
+    public void testMaxPrefetchEntriesSmallBatch() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setNumPrefetchEntriesPerLogSegment(2);
+        confLocal.setMaxPrefetchEntriesPerLogSegment(10);
+        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
+        generateCompletedLogSegments(dlm, confLocal, 1, 20);
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(segments.size() + " log segments found, expected to be only one",
+                1, segments.size());
+
+        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
+        reader.start();
+
+        // wait for the read ahead entries to become available
+        while (reader.readAheadEntries.size() < 10) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+
+        long txId = 1L;
+        long entryId = 0L;
+
+        assertEquals(10, reader.readAheadEntries.size());
+        assertEquals(10, reader.getNextEntryId());
+        assertFalse(reader.hasCaughtUpOnInprogress());
+        // read first entry
+        Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+        LogRecordWithDLSN record = entryReader.nextRecord();
+        while (null != record) {
+            if (!record.isControl()) {
+                DLMTestUtil.verifyLogRecord(record);
+                assertEquals(txId, record.getTransactionId());
+                ++txId;
+            }
+            DLSN dlsn = record.getDlsn();
+            assertEquals(1L, dlsn.getLogSegmentSequenceNo());
+            assertEquals(entryId, dlsn.getEntryId());
+            record = entryReader.nextRecord();
+        }
+        ++entryId;
+        assertEquals(2L, txId);
+        // wait for the read ahead entries to become 10 again
+        while (reader.readAheadEntries.size() < 10) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+
+        assertEquals(10, reader.readAheadEntries.size());
+        assertEquals(11, reader.getNextEntryId());
+        assertFalse(reader.hasCaughtUpOnInprogress());
+
+        Utils.close(reader);
+    }
+
+    @Test(timeout = 60000)
+    public void testMaxPrefetchEntriesLargeBatch() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setNumPrefetchEntriesPerLogSegment(10);
+        confLocal.setMaxPrefetchEntriesPerLogSegment(5);
+        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
+        generateCompletedLogSegments(dlm, confLocal, 1, 20);
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(segments.size() + " log segments found, expected to be only one",
+                1, segments.size());
+
+        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
+        reader.start();
+
+        // wait for the read ahead entries to become available
+        while (reader.readAheadEntries.size() < 5) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+
+        long txId = 1L;
+        long entryId = 0L;
+
+        assertEquals(5, reader.readAheadEntries.size());
+        assertEquals(5, reader.getNextEntryId());
+        // read first entry
+        Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+        LogRecordWithDLSN record = entryReader.nextRecord();
+        while (null != record) {
+            if (!record.isControl()) {
+                DLMTestUtil.verifyLogRecord(record);
+                assertEquals(txId, record.getTransactionId());
+                ++txId;
+            }
+            DLSN dlsn = record.getDlsn();
+            assertEquals(1L, dlsn.getLogSegmentSequenceNo());
+            assertEquals(entryId, dlsn.getEntryId());
+            record = entryReader.nextRecord();
+        }
+        ++entryId;
+        assertEquals(2L, txId);
+        // wait for the read ahead entries to become 10 again
+        while (reader.readAheadEntries.size() < 5) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+
+        assertEquals(5, reader.readAheadEntries.size());
+        assertEquals(6, reader.getNextEntryId());
+        assertFalse(reader.hasCaughtUpOnInprogress());
+
+        Utils.close(reader);
+    }
+
+    @Test(timeout = 60000)
+    public void testMaxPrefetchEntriesSmallSegment() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setNumPrefetchEntriesPerLogSegment(10);
+        confLocal.setMaxPrefetchEntriesPerLogSegment(20);
+        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
+        generateCompletedLogSegments(dlm, confLocal, 1, 5);
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(segments.size() + " log segments found, expected to be only one",
+                1, segments.size());
+
+        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
+        reader.start();
+
+        // wait for the read ahead entries to become available
+        while (reader.readAheadEntries.size() < (reader.getLastAddConfirmed() + 1)) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+
+        long txId = 1L;
+        long entryId = 0L;
+
+        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);
+        LogRecordWithDLSN record = entryReader.nextRecord();
+        while (null != record) {
+            if (!record.isControl()) {
+                DLMTestUtil.verifyLogRecord(record);
+                assertEquals(txId, record.getTransactionId());
+                ++txId;
+            }
+            DLSN dlsn = record.getDlsn();
+            assertEquals(1L, dlsn.getLogSegmentSequenceNo());
+            assertEquals(entryId, dlsn.getEntryId());
+            record = entryReader.nextRecord();
+        }
+        ++entryId;
+        assertEquals(2L, txId);
+        assertEquals(reader.getLastAddConfirmed(), reader.readAheadEntries.size());
+        assertEquals((reader.getLastAddConfirmed() + 1), reader.getNextEntryId());
+        assertFalse(reader.hasCaughtUpOnInprogress());
+
+        Utils.close(reader);
+    }
+
+    @Test(timeout = 60000)
+    public void testReadEntriesFromInprogressSegment() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setNumPrefetchEntriesPerLogSegment(20);
+        confLocal.setMaxPrefetchEntriesPerLogSegment(20);
+        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
+        AsyncLogWriter writer = createInprogressLogSegment(dlm, confLocal, 5);
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(segments.size() + " log segments found, expected to be only one",
+                1, segments.size());
+
+        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
+        reader.start();
+
+        long expectedLastAddConfirmed = 8L;
+        // wait until sending out all prefetch requests
+        while (reader.readAheadEntries.size() < expectedLastAddConfirmed + 2) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals(expectedLastAddConfirmed + 2, reader.getNextEntryId());
+
+        long txId = 1L;
+        long entryId = 0L;
+        while (true) {
+            Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+            LogRecordWithDLSN record = entryReader.nextRecord();
+            while (null != record) {
+                if (!record.isControl()) {
+                    DLMTestUtil.verifyLogRecord(record);
+                    assertEquals(txId, record.getTransactionId());
+                    ++txId;
+                }
+                DLSN dlsn = record.getDlsn();
+                assertEquals(1L, dlsn.getLogSegmentSequenceNo());
+                assertEquals(entryId, dlsn.getEntryId());
+                record = entryReader.nextRecord();
+            }
+            ++entryId;
+            if (entryId == expectedLastAddConfirmed + 1) {
+                break;
+            }
+        }
+        assertEquals(6L, txId);
+
+        Future<List<Entry.Reader>> nextReadFuture = reader.readNext(1);
+        // write another record to commit previous writes
+        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId)));
+        // the long poll will be satisfied
+        List<Entry.Reader> nextReadEntries = FutureUtils.result(nextReadFuture);
+        assertEquals(1, nextReadEntries.size());
+        assertTrue(reader.hasCaughtUpOnInprogress());
+        Entry.Reader entryReader = nextReadEntries.get(0);
+        LogRecordWithDLSN record = entryReader.nextRecord();
+        assertNotNull(record);
+        assertTrue(record.isControl());
+        assertNull(entryReader.nextRecord());
+        // once the read is advanced, we will prefetch next record
+        while (reader.getNextEntryId() <= entryId) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals(entryId + 2, reader.getNextEntryId());
+        assertEquals(1, reader.readAheadEntries.size());
+
+        Utils.close(reader);
+        Utils.close(writer);
+    }
+
+    @Test(timeout = 60000)
+    public void testReadEntriesOnStateChange() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setOutputBufferSize(0);
+        confLocal.setPeriodicFlushFrequencyMilliSeconds(0);
+        confLocal.setImmediateFlushEnabled(false);
+        confLocal.setNumPrefetchEntriesPerLogSegment(20);
+        confLocal.setMaxPrefetchEntriesPerLogSegment(20);
+        DistributedLogManager dlm = createNewDLM(confLocal, runtime.getMethodName());
+        AsyncLogWriter writer = createInprogressLogSegment(dlm, confLocal, 5);
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        assertEquals(segments.size() + " log segments found, expected to be only one",
+                1, segments.size());
+
+        BKLogSegmentEntryReader reader = createEntryReader(segments.get(0), 0, confLocal);
+        reader.start();
+
+        long expectedLastAddConfirmed = 8L;
+        // wait until sending out all prefetch requests
+        while (reader.readAheadEntries.size() < expectedLastAddConfirmed + 2) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals(expectedLastAddConfirmed + 2, reader.getNextEntryId());
+
+        long txId = 1L;
+        long entryId = 0L;
+        while (true) {
+            Entry.Reader entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+            LogRecordWithDLSN record = entryReader.nextRecord();
+            while (null != record) {
+                if (!record.isControl()) {
+                    DLMTestUtil.verifyLogRecord(record);
+                    assertEquals(txId, record.getTransactionId());
+                    ++txId;
+                }
+                DLSN dlsn = record.getDlsn();
+                assertEquals(1L, dlsn.getLogSegmentSequenceNo());
+                assertEquals(entryId, dlsn.getEntryId());
+                record = entryReader.nextRecord();
+            }
+            ++entryId;
+            if (entryId == expectedLastAddConfirmed + 1) {
+                break;
+            }
+        }
+        assertEquals(6L, txId);
+
+        Future<List<Entry.Reader>> nextReadFuture = reader.readNext(1);
+        // write another record to commit previous writes
+        FutureUtils.result(writer.write(DLMTestUtil.getLogRecordInstance(txId)));
+        // the long poll will be satisfied
+        List<Entry.Reader> nextReadEntries = FutureUtils.result(nextReadFuture);
+        assertEquals(1, nextReadEntries.size());
+        Entry.Reader entryReader = nextReadEntries.get(0);
+        LogRecordWithDLSN record = entryReader.nextRecord();
+        assertNotNull(record);
+        assertTrue(record.isControl());
+        assertNull(entryReader.nextRecord());
+        // once the read is advanced, we will prefetch next record
+        while (reader.getNextEntryId() <= entryId) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals(entryId + 2, reader.getNextEntryId());
+        assertEquals(1, reader.readAheadEntries.size());
+
+        // advance the entry id
+        ++entryId;
+        // close the writer, the write will be committed
+        Utils.close(writer);
+        entryReader = FutureUtils.result(reader.readNext(1)).get(0);
+        record = entryReader.nextRecord();
+        assertNotNull(record);
+        assertFalse(record.isControl());
+        assertNull(entryReader.nextRecord());
+        while (reader.getNextEntryId() <= entryId + 1) {
+            TimeUnit.MILLISECONDS.sleep(10);
+        }
+        assertEquals(entryId + 2, reader.getNextEntryId());
+        assertEquals(1, reader.readAheadEntries.size());
+
+        // get the new log segment
+        List<LogSegmentMetadata> newSegments = dlm.getLogSegments();
+        assertEquals(1, newSegments.size());
+        assertFalse(newSegments.get(0).isInProgress());
+        reader.onLogSegmentMetadataUpdated(newSegments.get(0));
+        // when reader received the new log segments. the outstanding long poll
+        // should be cancelled and end of log segment should be signaled correctly
+        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));
+            fail("Should reach end of log segment");
+        } catch (EndOfLogSegmentException eol) {
+            // expected
+        }
+        Utils.close(reader);
+    }
+
+}
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
new file mode 100644
index 0000000..f67de35
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStore.java
@@ -0,0 +1,327 @@
+/**
+ * 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.impl.metadata;
+
+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.TestZooKeeperClientBuilder;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+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.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;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Transaction;
+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 org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.List;
+
+import static org.apache.distributedlog.metadata.LogMetadata.*;
+import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.*;
+import static org.junit.Assert.*;
+
+/**
+ * Test {@link ZKLogStreamMetadataStore}
+ */
+public class TestZKLogStreamMetadataStore extends ZooKeeperClusterTestCase {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestZKLogStreamMetadataStore.class);
+
+    private final static int sessionTimeoutMs = 30000;
+
+    @Rule
+    public TestName testName = new TestName();
+
+    private ZooKeeperClient zkc;
+    private URI uri;
+
+    private static void createLog(ZooKeeperClient zk, URI uri, String logName, String logIdentifier)
+            throws Exception {
+        final String logRootPath = getLogRootPath(uri, logName, logIdentifier);
+        final String logSegmentsPath = logRootPath + LOGSEGMENTS_PATH;
+        final String maxTxIdPath = logRootPath + MAX_TXID_PATH;
+        final String lockPath = logRootPath + LOCK_PATH;
+        final String readLockPath = logRootPath + READ_LOCK_PATH;
+        final String versionPath = logRootPath + VERSION_PATH;
+        final String allocationPath = logRootPath + ALLOCATION_PATH;
+
+        Utils.zkCreateFullPathOptimistic(zk, logRootPath, new byte[0],
+                zk.getDefaultACL(), CreateMode.PERSISTENT);
+        Transaction txn = zk.get().transaction();
+        txn.create(logSegmentsPath, DLUtils.serializeLogSegmentSequenceNumber(
+                        DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO),
+                zk.getDefaultACL(), CreateMode.PERSISTENT);
+        txn.create(maxTxIdPath, DLUtils.serializeTransactionId(0L),
+                zk.getDefaultACL(), CreateMode.PERSISTENT);
+        txn.create(lockPath, DistributedLogConstants.EMPTY_BYTES,
+                zk.getDefaultACL(), CreateMode.PERSISTENT);
+        txn.create(readLockPath, DistributedLogConstants.EMPTY_BYTES,
+                zk.getDefaultACL(), CreateMode.PERSISTENT);
+        txn.create(versionPath, intToBytes(LAYOUT_VERSION),
+                zk.getDefaultACL(), CreateMode.PERSISTENT);
+        txn.create(allocationPath, DistributedLogConstants.EMPTY_BYTES,
+                zk.getDefaultACL(), CreateMode.PERSISTENT);
+        txn.commit();
+    }
+
+    @Before
+    public void setup() throws Exception {
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .name("zkc")
+                .uri(DLMTestUtil.createDLMURI(zkPort, "/"))
+                .sessionTimeoutMs(sessionTimeoutMs)
+                .build();
+        uri = DLMTestUtil.createDLMURI(zkPort, "");
+        try {
+            ZkUtils.createFullPathOptimistic(
+                    zkc.get(),
+                    uri.getPath(),
+                    new byte[0],
+                    ZooDefs.Ids.OPEN_ACL_UNSAFE,
+                    CreateMode.PERSISTENT);
+        } catch (KeeperException.NodeExistsException nee) {
+            logger.debug("The namespace uri already exists.");
+        }
+    }
+
+    @After
+    public void teardown() throws Exception {
+        zkc.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testCheckLogMetadataPathsWithAllocator() throws Exception {
+        String logRootPath = "/" + testName.getMethodName();
+        List<Versioned<byte[]>> metadatas =
+                FutureUtils.result(checkLogMetadataPaths(
+                        zkc.get(), logRootPath, true));
+        assertEquals("Should have 8 paths",
+                8, metadatas.size());
+        for (Versioned<byte[]> path : metadatas.subList(2, metadatas.size())) {
+            assertNull(path.getValue());
+            assertNull(path.getVersion());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testCheckLogMetadataPathsWithoutAllocator() throws Exception {
+        String logRootPath = "/" + testName.getMethodName();
+        List<Versioned<byte[]>> metadatas =
+                FutureUtils.result(checkLogMetadataPaths(
+                        zkc.get(), logRootPath, false));
+        assertEquals("Should have 7 paths",
+                7, metadatas.size());
+        for (Versioned<byte[]> path : metadatas.subList(2, metadatas.size())) {
+            assertNull(path.getValue());
+            assertNull(path.getVersion());
+        }
+    }
+
+    private void testCreateLogMetadataWithMissingPaths(URI uri,
+                                                       String logName,
+                                                       String logIdentifier,
+                                                       List<String> pathsToDelete,
+                                                       boolean ownAllocator,
+                                                       boolean createLogFirst)
+            throws Exception {
+        if (createLogFirst) {
+            createLog(zkc, uri, logName, logIdentifier);
+        }
+        // delete a path
+        for (String path : pathsToDelete) {
+            zkc.get().delete(path, -1);
+        }
+
+        LogMetadataForWriter logMetadata =
+                FutureUtils.result(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));
+
+        if (ownAllocator) {
+            assertEquals("Should have 8 paths : ownAllocator = " + ownAllocator,
+                    8, metadatas.size());
+        } else {
+            assertEquals("Should have 7 paths : ownAllocator = " + ownAllocator,
+                    7, metadatas.size());
+        }
+
+        for (Versioned<byte[]> metadata : metadatas) {
+            assertTrue(pathExists(metadata));
+            assertTrue(((ZkVersion) metadata.getVersion()).getZnodeVersion() >= 0);
+        }
+
+        Versioned<byte[]> logSegmentsData = logMetadata.getMaxLSSNData();
+
+        assertEquals(DistributedLogConstants.UNASSIGNED_LOGSEGMENT_SEQNO,
+                DLUtils.deserializeLogSegmentSequenceNumber(logSegmentsData.getValue()));
+
+        Versioned<byte[]> maxTxIdData = logMetadata.getMaxTxIdData();
+
+        assertEquals(0L, DLUtils.deserializeTransactionId(maxTxIdData.getValue()));
+
+        if (ownAllocator) {
+            Versioned<byte[]> allocationData = logMetadata.getAllocationData();
+            assertEquals(0, allocationData.getValue().length);
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogMetadataMissingLogSegmentsPath() throws Exception {
+        String logName = testName.getMethodName();
+        String logIdentifier = "<default>";
+        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
+        List<String> pathsToDelete = Lists.newArrayList(
+                logRootPath + LOGSEGMENTS_PATH);
+        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, false, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogMetadataMissingMaxTxIdPath() throws Exception {
+        String logName = testName.getMethodName();
+        String logIdentifier = "<default>";
+        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
+        List<String> pathsToDelete = Lists.newArrayList(
+                logRootPath + MAX_TXID_PATH);
+        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, false, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogMetadataMissingLockPath() throws Exception {
+        String logName = testName.getMethodName();
+        String logIdentifier = "<default>";
+        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
+        List<String> pathsToDelete = Lists.newArrayList(
+                logRootPath + LOCK_PATH);
+        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, false, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogMetadataMissingReadLockPath() throws Exception {
+        String logName = testName.getMethodName();
+        String logIdentifier = "<default>";
+        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
+        List<String> pathsToDelete = Lists.newArrayList(
+                logRootPath + READ_LOCK_PATH);
+        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, false, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogMetadataMissingVersionPath() throws Exception {
+        String logName = testName.getMethodName();
+        String logIdentifier = "<default>";
+        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
+        List<String> pathsToDelete = Lists.newArrayList(
+                logRootPath + VERSION_PATH);
+        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, false, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogMetadataMissingAllocatorPath() throws Exception {
+        URI uri = DLMTestUtil.createDLMURI(zkPort, "");
+        String logName = testName.getMethodName();
+        String logIdentifier = "<default>";
+        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
+        List<String> pathsToDelete = Lists.newArrayList(
+                logRootPath + ALLOCATION_PATH);
+        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, true, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogMetadataMissingAllPath() throws Exception {
+        String logName = testName.getMethodName();
+        String logIdentifier = "<default>";
+        String logRootPath = getLogRootPath(uri, logName, logIdentifier);
+        List<String> pathsToDelete = Lists.newArrayList(
+                logRootPath + LOGSEGMENTS_PATH,
+                logRootPath + MAX_TXID_PATH,
+                logRootPath + LOCK_PATH,
+                logRootPath + READ_LOCK_PATH,
+                logRootPath + VERSION_PATH,
+                logRootPath + ALLOCATION_PATH);
+        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, true, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogMetadataOnExistedLog() throws Exception {
+        String logName = testName.getMethodName();
+        String logIdentifier = "<default>";
+        List<String> pathsToDelete = Lists.newArrayList();
+        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, true, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogMetadata() throws Exception {
+        String logName = testName.getMethodName();
+        String logIdentifier = "<default>";
+        List<String> pathsToDelete = Lists.newArrayList();
+
+        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, true, false);
+    }
+
+    @Test(timeout = 60000, expected = LogNotFoundException.class)
+    public void testCreateLogMetadataWithCreateIfNotExistsSetToFalse() throws Exception {
+        String logName = testName.getMethodName();
+        String logIdentifier = "<default>";
+        FutureUtils.result(getLog(uri, logName, logIdentifier, zkc, true, false));
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateLogMetadataWithCustomMetadata() throws Exception {
+        String logName = testName.getMethodName();
+        String logIdentifier = "<default>";
+        List<String> pathsToDelete = Lists.newArrayList();
+
+        DLMetadata.create(new BKDLConfig(zkServers, "/ledgers")).update(uri);
+
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+            .conf(new DistributedLogConfiguration())
+            .uri(uri)
+            .build();
+
+        MetadataAccessor accessor = namespace.getNamespaceDriver().getMetadataAccessor(logName);
+        accessor.createOrUpdateMetadata(logName.getBytes("UTF-8"));
+        accessor.close();
+
+        testCreateLogMetadataWithMissingPaths(uri, logName, logIdentifier, pathsToDelete, true, false);
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStoreUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStoreUtils.java
new file mode 100644
index 0000000..3631ae9
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZKLogStreamMetadataStoreUtils.java
@@ -0,0 +1,208 @@
+/**
+ * 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.impl.metadata;
+
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.distributedlog.metadata.LogMetadata;
+import org.apache.distributedlog.metadata.LogMetadataForWriter;
+import org.apache.distributedlog.util.DLUtils;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.junit.Test;
+
+import java.net.URI;
+import java.util.List;
+
+import static org.apache.distributedlog.impl.metadata.ZKLogStreamMetadataStore.*;
+import static org.junit.Assert.*;
+
+public class TestZKLogStreamMetadataStoreUtils {
+
+    @SuppressWarnings("unchecked")
+    @Test(timeout = 60000, expected = UnexpectedException.class)
+    public void testProcessLogMetadatasMissingMaxTxnId() throws Exception {
+        String rootPath = "/test-missing-max-txn-id";
+        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
+        String logName = "test-log";
+        String logIdentifier = "<default>";
+        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(null, null));
+        processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test(timeout = 60000, expected = UnexpectedException.class)
+    public void testProcessLogMetadatasMissingVersion() throws Exception {
+        String rootPath = "/test-missing-version";
+        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
+        String logName = "test-log";
+        String logIdentifier = "<default>";
+        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1)),
+                new Versioned<byte[]>(null, null));
+        processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test(timeout = 60000, expected = UnexpectedException.class)
+    public void testProcessLogMetadatasWrongVersion() throws Exception {
+        String rootPath = "/test-missing-version";
+        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
+        String logName = "test-log";
+        String logIdentifier = "<default>";
+        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1)),
+                new Versioned<byte[]>(intToBytes(9999), null));
+        processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test(timeout = 60000, expected = UnexpectedException.class)
+    public void testProcessLogMetadatasMissingLockPath() throws Exception {
+        String rootPath = "/test-missing-version";
+        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
+        String logName = "test-log";
+        String logIdentifier = "<default>";
+        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1)),
+                new Versioned<byte[]>(intToBytes(LogMetadata.LAYOUT_VERSION), null),
+                new Versioned<byte[]>(null, null));
+        processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test(timeout = 60000, expected = UnexpectedException.class)
+    public void testProcessLogMetadatasMissingReadLockPath() throws Exception {
+        String rootPath = "/test-missing-version";
+        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
+        String logName = "test-log";
+        String logIdentifier = "<default>";
+        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1)),
+                new Versioned<byte[]>(intToBytes(LogMetadata.LAYOUT_VERSION), null),
+                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
+                new Versioned<byte[]>(null, null));
+        processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test(timeout = 60000, expected = UnexpectedException.class)
+    public void testProcessLogMetadatasMissingLogSegmentsPath() throws Exception {
+        String rootPath = "/test-missing-version";
+        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
+        String logName = "test-log";
+        String logIdentifier = "<default>";
+        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1)),
+                new Versioned<byte[]>(intToBytes(LogMetadata.LAYOUT_VERSION), null),
+                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
+                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
+                new Versioned<byte[]>(null, null));
+        processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test(timeout = 60000, expected = UnexpectedException.class)
+    public void testProcessLogMetadatasMissingAllocatorPath() throws Exception {
+        String rootPath = "/test-missing-version";
+        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
+        String logName = "test-log";
+        String logIdentifier = "<default>";
+        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1)),
+                new Versioned<byte[]>(intToBytes(LogMetadata.LAYOUT_VERSION), null),
+                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
+                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
+                new Versioned<byte[]>(DLUtils.serializeLogSegmentSequenceNumber(1L), new ZkVersion(1)),
+                new Versioned<byte[]>(null, null));
+        processLogMetadatas(uri, logName, logIdentifier, metadatas, true);
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test(timeout = 60000)
+    public void testProcessLogMetadatasNoAllocatorPath() throws Exception {
+        String rootPath = "/test-missing-version";
+        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
+        String logName = "test-log";
+        String logIdentifier = "<default>";
+        Versioned<byte[]> maxTxnIdData =
+                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1));
+        Versioned<byte[]> logSegmentsData =
+                new Versioned<byte[]>(DLUtils.serializeLogSegmentSequenceNumber(1L), new ZkVersion(1));
+        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(null, null),
+                maxTxnIdData,
+                new Versioned<byte[]>(intToBytes(LogMetadata.LAYOUT_VERSION), null),
+                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
+                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
+                logSegmentsData);
+        LogMetadataForWriter metadata =
+                processLogMetadatas(uri, logName, logIdentifier, metadatas, false);
+        assertTrue(maxTxnIdData == metadata.getMaxTxIdData());
+        assertTrue(logSegmentsData == metadata.getMaxLSSNData());
+        assertNull(metadata.getAllocationData().getValue());
+        assertNull(metadata.getAllocationData().getVersion());
+    }
+
+    @SuppressWarnings("unchecked")
+    @Test(timeout = 60000)
+    public void testProcessLogMetadatasAllocatorPath() throws Exception {
+        String rootPath = "/test-missing-version";
+        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
+        String logName = "test-log";
+        String logIdentifier = "<default>";
+        Versioned<byte[]> maxTxnIdData =
+                new Versioned<byte[]>(DLUtils.serializeTransactionId(1L), new ZkVersion(1));
+        Versioned<byte[]> logSegmentsData =
+                new Versioned<byte[]>(DLUtils.serializeLogSegmentSequenceNumber(1L), new ZkVersion(1));
+        Versioned<byte[]> allocationData =
+                new Versioned<byte[]>(DLUtils.logSegmentId2Bytes(1L), new ZkVersion(1));
+        List<Versioned<byte[]>> metadatas = Lists.newArrayList(
+                new Versioned<byte[]>(null, null),
+                new Versioned<byte[]>(null, null),
+                maxTxnIdData,
+                new Versioned<byte[]>(intToBytes(LogMetadata.LAYOUT_VERSION), null),
+                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
+                new Versioned<byte[]>(new byte[0], new ZkVersion(1)),
+                logSegmentsData,
+                allocationData);
+        LogMetadataForWriter metadata =
+                processLogMetadatas(uri, logName, logIdentifier, metadatas, true);
+        assertTrue(maxTxnIdData == metadata.getMaxTxIdData());
+        assertTrue(logSegmentsData == metadata.getMaxLSSNData());
+        assertTrue(allocationData == metadata.getAllocationData());
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZkMetadataResolver.java b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZkMetadataResolver.java
new file mode 100644
index 0000000..24ff0da
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/impl/metadata/TestZkMetadataResolver.java
@@ -0,0 +1,203 @@
+/**
+ * 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.impl.metadata;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.TestZooKeeperClientBuilder;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.impl.metadata.ZkMetadataResolver;
+import org.apache.distributedlog.metadata.DLMetadata;
+import org.apache.distributedlog.util.Utils;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.ZooKeeperClusterTestCase;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestZkMetadataResolver extends ZooKeeperClusterTestCase {
+
+    private static final BKDLConfig bkdlConfig = new BKDLConfig("127.0.0.1:7000", "ledgers");
+    private static final BKDLConfig bkdlConfig2 = new BKDLConfig("127.0.0.1:7000", "ledgers2");
+
+    private ZooKeeperClient zkc;
+    private ZkMetadataResolver resolver;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .uri(createURI("/"))
+                .sessionTimeoutMs(10000)
+                .build();
+        resolver = new ZkMetadataResolver(zkc);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        zkc.close();
+    }
+
+    private URI createURI(String path) {
+        return URI.create("distributedlog://127.0.0.1:" + zkPort + path);
+    }
+
+    @Test(timeout = 60000)
+    public void testResolveFailures() throws Exception {
+        // resolve unexisted path
+        try {
+            resolver.resolve(createURI("/unexisted/path"));
+            fail("Should fail if no metadata resolved.");
+        } catch (IOException e) {
+            // expected
+        }
+        // resolve existed unbound path
+        Utils.zkCreateFullPathOptimistic(zkc, "/existed/path", new byte[0],
+                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        try {
+            resolver.resolve(createURI("/existed/path"));
+            fail("Should fail if no metadata resolved.");
+        } catch (IOException e) {
+            // expected
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testResolve() throws Exception {
+        DLMetadata dlMetadata = DLMetadata.create(bkdlConfig);
+        dlMetadata.create(createURI("/messaging/distributedlog-testresolve"));
+        DLMetadata dlMetadata2 = DLMetadata.create(bkdlConfig2);
+        dlMetadata2.create(createURI("/messaging/distributedlog-testresolve/child"));
+        assertEquals(dlMetadata,
+                resolver.resolve(createURI("/messaging/distributedlog-testresolve")));
+        assertEquals(dlMetadata2,
+                resolver.resolve(createURI("/messaging/distributedlog-testresolve/child")));
+        assertEquals(dlMetadata2,
+                resolver.resolve(createURI("/messaging/distributedlog-testresolve/child/unknown")));
+        Utils.zkCreateFullPathOptimistic(zkc, "/messaging/distributedlog-testresolve/child/child2", new byte[0],
+                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        assertEquals(dlMetadata2,
+                resolver.resolve(createURI("/messaging/distributedlog-testresolve/child/child2")));
+    }
+
+    @Test(timeout = 60000)
+    public void testEncodeRegionID() throws Exception {
+        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
+
+        URI uri = createURI("/messaging/distributedlog-testencoderegionid/dl1");
+        DLMetadata meta1 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers"));
+        meta1.create(uri);
+        BKDLConfig read1 = BKDLConfig.resolveDLConfig(zkc, uri);
+        BKDLConfig.propagateConfiguration(read1, dlConf);
+        assertFalse(dlConf.getEncodeRegionIDInLogSegmentMetadata());
+
+        BKDLConfig.clearCachedDLConfigs();
+
+        DLMetadata meta2 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers").setEncodeRegionID(true));
+        meta2.update(uri);
+        BKDLConfig read2 = BKDLConfig.resolveDLConfig(zkc, uri);
+        BKDLConfig.propagateConfiguration(read2, dlConf);
+        assertTrue(dlConf.getEncodeRegionIDInLogSegmentMetadata());
+
+        BKDLConfig.clearCachedDLConfigs();
+
+        DLMetadata meta3 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers").setEncodeRegionID(false));
+        meta3.update(uri);
+        BKDLConfig read3 = BKDLConfig.resolveDLConfig(zkc, uri);
+        BKDLConfig.propagateConfiguration(read3, dlConf);
+        assertFalse(dlConf.getEncodeRegionIDInLogSegmentMetadata());
+
+        BKDLConfig.clearCachedDLConfigs();
+    }
+
+    @Test(timeout = 60000)
+    public void testFirstLogSegmentSequenceNumber() throws Exception {
+        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
+
+        URI uri = createURI("/messaging/distributedlog-testfirstledgerseqno/dl1");
+        DLMetadata meta1 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers"));
+        meta1.create(uri);
+        BKDLConfig read1 = BKDLConfig.resolveDLConfig(zkc, uri);
+        BKDLConfig.propagateConfiguration(read1, dlConf);
+        assertEquals(DistributedLogConstants.FIRST_LOGSEGMENT_SEQNO, dlConf.getFirstLogSegmentSequenceNumber());
+
+        BKDLConfig.clearCachedDLConfigs();
+
+        DLMetadata meta2 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers")
+                .setFirstLogSegmentSeqNo(9999L));
+        meta2.update(uri);
+        BKDLConfig read2 = BKDLConfig.resolveDLConfig(zkc, uri);
+        BKDLConfig.propagateConfiguration(read2, dlConf);
+        assertEquals(9999L, dlConf.getFirstLogSegmentSequenceNumber());
+
+        BKDLConfig.clearCachedDLConfigs();
+
+        DLMetadata meta3 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers")
+                .setFirstLogSegmentSeqNo(99L));
+        meta3.update(uri);
+        BKDLConfig read3 = BKDLConfig.resolveDLConfig(zkc, uri);
+        BKDLConfig.propagateConfiguration(read3, dlConf);
+        assertEquals(99L, dlConf.getFirstLogSegmentSequenceNumber());
+
+        BKDLConfig.clearCachedDLConfigs();
+    }
+
+    @Test(timeout = 60000)
+    public void testFederatedNamespace() throws Exception {
+        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
+
+        URI uri = createURI("/messaging/distributedlog-testfederatednamespace/dl1");
+        DLMetadata meta1 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers"));
+        meta1.create(uri);
+        BKDLConfig read1 = BKDLConfig.resolveDLConfig(zkc, uri);
+        BKDLConfig.propagateConfiguration(read1, dlConf);
+        assertTrue(dlConf.getCreateStreamIfNotExists());
+
+        BKDLConfig.clearCachedDLConfigs();
+
+        DLMetadata meta2 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers")
+                .setFederatedNamespace(true));
+        meta2.update(uri);
+        BKDLConfig read2 = BKDLConfig.resolveDLConfig(zkc, uri);
+        BKDLConfig.propagateConfiguration(read2, dlConf);
+        assertFalse(dlConf.getCreateStreamIfNotExists());
+
+        BKDLConfig.clearCachedDLConfigs();
+
+        DLMetadata meta3 = DLMetadata.create(new BKDLConfig("127.0.0.1:7000", "ledgers")
+                .setFederatedNamespace(false));
+        meta3.update(uri);
+        BKDLConfig read3 = BKDLConfig.resolveDLConfig(zkc, uri);
+        BKDLConfig.propagateConfiguration(read3, dlConf);
+        // if it is non-federated namespace, it won't change the create stream behavior.
+        assertFalse(dlConf.getCreateStreamIfNotExists());
+
+        BKDLConfig.clearCachedDLConfigs();
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/limiter/TestRequestLimiter.java b/distributedlog-core/src/test/java/org/apache/distributedlog/limiter/TestRequestLimiter.java
new file mode 100644
index 0000000..c368ca8
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/limiter/TestRequestLimiter.java
@@ -0,0 +1,60 @@
+/**
+ * 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.limiter;
+
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestRequestLimiter {
+
+    class MockRequest {
+    }
+
+    class MockRequestLimiter implements RequestLimiter<MockRequest> {
+        int count;
+        MockRequestLimiter() {
+            this.count = 0;
+        }
+        public void apply(MockRequest request) {
+            count++;
+        }
+        public int getCount() {
+            return count;
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testChainedRequestLimiter() throws Exception {
+        MockRequestLimiter limiter1 = new MockRequestLimiter();
+        MockRequestLimiter limiter2 = new MockRequestLimiter();
+        ChainedRequestLimiter.Builder<MockRequest> limiterBuilder =
+                new ChainedRequestLimiter.Builder<MockRequest>();
+        limiterBuilder.addLimiter(limiter1)
+                      .addLimiter(limiter2);
+        ChainedRequestLimiter<MockRequest> limiter = limiterBuilder.build();
+        assertEquals(0, limiter1.getCount());
+        assertEquals(0, limiter2.getCount());
+        limiter.apply(new MockRequest());
+        assertEquals(1, limiter1.getCount());
+        assertEquals(1, limiter2.getCount());
+        limiter.apply(new MockRequest());
+        assertEquals(2, limiter1.getCount());
+        assertEquals(2, limiter2.getCount());
+    }
+}
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
new file mode 100644
index 0000000..b2eee34
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestDistributedLock.java
@@ -0,0 +1,835 @@
+/**
+ * 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.lock;
+
+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.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;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+
+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.apache.distributedlog.lock.ZKSessionLock.asyncParseClientID;
+
+/**
+ * Distributed Lock Tests
+ */
+public class TestDistributedLock extends TestDistributedLogBase {
+
+    static final Logger logger = LoggerFactory.getLogger(TestDistributedLock.class);
+
+    @Rule
+    public TestName runtime = new TestName();
+
+    private final static int sessionTimeoutMs = 2000;
+
+    private ZooKeeperClient zkc;
+    private ZooKeeperClient zkc0; // used for checking
+    private OrderedScheduler lockStateExecutor;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = ZooKeeperClientBuilder.newBuilder()
+                .name("zkc")
+                .uri(createDLMURI("/"))
+                .sessionTimeoutMs(sessionTimeoutMs)
+                .zkAclId(null)
+                .build();
+        zkc0 = ZooKeeperClientBuilder.newBuilder()
+                .name("zkc0")
+                .uri(createDLMURI("/"))
+                .sessionTimeoutMs(sessionTimeoutMs)
+                .zkAclId(null)
+                .build();
+        lockStateExecutor = OrderedScheduler.newBuilder()
+                .name("test-scheduer")
+                .corePoolSize(1)
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        zkc.close();
+        zkc0.close();
+        lockStateExecutor.shutdown();
+    }
+
+    private static void createLockPath(ZooKeeper zk, String lockPath) throws Exception {
+        zk.create(lockPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+    }
+
+    private static List<String> getLockWaiters(ZooKeeperClient zkc, String lockPath) throws Exception {
+        List<String> children = zkc.get().getChildren(lockPath, false);
+        Collections.sort(children, ZKSessionLock.MEMBER_COMPARATOR);
+        return children;
+    }
+
+    static class TestLockFactory {
+        final String lockPath;
+        final String clientId;
+        final OrderedScheduler lockStateExecutor;
+
+        public TestLockFactory(String name,
+                               ZooKeeperClient defaultZkc,
+                               OrderedScheduler lockStateExecutor)
+                throws Exception {
+            this.lockPath = "/" + name + System.currentTimeMillis();
+            this.clientId = name;
+            createLockPath(defaultZkc.get(), lockPath);
+            this.lockStateExecutor = lockStateExecutor;
+
+        }
+        public ZKDistributedLock createLock(int id, ZooKeeperClient zkc) throws Exception {
+            SessionLockFactory lockFactory = new ZKSessionLockFactory(
+                    zkc,
+                    clientId + id,
+                    lockStateExecutor,
+                    0,
+                    Long.MAX_VALUE,
+                    sessionTimeoutMs,
+                    NullStatsLogger.INSTANCE);
+            return new ZKDistributedLock(
+                    this.lockStateExecutor,
+                    lockFactory,
+                    this.lockPath,
+                    Long.MAX_VALUE,
+                    NullStatsLogger.INSTANCE);
+        }
+        public String getLockPath() {
+            return lockPath;
+        }
+    }
+
+    static class CountDownThrowFailPointAction extends FailpointUtils.AbstractFailPointAction {
+
+        final AtomicInteger successCounter;
+        final AtomicInteger failureCounter;
+
+        CountDownThrowFailPointAction(int successCount, int failureCount) {
+            this.successCounter = new AtomicInteger(successCount);
+            this.failureCounter = new AtomicInteger(failureCount);
+        }
+
+        @Override
+        public boolean checkFailPoint() throws IOException {
+            int successCount = successCounter.getAndDecrement();
+            if (successCount > 0) {
+                return true;
+            }
+            int count = failureCounter.getAndDecrement();
+            if (count > 0) {
+                throw new IOException("counter = " + count);
+            }
+            return true;
+        }
+    }
+
+    private SessionLockFactory createLockFactory(String clientId,
+                                                 ZooKeeperClient zkc) {
+        return createLockFactory(clientId, zkc, Long.MAX_VALUE, 0);
+    }
+    private SessionLockFactory createLockFactory(String clientId,
+                                                 ZooKeeperClient zkc,
+                                                 long lockTimeoutMs,
+                                                 int recreationTimes) {
+        return new ZKSessionLockFactory(
+                zkc,
+                clientId,
+                lockStateExecutor,
+                recreationTimes,
+                lockTimeoutMs,
+                sessionTimeoutMs,
+                NullStatsLogger.INSTANCE);
+    }
+
+    private static void checkLockAndReacquire(ZKDistributedLock lock, boolean sync) throws Exception {
+        lock.checkOwnershipAndReacquire();
+        Future<ZKDistributedLock> reacquireFuture = lock.getLockReacquireFuture();
+        if (null != reacquireFuture && sync) {
+            FutureUtils.result(reacquireFuture);
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testZooKeeperConnectionLossOnLockCreation() throws Exception {
+        String lockPath = "/test-zookeeper-connection-loss-on-lock-creation-" + System.currentTimeMillis();
+        String clientId = "zookeeper-connection-loss";
+
+        createLockPath(zkc.get(), lockPath);
+
+        FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss,
+                new CountDownThrowFailPointAction(0, Integer.MAX_VALUE));
+        SessionLockFactory lockFactory = createLockFactory(clientId, zkc, Long.MAX_VALUE, 0);
+        try {
+            try {
+                ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
+                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
+                FutureUtils.result(lock.asyncAcquire());
+                fail("Should fail on creating lock if couldn't establishing connections to zookeeper");
+            } catch (IOException ioe) {
+                // expected.
+            }
+        } finally {
+            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss);
+        }
+
+        FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss,
+                new CountDownThrowFailPointAction(0, Integer.MAX_VALUE));
+        lockFactory = createLockFactory(clientId, zkc, Long.MAX_VALUE, 3);
+        try {
+            try {
+                ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
+                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
+                FutureUtils.result(lock.asyncAcquire());
+                fail("Should fail on creating lock if couldn't establishing connections to zookeeper after 3 retries");
+            } catch (IOException ioe) {
+                // expected.
+            }
+        } finally {
+            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss);
+        }
+
+        FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss,
+                new CountDownThrowFailPointAction(0, 3));
+        lockFactory = createLockFactory(clientId, zkc, Long.MAX_VALUE, 5);
+        try {
+            ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
+                Long.MAX_VALUE, NullStatsLogger.INSTANCE);
+            FutureUtils.result(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))));
+
+            lock.asyncClose();
+        } finally {
+            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_ZooKeeperConnectionLoss);
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testBasicAcquireRelease() throws Exception {
+        String lockPath = "/test-basic-acquire-release-" + System.currentTimeMillis();
+        String clientId = "basic-acquire-release";
+
+        createLockPath(zkc.get(), lockPath);
+
+        SessionLockFactory lockFactory = createLockFactory(clientId, zkc);
+        ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
+                Long.MAX_VALUE, NullStatsLogger.INSTANCE);
+        FutureUtils.result(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))));
+
+        FutureUtils.result(lock.asyncClose());
+
+        children = getLockWaiters(zkc, lockPath);
+        assertEquals(0, children.size());
+        assertFalse(lock.haveLock());
+
+        lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
+                Long.MAX_VALUE, NullStatsLogger.INSTANCE);
+        FutureUtils.result(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(lockId1, lockId2);
+
+        FutureUtils.result(lock.asyncClose());
+
+        children = getLockWaiters(zkc, lockPath);
+        assertEquals(0, children.size());
+        assertFalse(lock.haveLock());
+
+        try {
+            FutureUtils.result(lock.asyncAcquire());
+            fail("Should fail on acquiring a closed lock");
+        } catch (UnexpectedException le) {
+            // expected.
+        }
+        children = getLockWaiters(zkc, lockPath);
+        assertEquals(0, children.size());
+        assertFalse(lock.haveLock());
+    }
+
+    @Test(timeout = 60000)
+    public void testCheckWriteLockFailureWhenLockIsAcquiredByOthers() throws Exception {
+        String lockPath = "/test-check-write-lock-failure-when-lock-is-acquired-by-others-" + System.currentTimeMillis();
+        String clientId = "test-check-write-lock-failure";
+
+        createLockPath(zkc.get(), lockPath);
+
+        SessionLockFactory lockFactory0 = createLockFactory(clientId, zkc0);
+        ZKDistributedLock lock0 =
+                new ZKDistributedLock(lockStateExecutor, lockFactory0, lockPath,
+                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
+        FutureUtils.result(lock0.asyncAcquire());
+
+        Pair<String, Long> lockId0_1 = ((ZKSessionLock) lock0.getInternalLock()).getLockId();
+
+        List<String> children = getLockWaiters(zkc, lockPath);
+        assertEquals(1, children.size());
+        assertTrue(lock0.haveLock());
+        assertEquals(lockId0_1,
+                Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+
+        // expire the session
+        ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
+
+        // reacquire the lock and wait reacquire completed
+        checkLockAndReacquire(lock0, true);
+
+        Pair<String, Long> lockId0_2 = ((ZKSessionLock) lock0.getInternalLock()).getLockId();
+        assertFalse("New lock should be created under different session", lockId0_1.equals(lockId0_2));
+
+        children = getLockWaiters(zkc, lockPath);
+        assertEquals(1, children.size());
+        assertTrue(lock0.haveLock());
+        assertEquals(lockId0_2,
+                Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+
+
+        SessionLockFactory lockFactory = createLockFactory(clientId, zkc);
+        final ZKDistributedLock lock1 =
+                new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
+                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
+        final CountDownLatch lockLatch = new CountDownLatch(1);
+        Thread lockThread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    FutureUtils.result(lock1.asyncAcquire());
+                    lockLatch.countDown();
+                } catch (IOException e) {
+                    logger.error("Failed on locking lock1 : ", e);
+                }
+            }
+        }, "lock-thread");
+        lockThread.start();
+
+        // ensure lock1 is waiting for lock0
+        do {
+            Thread.sleep(1);
+            children = getLockWaiters(zkc, lockPath);
+        } while (children.size() < 2);
+
+        // expire the session
+        ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
+
+        lockLatch.await();
+        lockThread.join();
+
+        try {
+            checkLockAndReacquire(lock0, true);
+            fail("Should fail on checking write lock since lock is acquired by lock1");
+        } catch (LockingException le) {
+            // expected
+        }
+
+        try {
+            checkLockAndReacquire(lock0, false);
+            fail("Should fail on checking write lock since lock is acquired by lock1");
+        } catch (LockingException le) {
+            // expected
+        }
+    }
+
+    /**
+     * If no lock is acquired between session expired and re-acquisition, check write lock will acquire the lock.
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testLockReacquireSuccessAfterCheckWriteLock() throws Exception {
+        testLockReacquireSuccess(true);
+    }
+
+    /**
+     * If no lock is acquired between session expired and re-acquisition, check write lock will acquire the lock.
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testLockReacquireSuccessWithoutCheckWriteLock() throws Exception {
+        testLockReacquireSuccess(false);
+    }
+
+    private void testLockReacquireSuccess(boolean checkOwnershipAndReacquire) throws Exception {
+        String lockPath = "/test-lock-re-acquire-success-" + checkOwnershipAndReacquire
+                + "-" + System.currentTimeMillis();
+        String clientId = "test-lock-re-acquire";
+
+        createLockPath(zkc.get(), lockPath);
+
+        SessionLockFactory lockFactory0 = createLockFactory(clientId, zkc0);
+        ZKDistributedLock lock0 =
+                new ZKDistributedLock(lockStateExecutor, lockFactory0, lockPath,
+                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
+        FutureUtils.result(lock0.asyncAcquire());
+
+        Pair<String, Long> lockId0_1 = ((ZKSessionLock) lock0.getInternalLock()).getLockId();
+
+        List<String> children = getLockWaiters(zkc, lockPath);
+        assertEquals(1, children.size());
+        assertTrue(lock0.haveLock());
+        assertEquals(lockId0_1,
+                Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+
+        ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
+
+        if (checkOwnershipAndReacquire) {
+            checkLockAndReacquire(lock0, true);
+            checkLockAndReacquire(lock0, false);
+        } else {
+            // session expire will trigger lock re-acquisition
+            Future<ZKDistributedLock> asyncLockAcquireFuture;
+            do {
+                Thread.sleep(1);
+                asyncLockAcquireFuture = lock0.getLockReacquireFuture();
+            } while (null == asyncLockAcquireFuture && lock0.getReacquireCount() < 1);
+            if (null != asyncLockAcquireFuture) {
+                Await.result(asyncLockAcquireFuture);
+            }
+            checkLockAndReacquire(lock0, false);
+        }
+        children = getLockWaiters(zkc, lockPath);
+        assertEquals(1, children.size());
+        assertTrue(lock0.haveLock());
+        Pair<String, Long> lock0_2 = ((ZKSessionLock) lock0.getInternalLock()).getLockId();
+        assertEquals(lock0_2,
+                Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+        assertEquals(clientId, lock0_2.getLeft());
+        assertFalse(lockId0_1.equals(lock0_2));
+
+        FutureUtils.result(lock0.asyncClose());
+
+        children = getLockWaiters(zkc, lockPath);
+        assertEquals(0, children.size());
+    }
+
+    /**
+     * If lock is acquired between session expired and re-acquisition, check write lock will be failed.
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testLockReacquireFailureAfterCheckWriteLock() throws Exception {
+        testLockReacquireFailure(true);
+    }
+
+    /**
+     * If lock is acquired between session expired and re-acquisition, check write lock will be failed.
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testLockReacquireFailureWithoutCheckWriteLock() throws Exception {
+        testLockReacquireFailure(false);
+    }
+
+    private void testLockReacquireFailure(boolean checkOwnershipAndReacquire) throws Exception {
+        String lockPath = "/test-lock-re-acquire-failure-" + checkOwnershipAndReacquire
+                + "-" + System.currentTimeMillis();
+        String clientId = "test-lock-re-acquire";
+
+        createLockPath(zkc.get(), lockPath);
+
+        SessionLockFactory lockFactory0 = createLockFactory(clientId, zkc0);
+        ZKDistributedLock lock0 =
+                new ZKDistributedLock(lockStateExecutor, lockFactory0, lockPath,
+                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
+        FutureUtils.result(lock0.asyncAcquire());
+
+        final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
+        SessionLockFactory lockFactory1 = createLockFactory(clientId, zkc);
+        final ZKDistributedLock lock1 =
+                new ZKDistributedLock(lockStateExecutor, lockFactory1, lockPath,
+                        Long.MAX_VALUE, NullStatsLogger.INSTANCE);
+        Thread lock1Thread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    FutureUtils.result(lock1.asyncAcquire());
+                    lock1DoneLatch.countDown();
+                } catch (IOException e) {
+                    logger.error("Error on acquiring lock1 : ", e);
+                }
+            }
+        }, "lock1-thread");
+        lock1Thread.start();
+
+        List<String> children;
+        do {
+            Thread.sleep(1);
+            children = getLockWaiters(zkc, lockPath);
+        } while (children.size() < 2);
+        assertEquals(2, children.size());
+        assertTrue(lock0.haveLock());
+        assertFalse(lock1.haveLock());
+        assertEquals(((ZKSessionLock) lock0.getInternalLock()).getLockId(),
+                Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId(),
+                Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
+
+        logger.info("Expiring session on lock0");
+        ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
+        logger.info("Session on lock0 is expired");
+        lock1DoneLatch.await();
+        assertFalse(lock0.haveLock());
+        assertTrue(lock1.haveLock());
+
+        if (checkOwnershipAndReacquire) {
+            try {
+                checkLockAndReacquire(lock0, true);
+                fail("Should fail check write lock since lock is already held by other people");
+            } catch (OwnershipAcquireFailedException oafe) {
+                assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId().getLeft(),
+                        oafe.getCurrentOwner());
+            }
+            try {
+                checkLockAndReacquire(lock0, false);
+                fail("Should fail check write lock since lock is already held by other people");
+            } catch (OwnershipAcquireFailedException oafe) {
+                assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId().getLeft(),
+                        oafe.getCurrentOwner());
+            }
+        } else {
+            logger.info("Waiting lock0 to attempt acquisition after session expired");
+            // session expire will trigger lock re-acquisition
+            Future<ZKDistributedLock> asyncLockAcquireFuture;
+            do {
+                Thread.sleep(1);
+                asyncLockAcquireFuture = lock0.getLockReacquireFuture();
+            } while (null == asyncLockAcquireFuture);
+
+            try {
+                Await.result(asyncLockAcquireFuture);
+                fail("Should fail check write lock since lock is already held by other people");
+            } catch (OwnershipAcquireFailedException oafe) {
+                assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId().getLeft(),
+                        oafe.getCurrentOwner());
+            }
+            try {
+                checkLockAndReacquire(lock0, false);
+                fail("Should fail check write lock since lock is already held by other people");
+            } catch (OwnershipAcquireFailedException oafe) {
+                assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId().getLeft(),
+                        oafe.getCurrentOwner());
+            }
+        }
+        children = getLockWaiters(zkc, lockPath);
+        assertEquals(1, children.size());
+        assertFalse(lock0.haveLock());
+        assertTrue(lock1.haveLock());
+        assertEquals(((ZKSessionLock) lock1.getInternalLock()).getLockId(),
+                Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(0))));
+
+        FutureUtils.result(lock0.asyncClose());
+        FutureUtils.result(lock1.asyncClose());
+
+        children = getLockWaiters(zkc, lockPath);
+        assertEquals(0, children.size());
+    }
+
+    @Test(timeout = 60000)
+    public void testLockReacquire() throws Exception {
+        String lockPath = "/reacquirePath";
+        Utils.zkCreateFullPathOptimistic(zkc, lockPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE,
+                CreateMode.PERSISTENT);
+        String clientId = "lockHolder";
+        SessionLockFactory lockFactory = createLockFactory(clientId, zkc, conf.getLockTimeoutMilliSeconds(), 0);
+        ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
+            conf.getLockTimeoutMilliSeconds(), NullStatsLogger.INSTANCE);
+        FutureUtils.result(lock.asyncAcquire());
+
+        // try and cleanup the underlying lock
+        lock.getInternalLock().unlock();
+
+        // This should reacquire the lock
+        checkLockAndReacquire(lock, true);
+
+        assertEquals(true, lock.haveLock());
+        assertEquals(true, lock.getInternalLock().isLockHeld());
+
+        lockFactory = createLockFactory(clientId + "_2", zkc, conf.getLockTimeoutMilliSeconds(), 0);
+        ZKDistributedLock lock2 = new ZKDistributedLock(lockStateExecutor, lockFactory, lockPath,
+            0, NullStatsLogger.INSTANCE);
+
+        boolean exceptionEncountered = false;
+        try {
+            FutureUtils.result(lock2.asyncAcquire());
+        } catch (OwnershipAcquireFailedException exc) {
+            assertEquals(clientId, exc.getCurrentOwner());
+            exceptionEncountered = true;
+        }
+        assertTrue(exceptionEncountered);
+        FutureUtils.result(lock.asyncClose());
+        FutureUtils.result(lock2.asyncClose());
+    }
+
+    @Test(timeout = 60000)
+    public void testLockReacquireMultiple() throws Exception {
+        String lockPath = "/reacquirePathMultiple";
+        Utils.zkCreateFullPathOptimistic(zkc, lockPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE,
+            CreateMode.PERSISTENT);
+        String clientId = "lockHolder";
+        SessionLockFactory factory = createLockFactory(clientId, zkc, conf.getLockTimeoutMilliSeconds(), 0);
+        ZKDistributedLock lock = new ZKDistributedLock(lockStateExecutor, factory, lockPath,
+            conf.getLockTimeoutMilliSeconds(), NullStatsLogger.INSTANCE);
+        FutureUtils.result(lock.asyncAcquire());
+
+        // try and cleanup the underlying lock
+        lock.getInternalLock().unlock();
+
+        // This should reacquire the lock
+        checkLockAndReacquire(lock, true);
+
+        assertEquals(true, lock.haveLock());
+        assertEquals(true, lock.getInternalLock().isLockHeld());
+
+        factory = createLockFactory(clientId + "_2", zkc, 0, 0);
+        ZKDistributedLock lock2 = new ZKDistributedLock(lockStateExecutor, factory, lockPath,
+            0, NullStatsLogger.INSTANCE);
+
+        boolean exceptionEncountered = false;
+        try {
+            FutureUtils.result(lock2.asyncAcquire());
+        } catch (OwnershipAcquireFailedException exc) {
+            assertEquals(clientId, exc.getCurrentOwner());
+            exceptionEncountered = true;
+        }
+        assertTrue(exceptionEncountered);
+        FutureUtils.result(lock2.asyncClose());
+
+        FutureUtils.result(lock.asyncClose());
+        assertEquals(false, lock.haveLock());
+        assertEquals(false, lock.getInternalLock().isLockHeld());
+
+        factory = createLockFactory(clientId + "_3", zkc, 0, 0);
+        ZKDistributedLock lock3 = new ZKDistributedLock(lockStateExecutor, factory, lockPath,
+            0, NullStatsLogger.INSTANCE);
+
+        FutureUtils.result(lock3.asyncAcquire());
+        assertEquals(true, lock3.haveLock());
+        assertEquals(true, lock3.getInternalLock().isLockHeld());
+        FutureUtils.result(lock3.asyncClose());
+    }
+
+    void assertLatchesSet(CountDownLatch[] latches, int endIndex) {
+        for (int i = 0; i < endIndex; i++) {
+            assertEquals("latch " + i + " should have been set", 0, latches[i].getCount());
+        }
+        for (int i = endIndex; i < latches.length; i++) {
+            assertEquals("latch " + i + " should not have been set", 1, latches[i].getCount());
+        }
+    }
+
+    // Assert key lock state (is locked, is internal locked, lock count, etc.) for two dlocks.
+    void assertLockState(ZKDistributedLock lock0, boolean owned0, boolean intOwned0,
+                         ZKDistributedLock lock1, boolean owned1, boolean intOwned1,
+                         int waiters, String lockPath) throws Exception {
+        assertEquals(owned0, lock0.haveLock());
+        assertEquals(intOwned0, lock0.getInternalLock() != null && lock0.getInternalLock().isLockHeld());
+        assertEquals(owned1, lock1.haveLock());
+        assertEquals(intOwned1, lock1.getInternalLock() != null && lock1.getInternalLock().isLockHeld());
+        assertEquals(waiters, getLockWaiters(zkc, lockPath).size());
+    }
+
+    @Test(timeout = 60000)
+    public void testAsyncAcquireBasics() throws Exception {
+        TestLockFactory locks = new TestLockFactory(runtime.getMethodName(), zkc, lockStateExecutor);
+
+        int count = 3;
+        ArrayList<Future<ZKDistributedLock>> results =
+                new ArrayList<Future<ZKDistributedLock>>(count);
+        ZKDistributedLock[] lockArray = new ZKDistributedLock[count];
+        final CountDownLatch[] latches = new CountDownLatch[count];
+
+        // Set up <count> waiters, save async results, count down a latch when lock is acquired in
+        // the future.
+        for (int i = 0; i < count; i++) {
+            latches[i] = new CountDownLatch(1);
+            lockArray[i] = locks.createLock(i, zkc);
+            final int index = i;
+            results.add(lockArray[i].asyncAcquire().addEventListener(
+                new FutureEventListener<ZKDistributedLock>() {
+                    @Override
+                    public void onSuccess(ZKDistributedLock lock) {
+                        latches[index].countDown();
+                    }
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        fail("unexpected failure " + cause);
+                    }
+                }
+            ));
+        }
+
+        // Now await ownership and release ownership of locks one by one (in the order they were
+        // acquired).
+        for (int i = 0; i < count; i++) {
+            latches[i].await();
+            assertLatchesSet(latches, i+1);
+            Await.result(results.get(i));
+            FutureUtils.result(lockArray[i].asyncClose());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testAsyncAcquireSyncThenAsyncOnSameLock() throws Exception {
+        TestLockFactory locks = new TestLockFactory(runtime.getMethodName(), zkc, lockStateExecutor);
+        final ZKDistributedLock lock0 = locks.createLock(0, zkc);
+        final ZKDistributedLock lock1 = locks.createLock(1, zkc0);
+
+        FutureUtils.result(lock0.asyncAcquire());
+
+        // Initial state.
+        assertLockState(lock0, true, true, lock1, false, false, 1, locks.getLockPath());
+
+        Thread lock1Thread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    FutureUtils.result(lock1.asyncAcquire());
+                } catch (IOException e) {
+                    fail("shouldn't fail to acquire");
+                }
+            }
+        }, "lock1-thread");
+        lock1Thread.start();
+
+        // Wait for lock count to increase, indicating background acquire has succeeded.
+        while (getLockWaiters(zkc, locks.getLockPath()).size() < 2) {
+            Thread.sleep(1);
+        }
+        assertLockState(lock0, true, true, lock1, false, false, 2, locks.getLockPath());
+
+        FutureUtils.result(lock0.asyncClose());
+        Await.result(lock1.getLockAcquireFuture());
+
+        assertLockState(lock0, false, false, lock1, true, true, 1, locks.getLockPath());
+
+        // Release lock1
+        FutureUtils.result(lock1.asyncClose());
+        assertLockState(lock0, false, false, lock1, false, false, 0, locks.getLockPath());
+    }
+
+    @Test(timeout = 60000)
+    public void testAsyncAcquireExpireDuringWait() throws Exception {
+        TestLockFactory locks = new TestLockFactory(runtime.getMethodName(), zkc, lockStateExecutor);
+        final ZKDistributedLock lock0 = locks.createLock(0, zkc);
+        final ZKDistributedLock lock1 = locks.createLock(1, zkc0);
+
+        FutureUtils.result(lock0.asyncAcquire());
+        Future<ZKDistributedLock> result = lock1.asyncAcquire();
+        // make sure we place a waiter for lock1
+        while (null == lock1.getLockWaiter()) {
+            TimeUnit.MILLISECONDS.sleep(20);
+        }
+
+        // Expire causes acquire future to be failed and unset.
+        ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
+        try {
+            Await.result(result);
+            fail("future should have been failed");
+        } catch (OwnershipAcquireFailedException ex) {
+        }
+
+        assertLockState(lock0, true, true, lock1, false, false, 1, locks.getLockPath());
+        lock0.asyncClose();
+        lock1.asyncClose();
+    }
+
+    @Test(timeout = 60000)
+    public void testAsyncAcquireCloseDuringWait() throws Exception {
+        TestLockFactory locks = new TestLockFactory(runtime.getMethodName(), zkc, lockStateExecutor);
+        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());
+        try {
+            Await.result(result);
+            fail("future should have been failed");
+        } catch (LockClosedException ex) {
+        }
+
+        assertLockState(lock0, true, true, lock1, false, false, 1, locks.getLockPath());
+        lock0.asyncClose();
+    }
+
+    @Test(timeout = 60000)
+    public void testAsyncAcquireCloseAfterAcquire() throws Exception {
+        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());
+
+        // Already have this, stays satisfied.
+        Await.result(result);
+
+        // But we no longer have the lock.
+        assertEquals(false, lock0.haveLock());
+        assertEquals(false, lock0.getInternalLock().isLockHeld());
+    }
+}
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
new file mode 100644
index 0000000..4d4a008
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/lock/TestZKSessionLock.java
@@ -0,0 +1,1224 @@
+/**
+ * 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.lock;
+
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClientBuilder;
+import org.apache.distributedlog.ZooKeeperClientUtils;
+import org.apache.distributedlog.ZooKeeperClusterTestCase;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+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.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs;
+import org.apache.zookeeper.ZooKeeper;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+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;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.apache.distributedlog.lock.ZKSessionLock.*;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+/**
+ * Distributed Lock Tests
+ */
+public class TestZKSessionLock extends ZooKeeperClusterTestCase {
+
+    @Rule
+    public TestName testNames = new TestName();
+
+    static final Logger logger = LoggerFactory.getLogger(TestZKSessionLock.class);
+
+    private final static int sessionTimeoutMs = 2000;
+
+    private ZooKeeperClient zkc;
+    private ZooKeeperClient zkc0; // used for checking
+    private OrderedScheduler lockStateExecutor;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = ZooKeeperClientBuilder.newBuilder()
+                .name("zkc")
+                .uri(DLMTestUtil.createDLMURI(zkPort, "/"))
+                .sessionTimeoutMs(sessionTimeoutMs)
+                .zkServers(zkServers)
+                .zkAclId(null)
+                .build();
+        zkc0 = ZooKeeperClientBuilder.newBuilder()
+                .name("zkc0")
+                .uri(DLMTestUtil.createDLMURI(zkPort, "/"))
+                .sessionTimeoutMs(sessionTimeoutMs)
+                .zkServers(zkServers)
+                .zkAclId(null)
+                .build();
+        lockStateExecutor = OrderedScheduler.newBuilder()
+                .corePoolSize(1)
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        zkc.close();
+        zkc0.close();
+        lockStateExecutor.shutdown();
+    }
+
+    private static void createLockPath(ZooKeeper zk, String lockPath) throws Exception {
+        zk.create(lockPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+    }
+
+    private static String createLockNodeV1(ZooKeeper zk, String lockPath, String clientId) throws Exception {
+        return zk.create(getLockPathPrefixV1(lockPath), serializeClientId(clientId),
+                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
+    }
+
+    private static String createLockNodeV2(ZooKeeper zk, String lockPath, String clientId) throws Exception {
+        return zk.create(getLockPathPrefixV2(lockPath, clientId), serializeClientId(clientId),
+                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
+    }
+
+    private static String createLockNodeV3(ZooKeeper zk, String lockPath, String clientId) throws Exception {
+        return zk.create(getLockPathPrefixV3(lockPath, clientId, zk.getSessionId()), serializeClientId(clientId),
+                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
+    }
+
+    private static String createLockNodeWithBadNodeName(ZooKeeper zk, String lockPath, String clientId, String badNodeName)
+            throws Exception {
+        return zk.create(lockPath + "/" + badNodeName, serializeClientId(clientId),
+                ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL);
+    }
+
+    private static List<String> getLockWaiters(ZooKeeperClient zkc, String lockPath) throws Exception {
+        List<String> children = zkc.get().getChildren(lockPath, false);
+        Collections.sort(children, ZKSessionLock.MEMBER_COMPARATOR);
+        return children;
+    }
+
+    @Test(timeout = 60000)
+    public void testParseClientID() throws Exception {
+        ZooKeeper zk = zkc.get();
+
+        String lockPath = "/test-parse-clientid";
+        String clientId = "test-parse-clientid-" + System.currentTimeMillis();
+        Pair<String, Long> lockId = Pair.of(clientId, zk.getSessionId());
+
+        createLockPath(zk, lockPath);
+
+        // Correct data
+        String node1 = getLockIdFromPath(createLockNodeV1(zk, lockPath, clientId));
+        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)));
+
+        // Bad Lock Node Name
+        String node4 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member"));
+        String node5 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode"));
+        String node6 = getLockIdFromPath(createLockNodeWithBadNodeName(zk, lockPath, clientId, "member_badnode_badnode"));
+        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)));
+
+        // 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)));
+    }
+
+    @Test(timeout = 60000)
+    public void testParseMemberID() throws Exception {
+        assertEquals(Integer.MAX_VALUE, parseMemberID("badnode"));
+        assertEquals(Integer.MAX_VALUE, parseMemberID("badnode_badnode"));
+        assertEquals(0, parseMemberID("member_000000"));
+        assertEquals(123, parseMemberID("member_000123"));
+    }
+
+    @Test(timeout = 60000)
+    public void testAreLockWaitersInSameSession() throws Exception {
+        ZooKeeper zk = zkc.get();
+
+        String lockPath = "/test-are-lock-waiters-in-same-session";
+        String clientId1 = "test-are-lock-waiters-in-same-session-1";
+        String clientId2 = "test-are-lock-waiters-in-same-session-2";
+
+        createLockPath(zk, lockPath);
+
+        String node1 = getLockIdFromPath(createLockNodeV3(zk, lockPath, clientId1));
+        String node2 = getLockIdFromPath(createLockNodeV3(zk, lockPath, clientId2));
+        String node3 = getLockIdFromPath(createLockNodeV3(zk, lockPath, clientId1));
+
+        assertEquals(node1 + " and " + node3 + " should be in same session.",
+                true, areLockWaitersInSameSession(node1, node3));
+        assertEquals(node1 + " and " + node2 + " should be not in same session.",
+                false, areLockWaitersInSameSession(node1, node2));
+        assertEquals(node3 + " and " + node2 + " should be not in same session.",
+                false, areLockWaitersInSameSession(node3, node2));
+    }
+
+    @Test(timeout = 60000)
+    public void testExecuteLockAction() throws Exception {
+        String lockPath = "/test-execute-lock-action";
+        String clientId = "test-execute-lock-action-" + System.currentTimeMillis();
+
+        ZKSessionLock lock =
+                new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
+
+        final AtomicInteger counter = new AtomicInteger(0);
+
+        // lock action would be executed in same epoch
+        final CountDownLatch latch1 = new CountDownLatch(1);
+        lock.executeLockAction(lock.getEpoch().get(), new LockAction() {
+            @Override
+            public void execute() {
+                counter.incrementAndGet();
+                latch1.countDown();
+            }
+
+            @Override
+            public String getActionName() {
+                return "increment1";
+            }
+        });
+        latch1.await();
+        assertEquals("counter should be increased in same epoch", 1, counter.get());
+
+        // lock action would not be executed in same epoch
+        final CountDownLatch latch2 = new CountDownLatch(1);
+        lock.executeLockAction(lock.getEpoch().get() + 1, new LockAction() {
+            @Override
+            public void execute() {
+                counter.incrementAndGet();
+            }
+
+            @Override
+            public String getActionName() {
+                return "increment2";
+            }
+        });
+        lock.executeLockAction(lock.getEpoch().get(), new LockAction() {
+            @Override
+            public void execute() {
+                latch2.countDown();
+            }
+
+            @Override
+            public String getActionName() {
+                return "countdown";
+            }
+        });
+        latch2.await();
+        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>();
+        lock.executeLockAction(lock.getEpoch().get() + 1, new LockAction() {
+            @Override
+            public void execute() {
+                counter.incrementAndGet();
+            }
+
+            @Override
+            public String getActionName() {
+                return "increment3";
+            }
+        }, promise);
+        try {
+            Await.result(promise);
+            fail("Should satisfy promise with epoch changed exception.");
+        } catch (EpochChangedException ece) {
+            // expected
+        }
+        assertEquals("counter should not be increased in different epochs", 1, counter.get());
+
+        lockStateExecutor.shutdown();
+    }
+
+    /**
+     * Test lock after unlock is called.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testLockAfterUnlock() throws Exception {
+        String lockPath = "/test-lock-after-unlock";
+        String clientId = "test-lock-after-unlock";
+
+        ZKSessionLock lock = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
+        lock.unlock();
+        assertEquals(State.CLOSED, lock.getLockState());
+
+        try {
+            lock.tryLock(0, TimeUnit.MILLISECONDS);
+            fail("Should fail on tryLock since lock state has changed.");
+        } catch (LockStateChangedException lsce) {
+            // expected
+        }
+        assertEquals(State.CLOSED, lock.getLockState());
+
+        try {
+            lock.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+            fail("Should fail on tryLock immediately if lock state has changed.");
+        } catch (LockStateChangedException lsce) {
+            // expected
+        }
+        assertEquals(State.CLOSED, lock.getLockState());
+    }
+
+    class DelayFailpointAction extends FailpointUtils.AbstractFailPointAction {
+        long timeout;
+        DelayFailpointAction(long timeout) {
+            this.timeout = timeout;
+        }
+        @Override
+        public boolean checkFailPoint() throws IOException {
+            try {
+                Thread.sleep(timeout);
+            } catch (InterruptedException ie) {
+            }
+            return true;
+        }
+    }
+
+    /**
+     * Test unlock timeout.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testUnlockTimeout() throws Exception {
+        String name = testNames.getMethodName();
+        String lockPath = "/" + name;
+        String clientId = name;
+
+        createLockPath(zkc.get(), lockPath);
+
+        ZKSessionLock lock = new ZKSessionLock(
+                zkc, lockPath, clientId, lockStateExecutor,
+                1*1000 /* op timeout */, NullStatsLogger.INSTANCE,
+                new DistributedLockContext());
+
+        lock.tryLock(0, TimeUnit.MILLISECONDS);
+        assertEquals(State.CLAIMED, lock.getLockState());
+
+        try {
+            FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_LockUnlockCleanup,
+                                        new DelayFailpointAction(60*60*1000));
+
+            lock.unlock();
+            assertEquals(State.CLOSING, lock.getLockState());
+        } finally {
+            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_LockUnlockCleanup);
+        }
+    }
+
+    /**
+     * Test try-create after close race condition.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testTryCloseRaceCondition() throws Exception {
+        String name = testNames.getMethodName();
+        String lockPath = "/" + name;
+        String clientId = name;
+
+        createLockPath(zkc.get(), lockPath);
+
+        ZKSessionLock lock = new ZKSessionLock(
+                zkc, lockPath, clientId, lockStateExecutor,
+                1*1000 /* op timeout */, NullStatsLogger.INSTANCE,
+                new DistributedLockContext());
+
+        try {
+            FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_LockTryCloseRaceCondition,
+                                        FailpointUtils.DEFAULT_ACTION);
+
+            lock.tryLock(0, TimeUnit.MILLISECONDS);
+        } catch (LockClosedException ex) {
+            ;
+        } finally {
+            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_LockTryCloseRaceCondition);
+        }
+
+        assertEquals(State.CLOSED, lock.getLockState());
+        List<String> children = getLockWaiters(zkc, lockPath);
+        assertEquals(0, children.size());
+    }
+
+    /**
+     * Test try acquire timeout.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testTryAcquireTimeout() throws Exception {
+        String name = testNames.getMethodName();
+        String lockPath = "/" + name;
+        String clientId = name;
+
+        createLockPath(zkc.get(), lockPath);
+
+        ZKSessionLock lock = new ZKSessionLock(
+                zkc, lockPath, clientId, lockStateExecutor,
+                1 /* op timeout */, NullStatsLogger.INSTANCE,
+                new DistributedLockContext());
+
+        try {
+            FailpointUtils.setFailpoint(FailpointUtils.FailPointName.FP_LockTryAcquire,
+                                        new DelayFailpointAction(60*60*1000));
+
+            lock.tryLock(0, TimeUnit.MILLISECONDS);
+            assertEquals(State.CLOSED, lock.getLockState());
+        } catch (LockingException le) {
+        } catch (Exception e) {
+            fail("expected locking exception");
+        } finally {
+            FailpointUtils.removeFailpoint(FailpointUtils.FailPointName.FP_LockTryAcquire);
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testBasicLockUnlock0() throws Exception {
+        testBasicLockUnlock(0);
+    }
+
+    @Test(timeout = 60000)
+    public void testBasicLockUnlock1() throws Exception {
+        testBasicLockUnlock(Long.MAX_VALUE);
+    }
+
+    /**
+     * Test Basic Lock and Unlock
+     *
+     * - lock should succeed if there is no lock held
+     * - lock should fail on a success lock
+     * - unlock should release the held lock
+     *
+     * @param timeout
+     *          timeout to wait for the lock
+     * @throws Exception
+     */
+    private void testBasicLockUnlock(long timeout) throws Exception {
+        String lockPath = "/test-basic-lock-unlock-" + timeout + System.currentTimeMillis();
+        String clientId = "test-basic-lock-unlock";
+
+        createLockPath(zkc.get(), lockPath);
+
+        ZKSessionLock lock = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
+        // lock
+        lock.tryLock(timeout, TimeUnit.MILLISECONDS);
+        // verification after lock
+        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))));
+
+        // lock should fail on a success lock
+        try {
+            lock.tryLock(timeout, TimeUnit.MILLISECONDS);
+            fail("Should fail on locking a failure lock.");
+        } catch (LockStateChangedException lsce) {
+            // expected
+        }
+        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))));
+
+        // unlock
+        lock.unlock();
+        // verification after unlock
+        assertEquals(State.CLOSED, lock.getLockState());
+        assertEquals(0, getLockWaiters(zkc, lockPath).size());
+    }
+
+    /**
+     * Test lock on non existed lock.
+     *
+     * - lock should fail on a non existed lock.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testLockOnNonExistedLock() throws Exception {
+        String lockPath = "/test-lock-on-non-existed-lock";
+        String clientId = "test-lock-on-non-existed-lock";
+
+        ZKSessionLock lock = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
+        // lock
+        try {
+            lock.tryLock(0, TimeUnit.MILLISECONDS);
+            fail("Should fail on locking a non-existed lock.");
+        } catch (LockingException le) {
+            Throwable cause = le.getCause();
+            assertTrue(cause instanceof KeeperException);
+            assertEquals(KeeperException.Code.NONODE, ((KeeperException) cause).code());
+        }
+        assertEquals(State.CLOSED, lock.getLockState());
+
+        // lock should failed on a failure lock
+        try {
+            lock.tryLock(0, TimeUnit.MILLISECONDS);
+            fail("Should fail on locking a failure lock.");
+        } catch (LockStateChangedException lsce) {
+            // expected
+        }
+        assertEquals(State.CLOSED, lock.getLockState());
+    }
+
+    @Test(timeout = 60000)
+    public void testLockWhenSomeoneHeldLock0() throws Exception {
+        testLockWhenSomeoneHeldLock(0);
+    }
+
+    @Test(timeout = 60000)
+    public void testLockWhenSomeoneHeldLock1() throws Exception {
+        testLockWhenSomeoneHeldLock(500);
+    }
+
+    /**
+     * Test lock if the lock is already held by someone else. Any lock in this situation will
+     * fail with current owner.
+     *
+     * @param timeout
+     *          timeout to wait for the lock
+     * @throws Exception
+     */
+    private void testLockWhenSomeoneHeldLock(long timeout) throws Exception {
+        String lockPath = "/test-lock-nowait-" + timeout + "-" + System.currentTimeMillis();
+        String clientId0 = "test-lock-nowait-0-" + System.currentTimeMillis();
+        String clientId1 = "test-lock-nowait-1-" + System.currentTimeMillis();
+        String clientId2 = "test-lock-nowait-2-" + System.currentTimeMillis();
+
+        createLockPath(zkc.get(), lockPath);
+
+        ZKSessionLock lock0 = new ZKSessionLock(zkc0, lockPath, clientId0, lockStateExecutor);
+        ZKSessionLock lock1 = new ZKSessionLock(zkc, lockPath, clientId1, lockStateExecutor);
+
+        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        // verification after lock0 lock
+        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))));
+
+        try {
+            lock1.tryLock(timeout, TimeUnit.MILLISECONDS);
+            fail("lock1 should fail on locking since lock0 is holding the lock.");
+        } catch (OwnershipAcquireFailedException oafe) {
+            assertEquals(lock0.getLockId().getLeft(), oafe.getCurrentOwner());
+        }
+        // verification after lock1 tryLock
+        assertEquals(State.CLAIMED, lock0.getLockState());
+        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))));
+
+        lock0.unlock();
+        // verification after unlock lock0
+        assertEquals(State.CLOSED, lock0.getLockState());
+        assertEquals(0, getLockWaiters(zkc, lockPath).size());
+
+        ZKSessionLock lock2 = new ZKSessionLock(zkc, lockPath, clientId2, lockStateExecutor);
+        lock2.tryLock(timeout, TimeUnit.MILLISECONDS);
+        // verification after lock2 lock
+        assertEquals(State.CLOSED, lock0.getLockState());
+        assertEquals(State.CLOSED, lock1.getLockState());
+        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))));
+
+        lock2.unlock();
+    }
+
+    @Test(timeout = 60000)
+    public void testLockWhenPreviousLockZnodeStillExists() throws Exception {
+        String lockPath = "/test-lock-when-previous-lock-znode-still-exists-" +
+                System.currentTimeMillis();
+        String clientId = "client-id";
+
+        ZooKeeper zk = zkc.get();
+
+        createLockPath(zk, lockPath);
+
+        final ZKSessionLock lock0 = new ZKSessionLock(zkc0, lockPath, clientId, lockStateExecutor);
+        // lock0 lock
+        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+
+        // simulate lock0 expires but znode still exists
+        final DistributedLockContext context1 = new DistributedLockContext();
+        context1.addLockId(lock0.getLockId());
+
+        final ZKSessionLock lock1 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor,
+                60000, NullStatsLogger.INSTANCE, context1);
+        lock1.tryLock(0L, TimeUnit.MILLISECONDS);
+        assertEquals(State.CLAIMED, lock1.getLockState());
+        lock1.unlock();
+
+        final DistributedLockContext context2 = new DistributedLockContext();
+        context2.addLockId(lock0.getLockId());
+
+        final ZKSessionLock lock2 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor,
+                60000, NullStatsLogger.INSTANCE, context2);
+        lock2.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        assertEquals(State.CLAIMED, lock2.getLockState());
+        lock2.unlock();
+
+        lock0.unlock();
+    }
+
+    @Test(timeout = 60000)
+    public void testWaitForLockUnlock() throws Exception {
+        testWaitForLockReleased("/test-wait-for-lock-unlock", true);
+    }
+
+    @Test(timeout = 60000)
+    public void testWaitForLockExpired() throws Exception {
+        testWaitForLockReleased("/test-wait-for-lock-expired", false);
+    }
+
+    /**
+     * Test lock wait for the lock owner to release the lock. The lock waiter should acquire lock successfully
+     * if the lock owner unlock or it is expired.
+     *
+     * @param lockPath
+     *          lock path
+     * @param isUnlock
+     *          whether to unlock or expire the lock
+     * @throws Exception
+     */
+    private void testWaitForLockReleased(String lockPath, boolean isUnlock) throws Exception {
+        String clientId0 = "test-wait-for-lock-released-0-" + System.currentTimeMillis();
+        String clientId1 = "test-wait-for-lock-released-1-" + System.currentTimeMillis();
+
+        createLockPath(zkc.get(), lockPath);
+
+        final ZKSessionLock lock0 = new ZKSessionLock(zkc0, lockPath, clientId0, lockStateExecutor);
+        final ZKSessionLock lock1 = new ZKSessionLock(zkc, lockPath, clientId1, lockStateExecutor);
+
+        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        // verification after lock0 lock
+        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))));
+
+        final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
+
+        Thread lock1Thread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    lock1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+                    lock1DoneLatch.countDown();
+                } catch (LockingException e) {
+                    logger.error("Failed on locking lock1 : ", e);
+                }
+            }
+        }, "lock1-thread");
+        lock1Thread.start();
+
+        // ensure lock1 is waiting for lock0
+        children = awaitWaiters(2, zkc, lockPath);
+
+        if (isUnlock) {
+            lock0.unlock();
+        } else {
+            ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
+        }
+
+        lock1DoneLatch.await();
+        lock1Thread.join();
+
+        // verification after lock2 lock
+        if (isUnlock) {
+            assertEquals(State.CLOSED, lock0.getLockState());
+        } else {
+            assertEquals(State.EXPIRED, lock0.getLockState());
+        }
+        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))));
+
+        lock1.unlock();
+    }
+
+    /**
+     * Test session expired after claimed the lock: lock state should be changed to expired and notify
+     * the lock listener about expiry.
+     *
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testLockListenerOnExpired() throws Exception {
+        String lockPath = "/test-lock-listener-on-expired";
+        String clientId = "test-lock-listener-on-expired-" + System.currentTimeMillis();
+
+        createLockPath(zkc.get(), lockPath);
+
+        final CountDownLatch expiredLatch = new CountDownLatch(1);
+        LockListener listener = new LockListener() {
+            @Override
+            public void onExpired() {
+                expiredLatch.countDown();
+            }
+        };
+        final ZKSessionLock lock =
+                new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor).setLockListener(listener);
+        lock.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        // verification after lock
+        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))));
+
+        ZooKeeperClientUtils.expireSession(zkc, zkServers, sessionTimeoutMs);
+        expiredLatch.await();
+        assertEquals(State.EXPIRED, lock.getLockState());
+        children = getLockWaiters(zkc, lockPath);
+        assertEquals(0, children.size());
+
+        try {
+            lock.tryLock(0, TimeUnit.MILLISECONDS);
+            fail("Should fail on tryLock since lock state has changed.");
+        } catch (LockStateChangedException lsce) {
+            // expected
+        }
+
+        lock.unlock();
+    }
+
+    @Test(timeout = 60000)
+    public void testSessionExpiredBeforeLock0() throws Exception {
+        testSessionExpiredBeforeLock(0);
+    }
+
+    @Test(timeout = 60000)
+    public void testSessionExpiredBeforeLock1() throws Exception {
+        testSessionExpiredBeforeLock(Long.MAX_VALUE);
+    }
+
+    /**
+     * Test Session Expired Before Lock does locking. The lock should be closed since
+     * all zookeeper operations would be failed.
+     *
+     * @param timeout
+     *          timeout to wait for the lock
+     * @throws Exception
+     */
+    private void testSessionExpiredBeforeLock(long timeout) throws Exception {
+        String lockPath = "/test-session-expired-before-lock-" + timeout + "-" + System.currentTimeMillis();
+        String clientId = "test-session-expired-before-lock-" + System.currentTimeMillis();
+
+        createLockPath(zkc.get(), lockPath);
+        final AtomicInteger expireCounter = new AtomicInteger(0);
+        final CountDownLatch expiredLatch = new CountDownLatch(1);
+        LockListener listener = new LockListener() {
+            @Override
+            public void onExpired() {
+                expireCounter.incrementAndGet();
+            }
+        };
+        final ZKSessionLock lock = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor)
+                .setLockListener(listener);
+        // expire session
+        ZooKeeperClientUtils.expireSession(zkc, zkServers, sessionTimeoutMs);
+        // submit a runnable to lock state executor to ensure any state changes happened when session expired
+        lockStateExecutor.submit(lockPath, new SafeRunnable() {
+            @Override
+            public void safeRun() {
+                expiredLatch.countDown();
+            }
+        });
+        expiredLatch.await();
+        // no watcher was registered if never acquired lock successfully
+        assertEquals(State.INIT, lock.getLockState());
+        try {
+            lock.tryLock(timeout, TimeUnit.MILLISECONDS);
+            fail("Should fail locking using an expired lock");
+        } catch (LockingException le) {
+            assertTrue(le.getCause() instanceof KeeperException.SessionExpiredException);
+        }
+        assertEquals(State.CLOSED, lock.getLockState());
+        List<String> children = getLockWaiters(zkc, lockPath);
+        assertEquals(0, children.size());
+    }
+
+    @Test(timeout = 60000)
+    public void testSessionExpiredForLockWaiter() throws Exception {
+        String lockPath = "/test-session-expired-for-lock-waiter";
+        String clientId0 = "test-session-expired-for-lock-waiter-0";
+        String clientId1 = "test-session-expired-for-lock-waiter-1";
+
+        createLockPath(zkc.get(), lockPath);
+
+        final ZKSessionLock lock0 = new ZKSessionLock(zkc0, lockPath, clientId0, lockStateExecutor);
+        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        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))));
+
+        final ZKSessionLock lock1 = new ZKSessionLock(zkc, lockPath, clientId1, lockStateExecutor);
+        final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
+
+        Thread lock1Thread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    lock1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+                } catch (OwnershipAcquireFailedException oafe) {
+                    lock1DoneLatch.countDown();
+                } catch (LockingException e) {
+                    logger.error("Failed on locking lock1 : ", e);
+                }
+            }
+        }, "lock1-thread");
+        lock1Thread.start();
+
+        // check lock1 is waiting for lock0
+        children = awaitWaiters(2, zkc, lockPath);
+
+        assertEquals(2, children.size());
+        assertEquals(State.CLAIMED, lock0.getLockState());
+        assertEquals(lock0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+        awaitState(State.WAITING, lock1);
+        assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
+
+        // expire lock1
+        ZooKeeperClientUtils.expireSession(zkc, zkServers, sessionTimeoutMs);
+
+        lock1DoneLatch.countDown();
+        lock1Thread.join();
+        assertEquals(State.CLAIMED, lock0.getLockState());
+        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))));
+    }
+
+    public void awaitState(State state, ZKSessionLock lock) throws InterruptedException {
+        while (lock.getLockState() != state) {
+            Thread.sleep(50);
+        }
+    }
+
+    public List<String> awaitWaiters(int waiters, ZooKeeperClient zkc, String lockPath) throws Exception {
+        List<String> children = getLockWaiters(zkc, lockPath);
+        while (children.size() < waiters) {
+            Thread.sleep(50);
+            children = getLockWaiters(zkc, lockPath);
+        }
+        return children;
+    }
+
+    @Test(timeout = 60000)
+    public void testLockUseSameClientIdButDifferentSessions0() throws Exception {
+        testLockUseSameClientIdButDifferentSessions(true);
+    }
+
+    @Test(timeout = 60000)
+    public void testLockUseSameClientIdButDifferentSessions1() throws Exception {
+        testLockUseSameClientIdButDifferentSessions(false);
+    }
+
+    private void testLockUseSameClientIdButDifferentSessions(boolean isUnlock) throws Exception {
+        String lockPath = "/test-lock-use-same-client-id-but-different-sessions-" + isUnlock + System.currentTimeMillis();
+        String clientId = "test-lock-use-same-client-id-but-different-sessions";
+
+        createLockPath(zkc.get(), lockPath);
+
+        final ZKSessionLock lock0 = new ZKSessionLock(zkc0, lockPath, clientId, lockStateExecutor);
+
+        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        // lock1_0 couldn't claim ownership since owner is in a different zk session.
+        final ZKSessionLock lock1_0 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
+        try {
+            lock1_0.tryLock(0, TimeUnit.MILLISECONDS);
+            fail("Should fail locking since the lock is held in a different zk session.");
+        } catch (OwnershipAcquireFailedException oafe) {
+            assertEquals(clientId, oafe.getCurrentOwner());
+        }
+        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))));
+
+        // lock1_1 would wait the ownership
+        final ZKSessionLock lock1_1 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
+        final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
+
+        Thread lock1Thread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    lock1_1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+                    lock1DoneLatch.countDown();
+                } catch (LockingException e) {
+                    logger.error("Failed on locking lock1 : ", e);
+                }
+            }
+        }, "lock1-thread");
+        lock1Thread.start();
+
+        // check lock1 is waiting for lock0
+        children = awaitWaiters(2, zkc, lockPath);
+
+        logger.info("Found {} lock waiters : {}", children.size(), children);
+
+        assertEquals(2, children.size());
+        assertEquals(State.CLAIMED, lock0.getLockState());
+        assertEquals(lock0.getLockId(), Await.result(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))));
+
+        if (isUnlock) {
+            lock0.unlock();
+        } else {
+            ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
+        }
+        lock1DoneLatch.await();
+        lock1Thread.join();
+
+        // verification
+        if (isUnlock) {
+            assertEquals(State.CLOSED, lock0.getLockState());
+        } else {
+            assertEquals(State.EXPIRED, lock0.getLockState());
+        }
+        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))));
+
+        lock1_1.unlock();
+    }
+
+    @Test(timeout = 60000)
+    public void testLockWithMultipleSiblingWaiters() throws Exception {
+        String lockPath = "/test-lock-with-multiple-sibling-waiters";
+        String clientId = "client-id";
+
+        createLockPath(zkc.get(), lockPath);
+
+        final ZKSessionLock lock0 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
+        final ZKSessionLock lock1 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
+        final ZKSessionLock lock2 = new ZKSessionLock(zkc, lockPath, clientId, lockStateExecutor);
+
+        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        lock1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        lock2.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+
+        List<String> children = awaitWaiters(3, zkc, lockPath);
+
+        assertEquals(3, children.size());
+        assertEquals(State.CLAIMED, lock0.getLockState());
+        assertEquals(State.CLAIMED, lock1.getLockState());
+        assertEquals(State.CLAIMED, lock2.getLockState());
+
+        lock0.unlock();
+        lock1.unlock();
+        lock2.unlock();
+    }
+
+    /**
+     * Immediate lock and unlock first lock
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testLockWhenSiblingUseDifferentLockId0() throws Exception {
+        testLockWhenSiblingUseDifferentLockId(0, true);
+    }
+
+    /**
+     * Immediate lock and expire first lock
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testLockWhenSiblingUseDifferentLockId1() throws Exception {
+        testLockWhenSiblingUseDifferentLockId(0, false);
+    }
+
+    /**
+     * Wait Lock and unlock lock0_0 and lock1
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testLockWhenSiblingUseDifferentLockId2() throws Exception {
+        testLockWhenSiblingUseDifferentLockId(Long.MAX_VALUE, true);
+    }
+
+    /**
+     * Wait Lock and expire first & third lock
+     * @throws Exception
+     */
+    @Test(timeout = 60000)
+    public void testLockWhenSiblingUseDifferentLockId3() throws Exception {
+        testLockWhenSiblingUseDifferentLockId(Long.MAX_VALUE, false);
+    }
+
+    private void testLockWhenSiblingUseDifferentLockId(long timeout, final boolean isUnlock) throws Exception {
+        String lockPath = "/test-lock-when-sibling-use-different-lock-id-" + timeout
+                + "-" + isUnlock + "-" + System.currentTimeMillis();
+        String clientId0 = "client-id-0";
+        String clientId1 = "client-id-1";
+
+        createLockPath(zkc.get(), lockPath);
+
+        final ZKSessionLock lock0_0 = new ZKSessionLock(zkc0, lockPath, clientId0, lockStateExecutor);
+        final ZKSessionLock lock0_1 = new ZKSessionLock(zkc0, lockPath, clientId0, lockStateExecutor);
+        final ZKSessionLock lock1   = new ZKSessionLock(zkc, lockPath, clientId1, lockStateExecutor);
+
+        lock0_0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+
+        // lock1 wait for the lock ownership.
+        final CountDownLatch lock1DoneLatch = new CountDownLatch(1);
+        Thread lock1Thread = new Thread(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    lock1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+                    lock1DoneLatch.countDown();
+                } catch (LockingException e) {
+                    logger.error("Failed on locking lock1 : ", e);
+                }
+            }
+        }, "lock1-thread");
+        lock1Thread.start();
+
+        // check lock1 is waiting for lock0_0
+        List<String> children = awaitWaiters(2, 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))));
+        awaitState(State.WAITING, lock1);
+        assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
+
+        final CountDownLatch lock0DoneLatch = new CountDownLatch(1);
+        final AtomicReference<String> ownerFromLock0 = new AtomicReference<String>(null);
+        Thread lock0Thread = null;
+        if (timeout == 0) {
+            try {
+                lock0_1.tryLock(0, TimeUnit.MILLISECONDS);
+                fail("Should fail on locking if sibling is using differnt lock id.");
+            } catch (OwnershipAcquireFailedException oafe) {
+                assertEquals(clientId0, oafe.getCurrentOwner());
+            }
+            assertEquals(State.CLOSED, lock0_1.getLockState());
+            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(State.WAITING, lock1.getLockState());
+            assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc.get(), lockPath, children.get(1))));
+        } else {
+            lock0Thread = new Thread(new Runnable() {
+                @Override
+                public void run() {
+                    try {
+                        lock0_1.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+                        if (isUnlock) {
+                            lock0DoneLatch.countDown();
+                        }
+                    } catch (OwnershipAcquireFailedException oafe) {
+                        if (!isUnlock) {
+                            ownerFromLock0.set(oafe.getCurrentOwner());
+                            lock0DoneLatch.countDown();
+                        }
+                    } catch (LockingException le) {
+                        logger.error("Failed on locking lock0_1 : ", le);
+                    }
+                }
+            }, "lock0-thread");
+            lock0Thread.start();
+
+            // check lock1 is waiting for lock0_0
+            children = awaitWaiters(3, zkc, lockPath);
+
+            assertEquals(3, children.size());
+            assertEquals(State.CLAIMED, lock0_0.getLockState());
+            assertEquals(lock0_0.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(0))));
+            awaitState(State.WAITING, lock1);
+            assertEquals(lock1.getLockId(), Await.result(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))));
+        }
+
+        if (isUnlock) {
+            lock0_0.unlock();
+        } else {
+            ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
+        }
+
+        lock1DoneLatch.await();
+        lock1Thread.join();
+
+        // check the state of lock0_0
+        if (isUnlock) {
+            assertEquals(State.CLOSED, lock0_0.getLockState());
+        } else {
+            assertEquals(State.EXPIRED, lock0_0.getLockState());
+        }
+
+        if (timeout == 0) {
+            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))));
+        } else {
+            assertNotNull(lock0Thread);
+            if (!isUnlock) {
+                // both lock0_0 and lock0_1 would be expired
+                lock0DoneLatch.await();
+                lock0Thread.join();
+
+                assertEquals(clientId0, ownerFromLock0.get());
+                assertEquals(State.CLOSED, lock0_1.getLockState());
+
+                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))));
+            } 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(State.WAITING, lock0_1.getLockState());
+                assertEquals(lock0_1.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(1))));
+            }
+        }
+
+        lock1.unlock();
+
+        if (timeout != 0 && isUnlock) {
+            lock0DoneLatch.await();
+            lock0Thread.join();
+
+            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))));
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testLockWhenSiblingUseSameLockId0() throws Exception {
+        testLockWhenSiblingUseSameLockId(0, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testLockWhenSiblingUseSameLockId1() throws Exception {
+        testLockWhenSiblingUseSameLockId(0, false);
+    }
+
+    @Test(timeout = 60000)
+    public void testLockWhenSiblingUseSameLockId2() throws Exception {
+        testLockWhenSiblingUseSameLockId(Long.MAX_VALUE, true);
+    }
+
+    @Test(timeout = 60000)
+    public void testLockWhenSiblingUseSameLockId3() throws Exception {
+        testLockWhenSiblingUseSameLockId(Long.MAX_VALUE, false);
+    }
+
+    private void testLockWhenSiblingUseSameLockId(long timeout, final boolean isUnlock) throws Exception {
+        String lockPath = "/test-lock-when-sibling-use-same-lock-id-" + timeout
+                + "-" + isUnlock + "-" + System.currentTimeMillis();
+        String clientId = "client-id";
+
+        createLockPath(zkc.get(), lockPath);
+
+        final ZKSessionLock lock0 = new ZKSessionLock(zkc0, lockPath, clientId, lockStateExecutor);
+        final ZKSessionLock lock1 = new ZKSessionLock(zkc0, lockPath, clientId, lockStateExecutor);
+
+        lock0.tryLock(Long.MAX_VALUE, TimeUnit.MILLISECONDS);
+        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))));
+
+        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(State.CLAIMED, lock1.getLockState());
+        assertEquals(lock1.getLockId(), Await.result(asyncParseClientID(zkc0.get(), lockPath, children.get(1))));
+
+        if (isUnlock) {
+            lock0.unlock();
+            assertEquals(State.CLOSED, lock0.getLockState());
+            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))));
+            lock1.unlock();
+        } else {
+            ZooKeeperClientUtils.expireSession(zkc0, zkServers, sessionTimeoutMs);
+            final CountDownLatch latch = new CountDownLatch(1);
+            lockStateExecutor.submit(lockPath, new SafeRunnable() {
+                @Override
+                public void safeRun() {
+                    latch.countDown();
+                }
+            });
+            latch.await();
+            children = getLockWaiters(zkc, lockPath);
+            assertEquals(0, children.size());
+            assertEquals(State.EXPIRED, lock0.getLockState());
+            assertEquals(State.EXPIRED, lock1.getLockState());
+        }
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestPerStreamLogSegmentCache.java b/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestPerStreamLogSegmentCache.java
new file mode 100644
index 0000000..4370687
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestPerStreamLogSegmentCache.java
@@ -0,0 +1,186 @@
+/**
+ * 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.logsegment;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.junit.Test;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test Case for Per Stream Log Segment Cache.
+ */
+public class TestPerStreamLogSegmentCache {
+
+    @Test(timeout = 60000)
+    public void testBasicOperations() {
+        LogSegmentMetadata metadata =
+                DLMTestUtil.completedLogSegment("/segment1", 1L, 1L, 100L, 100, 1L, 99L, 0L);
+        String name = DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(1L);
+
+        PerStreamLogSegmentCache cache = new PerStreamLogSegmentCache("test-basic-operations");
+        assertNull("No log segment " + name + " should be cached", cache.get(name));
+        cache.add(name, metadata);
+        LogSegmentMetadata metadataRetrieved = cache.get(name);
+        assertNotNull("log segment " + name + " should be cached", metadataRetrieved);
+        assertEquals("Wrong log segment metadata returned for " + name,
+                metadata, metadataRetrieved);
+        LogSegmentMetadata metadataRemoved = cache.remove(name);
+        assertNull("log segment " + name + " should be removed from cache", cache.get(name));
+        assertEquals("Wrong log segment metadata removed for " + name,
+                metadata, metadataRemoved);
+        assertNull("No log segment " + name + " to be removed", cache.remove(name));
+    }
+
+    @Test(timeout = 60000)
+    public void testDiff() {
+        PerStreamLogSegmentCache cache = new PerStreamLogSegmentCache("test-diff");
+        // add 5 completed log segments
+        for (int i = 1; i <= 5; i++) {
+            LogSegmentMetadata metadata =
+                    DLMTestUtil.completedLogSegment("/segment" + i, i, i, i * 100L, 100, i, 99L, 0L);
+            String name = DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i);
+            cache.add(name, metadata);
+        }
+        // add one inprogress log segment
+        LogSegmentMetadata inprogress =
+                DLMTestUtil.inprogressLogSegment("/inprogress-6", 6, 600L, 6);
+        String name = DLMTestUtil.inprogressZNodeName(6);
+        cache.add(name, inprogress);
+
+        // deleted first 2 completed log segments and completed the last one
+        Set<String> segmentRemoved = Sets.newHashSet();
+        for (int i = 1; i <= 2; i++) {
+            segmentRemoved.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i));
+        }
+        segmentRemoved.add((DLMTestUtil.inprogressZNodeName(6)));
+        Set<String> segmentReceived = Sets.newHashSet();
+        Set<String> segmentAdded = Sets.newHashSet();
+        for (int i = 3; i <= 6; i++) {
+            segmentReceived.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i));
+            if (i == 6) {
+                segmentAdded.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i));
+            }
+        }
+
+        Pair<Set<String>, Set<String>> segmentChanges = cache.diff(segmentReceived);
+        assertTrue("Should remove " + segmentRemoved + ", but removed " + segmentChanges.getRight(),
+                Sets.difference(segmentRemoved, segmentChanges.getRight()).isEmpty());
+        assertTrue("Should add " + segmentAdded + ", but added " + segmentChanges.getLeft(),
+                Sets.difference(segmentAdded, segmentChanges.getLeft()).isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testUpdate() {
+        PerStreamLogSegmentCache cache = new PerStreamLogSegmentCache("test-update");
+        // add 5 completed log segments
+        for (int i = 1; i <= 5; i++) {
+            LogSegmentMetadata metadata =
+                    DLMTestUtil.completedLogSegment("/segment" + i, i, i, i * 100L, 100, i, 99L, 0L);
+            String name = DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i);
+            cache.add(name, metadata);
+        }
+        // add one inprogress log segment
+        LogSegmentMetadata inprogress =
+                DLMTestUtil.inprogressLogSegment("/inprogress-6", 6, 600L, 6);
+        String name = DLMTestUtil.inprogressZNodeName(6);
+        cache.add(name, inprogress);
+
+        // deleted first 2 completed log segments and completed the last one
+        Set<String> segmentRemoved = Sets.newHashSet();
+        for (int i = 1; i <= 2; i++) {
+            segmentRemoved.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i));
+        }
+        segmentRemoved.add((DLMTestUtil.inprogressZNodeName(6)));
+        Set<String> segmentReceived = Sets.newHashSet();
+        Map<String, LogSegmentMetadata> segmentAdded = Maps.newHashMap();
+        for (int i = 3; i <= 6; i++) {
+            segmentReceived.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i));
+            if (i == 6) {
+                segmentAdded.put(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(i),
+                        DLMTestUtil.completedLogSegment("/segment" + i, i, i, i * 100L, 100, i, 99L, 0L));
+            }
+        }
+
+        // update the cache
+        cache.update(segmentRemoved, segmentAdded);
+        for (String segment : segmentRemoved) {
+            assertNull("Segment " + segment + " should be removed.", cache.get(segment));
+        }
+        for (String segment : segmentReceived) {
+            assertNotNull("Segment " + segment + " should not be removed", cache.get(segment));
+        }
+        for (Map.Entry<String, LogSegmentMetadata> entry : segmentAdded.entrySet()) {
+            assertEquals("Segment " + entry.getKey() + " should be added.",
+                    entry.getValue(), entry.getValue());
+        }
+    }
+
+    @Test(timeout = 60000, expected = UnexpectedException.class)
+    public void testGapDetection() throws Exception {
+        PerStreamLogSegmentCache cache = new PerStreamLogSegmentCache("test-gap-detection");
+        cache.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(1L),
+                DLMTestUtil.completedLogSegment("/segment-1", 1L, 1L, 100L, 100, 1L, 99L, 0L));
+        cache.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(3L),
+                DLMTestUtil.completedLogSegment("/segment-3", 3L, 3L, 300L, 100, 3L, 99L, 0L));
+        cache.getLogSegments(LogSegmentMetadata.COMPARATOR);
+    }
+
+    @Test(timeout = 60000)
+    public void testGapDetectionOnLogSegmentsWithoutLogSegmentSequenceNumber() throws Exception {
+        PerStreamLogSegmentCache cache = new PerStreamLogSegmentCache("test-gap-detection");
+        LogSegmentMetadata segment1 =
+                DLMTestUtil.completedLogSegment("/segment-1", 1L, 1L, 100L, 100, 1L, 99L, 0L)
+                        .mutator().setVersion(LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V1_ORIGINAL).build();
+        cache.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(1L), segment1);
+        LogSegmentMetadata segment3 =
+                DLMTestUtil.completedLogSegment("/segment-3", 3L, 3L, 300L, 100, 3L, 99L, 0L)
+                        .mutator().setVersion(LogSegmentMetadata.LogSegmentMetadataVersion.VERSION_V2_LEDGER_SEQNO).build();
+        cache.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(3L), segment3);
+        List<LogSegmentMetadata> expectedList = Lists.asList(segment1, new LogSegmentMetadata[] { segment3 });
+        List<LogSegmentMetadata> resultList = cache.getLogSegments(LogSegmentMetadata.COMPARATOR);
+        assertEquals(expectedList, resultList);
+    }
+
+    @Test(timeout = 60000, expected = UnexpectedException.class)
+    public void testSameLogSegment() throws Exception {
+        PerStreamLogSegmentCache cache = new PerStreamLogSegmentCache("test-same-log-segment");
+        List<LogSegmentMetadata> expectedList = Lists.newArrayListWithExpectedSize(2);
+        LogSegmentMetadata inprogress =
+                DLMTestUtil.inprogressLogSegment("/inprogress-1", 1L, 1L, 1L);
+        expectedList.add(inprogress);
+        cache.add(DLMTestUtil.inprogressZNodeName(1L), inprogress);
+        LogSegmentMetadata completed =
+                DLMTestUtil.completedLogSegment("/segment-1", 1L, 1L, 100L, 100, 1L, 99L, 0L);
+        expectedList.add(completed);
+        cache.add(DLMTestUtil.completedLedgerZNodeNameWithLogSegmentSequenceNumber(1L), completed);
+
+        cache.getLogSegments(LogSegmentMetadata.COMPARATOR);
+    }
+
+}
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
new file mode 100644
index 0000000..5943b64
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/logsegment/TestRollingPolicy.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.logsegment;
+
+import org.apache.distributedlog.util.Sizable;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test Case for {@link RollingPolicy}s.
+ */
+public class TestRollingPolicy {
+
+    static class TestSizable implements Sizable {
+
+        long size;
+
+        TestSizable(long size) {
+            this.size = size;
+        }
+
+        @Override
+        public long size() {
+            return size;
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testTimeBasedRollingPolicy() {
+        TimeBasedRollingPolicy policy1 = new TimeBasedRollingPolicy(Long.MAX_VALUE);
+        TestSizable maxSize = new TestSizable(Long.MAX_VALUE);
+        assertFalse(policy1.shouldRollover(maxSize, System.currentTimeMillis()));
+
+        long currentMs = System.currentTimeMillis();
+        TimeBasedRollingPolicy policy2 = new TimeBasedRollingPolicy(1000);
+        assertTrue(policy2.shouldRollover(maxSize, currentMs - 2 * 1000));
+    }
+
+    @Test(timeout = 60000)
+    public void testSizeBasedRollingPolicy() {
+        SizeBasedRollingPolicy policy = new SizeBasedRollingPolicy(1000);
+        TestSizable sizable1 = new TestSizable(10);
+        assertFalse(policy.shouldRollover(sizable1, 0L));
+        TestSizable sizable2 = new TestSizable(10000);
+        assertTrue(policy.shouldRollover(sizable2, 0L));
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestDLMetadata.java b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestDLMetadata.java
new file mode 100644
index 0000000..489c66b
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestDLMetadata.java
@@ -0,0 +1,195 @@
+/**
+ * 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.metadata;
+
+import org.apache.distributedlog.LocalDLMEmulator;
+import org.apache.distributedlog.ZooKeeperClusterTestCase;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.URI;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+public class TestDLMetadata extends ZooKeeperClusterTestCase {
+
+    private static final BKDLConfig bkdlConfig =
+            new BKDLConfig("127.0.0.1:7000", "127.0.0.1:7000",
+                           "127.0.0.1:7000", "127.0.0.1:7000", "ledgers");
+    private static final BKDLConfig bkdlConfig2 =
+            new BKDLConfig("127.0.0.1:7001", "127.0.0.1:7002",
+                           "127.0.0.1:7003", "127.0.0.1:7004", "ledgers2");
+
+    private ZooKeeper zkc;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = LocalDLMEmulator.connectZooKeeper("127.0.0.1", zkPort);
+    }
+
+    @After
+    public void teardown() throws Exception {
+        zkc.close();
+    }
+
+    private URI createURI(String path) {
+        return URI.create("distributedlog://127.0.0.1:" + zkPort + path);
+    }
+
+    @Test(timeout = 60000)
+    public void testBadMetadata() throws Exception {
+        URI uri = createURI("/");
+        try {
+            DLMetadata.deserialize(uri, new byte[0]);
+            fail("Should fail to deserialize invalid metadata");
+        } catch (IOException ie) {
+            // expected
+        }
+        try {
+            DLMetadata.deserialize(uri, new DLMetadata("unknown", bkdlConfig).serialize());
+            fail("Should fail to deserialize due to unknown dl type.");
+        } catch (IOException ie) {
+            // expected
+        }
+        try {
+            DLMetadata.deserialize(uri, new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig, 9999).serialize());
+            fail("Should fail to deserialize due to invalid version.");
+        } catch (IOException ie) {
+            // expected
+        }
+        byte[] data = new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig).serialize();
+        // truncate data
+        byte[] badData = new byte[data.length - 3];
+        System.arraycopy(data, 0, badData, 0, badData.length);
+        try {
+            DLMetadata.deserialize(uri, badData);
+            fail("Should fail to deserialize truncated data.");
+        } catch (IOException ie) {
+            // expected
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testGoodMetadata() throws Exception {
+        URI uri = createURI("/");
+        byte[] data = new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig).serialize();
+        DLMetadata deserailized = DLMetadata.deserialize(uri, data);
+        assertEquals(bkdlConfig, deserailized.getDLConfig());
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteMetadata() throws Exception {
+        DLMetadata metadata = new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig);
+        try {
+            metadata.create(createURI("//metadata"));
+            fail("Should fail due to invalid uri.");
+        } catch (IllegalArgumentException e) {
+            // expected
+        }
+        URI uri = createURI("/metadata");
+        metadata.create(uri);
+        // create on existed path
+        try {
+            metadata.create(uri);
+            fail("Should fail when create on existed path");
+        } catch (IOException e) {
+            // expected
+        }
+        // update on unexisted path
+        try {
+            metadata.update(createURI("/unexisted"));
+            fail("Should fail when update on unexisted path");
+        } catch (IOException e) {
+            // expected
+        }
+        byte[] data = zkc.getData("/metadata", false, new Stat());
+        assertEquals(bkdlConfig, DLMetadata.deserialize(uri, data).getDLConfig());
+        // update on existed path
+        DLMetadata newMetadata = new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig2);
+        newMetadata.update(createURI("/metadata"));
+        byte[] newData = zkc.getData("/metadata", false, new Stat());
+        assertEquals(bkdlConfig2, DLMetadata.deserialize(uri, newData).getDLConfig());
+    }
+
+    // Missing dlZkServersForWriter, dlZkServersForReader default to configured server.
+    @Test(timeout = 60000)
+    public void testMetadataWithoutDLZKServers() throws Exception {
+        testMetadataWithOrWithoutZkServers(
+                "/metadata-without-dlzk-servers",
+                null, null, "127.0.0.1:7003", "127.0.0.1:7004",
+                "127.0.0.1:" + zkPort, "127.0.0.1:" + zkPort, "127.0.0.1:7003", "127.0.0.1:7004");
+    }
+
+    @Test(timeout = 60000)
+    public void testMetadataWithoutDLZKServersForRead() throws Exception {
+        testMetadataWithOrWithoutZkServers(
+                "/metadata-without-dlzk-servers-for-read",
+                "127.0.0.1:7001", null, "127.0.0.1:7003", "127.0.0.1:7004",
+                "127.0.0.1:7001", "127.0.0.1:7001", "127.0.0.1:7003", "127.0.0.1:7004");
+    }
+
+    @Test(timeout = 60000)
+    public void testMetadataWithoutBKZKServersForRead() throws Exception {
+        testMetadataWithOrWithoutZkServers(
+                "/metadata-without-bkzk-servers-for-read",
+                "127.0.0.1:7001", null, "127.0.0.1:7003", null,
+                "127.0.0.1:7001", "127.0.0.1:7001", "127.0.0.1:7003", "127.0.0.1:7003");
+    }
+
+    private void testMetadataWithOrWithoutZkServers(
+            String metadataPath,
+            String dlZkServersForWriter, String dlZkServersForReader,
+            String bkZkServersForWriter, String bkZkServersForReader,
+            String expectedDlZkServersForWriter, String expectedDlZkServersForReader,
+            String expectedBkZkServersForWriter, String expectedBkZkServersForReader
+    ) throws Exception {
+        BKDLConfig bkdlConfig = new BKDLConfig(dlZkServersForWriter, dlZkServersForReader,
+                                               bkZkServersForWriter, bkZkServersForReader, "ledgers");
+        BKDLConfig expectedBKDLConfig =
+                new BKDLConfig(expectedDlZkServersForWriter, expectedDlZkServersForReader,
+                               expectedBkZkServersForWriter, expectedBkZkServersForReader, "ledgers");
+        URI uri = createURI(metadataPath);
+        DLMetadata metadata = new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig);
+        metadata.create(uri);
+        // read serialized metadata
+        byte[] data = zkc.getData(metadataPath, false, new Stat());
+        assertEquals(expectedBKDLConfig, DLMetadata.deserialize(uri, data).getDLConfig());
+    }
+
+    @Test(timeout = 60000)
+    public void testMetadataMissingRequiredFields() throws Exception {
+        BKDLConfig bkdlConfig = new BKDLConfig(null, null, null, null, "ledgers");
+        URI uri = createURI("/metadata-missing-fields");
+        DLMetadata metadata = new DLMetadata(DLMetadata.BK_DL_TYPE, bkdlConfig);
+        metadata.create(uri);
+        // read serialized metadata
+        byte[] data = zkc.getData("/metadata-missing-fields", false, new Stat());
+        try {
+            DLMetadata.deserialize(uri, data);
+            fail("Should fail on deserializing metadata missing fields");
+        } catch (IOException ioe) {
+            // expected
+        }
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogMetadata.java b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogMetadata.java
new file mode 100644
index 0000000..5216861
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogMetadata.java
@@ -0,0 +1,59 @@
+/**
+ * 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.metadata;
+
+import org.apache.distributedlog.DLMTestUtil;
+import org.junit.Test;
+
+import java.net.URI;
+
+import static org.apache.distributedlog.metadata.LogMetadata.*;
+import static org.junit.Assert.*;
+
+public class TestLogMetadata {
+
+    @Test(timeout = 60000)
+    public void testGetPaths() throws Exception {
+        String rootPath = "/test-get-paths";
+        URI uri = DLMTestUtil.createDLMURI(2181, rootPath);
+        String logName = "test-log";
+        String logIdentifier = "<default>";
+        String logRootPath = uri.getPath() + "/" + logName + "/" + logIdentifier;
+        String logSegmentName = "test-segment";
+
+        LogMetadata logMetadata = new LogMetadata(uri, logName, logIdentifier);
+        assertEquals("wrong log name", logName, logMetadata.getLogName());
+        assertEquals("wrong root path", logRootPath, logMetadata.getLogRootPath());
+        assertEquals("wrong log segments path",
+                logRootPath + LOGSEGMENTS_PATH,
+                logMetadata.getLogSegmentsPath());
+        assertEquals("wrong log segment path",
+                logRootPath + LOGSEGMENTS_PATH + "/" + logSegmentName,
+                logMetadata.getLogSegmentPath(logSegmentName));
+        assertEquals("wrong lock path",
+                logRootPath + LOCK_PATH, logMetadata.getLockPath());
+        assertEquals("wrong max tx id path",
+                logRootPath + MAX_TXID_PATH, logMetadata.getMaxTxIdPath());
+        assertEquals("wrong allocation path",
+                logRootPath + ALLOCATION_PATH, logMetadata.getAllocationPath());
+        assertEquals("wrong qualified name",
+                logName + ":" + logIdentifier, logMetadata.getFullyQualifiedName());
+    }
+
+
+}
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
new file mode 100644
index 0000000..e18fb3f
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/metadata/TestLogSegmentMetadataStoreUpdater.java
@@ -0,0 +1,284 @@
+/**
+ * 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.metadata;
+
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.LogRecordWithDLSN;
+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.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.net.URI;
+import java.util.HashMap;
+import java.util.Map;
+
+import static org.junit.Assert.*;
+
+public class TestLogSegmentMetadataStoreUpdater extends ZooKeeperClusterTestCase {
+
+    static final Logger LOG = LoggerFactory.getLogger(TestLogSegmentMetadataStoreUpdater.class);
+
+    private ZooKeeperClient zkc;
+    private OrderedScheduler scheduler;
+    private LogSegmentMetadataStore metadataStore;
+    private DistributedLogConfiguration conf = new DistributedLogConfiguration()
+            .setDLLedgerMetadataLayoutVersion(LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
+
+    @Before
+    public void setup() throws Exception {
+        scheduler = OrderedScheduler.newBuilder()
+                .name("test-logsegment-metadata-store-updater")
+                .corePoolSize(1)
+                .build();
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .uri(createURI("/"))
+                .sessionTimeoutMs(10000)
+                .build();
+        metadataStore = new ZKLogSegmentMetadataStore(conf, zkc, scheduler);
+    }
+
+    @After
+    public void tearDown() throws Exception {
+        metadataStore.close();
+        scheduler.shutdown();
+        zkc.close();
+    }
+
+    private URI createURI(String path) {
+        return URI.create("distributedlog://127.0.0.1:" + zkPort + path);
+    }
+
+    Map<Long, LogSegmentMetadata> readLogSegments(String ledgerPath) throws Exception {
+        return DLMTestUtil.readLogSegments(zkc, ledgerPath);
+    }
+
+    @Test(timeout = 60000)
+    public void testChangeSequenceNumber() throws Exception {
+        String ledgerPath = "/testChangeSequenceNumber";
+        zkc.get().create(ledgerPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        Map<Long, LogSegmentMetadata> completedLogSegments = new HashMap<Long, LogSegmentMetadata>();
+        // Create 5 completed log segments
+        for (int i = 1; i <= 5; i++) {
+            LogSegmentMetadata segment = DLMTestUtil.completedLogSegment(ledgerPath, i, (i - 1) * 100, i * 100 - 1, 100, i, 100, 0);
+            completedLogSegments.put(((long)i), segment);
+            LOG.info("Create completed segment {} : {}", segment.getZkPath(), segment);
+            segment.write(zkc);
+        }
+        // Create a smaller inprogress log segment
+        long inprogressSeqNo = 3;
+        LogSegmentMetadata segment = DLMTestUtil.inprogressLogSegment(ledgerPath, inprogressSeqNo, 5 * 100, inprogressSeqNo);
+        LOG.info("Create inprogress segment {} : {}", segment.getZkPath(), segment);
+        segment.write(zkc);
+
+        Map<Long, LogSegmentMetadata> segmentList = readLogSegments(ledgerPath);
+        assertEquals(5, segmentList.size());
+
+        // Dryrun
+        MetadataUpdater dryrunUpdater = new DryrunLogSegmentMetadataStoreUpdater(conf, metadataStore);
+        FutureUtils.result(dryrunUpdater.changeSequenceNumber(segment, 6L));
+
+        segmentList = readLogSegments(ledgerPath);
+        assertEquals(5, segmentList.size());
+
+        // Fix the inprogress log segments
+
+        MetadataUpdater updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
+        FutureUtils.result(updater.changeSequenceNumber(segment, 6L));
+
+        segmentList = readLogSegments(ledgerPath);
+        assertEquals(6, segmentList.size());
+
+        // check first 5 log segments
+        for (int i = 1; i <= 5; i++) {
+            LogSegmentMetadata s = segmentList.get((long)i);
+            assertNotNull(s);
+            assertEquals(completedLogSegments.get((long)i), s);
+        }
+
+        // get log segment 6
+        LogSegmentMetadata segmentChanged = segmentList.get(6L);
+        assertNotNull(segmentChanged);
+        assertEquals(6L, segmentChanged.getLogSegmentSequenceNumber());
+        assertTrue(segmentChanged.isInProgress());
+        assertEquals(5 * 100, segmentChanged.getFirstTxId());
+        assertEquals(3L, segmentChanged.getLogSegmentId());
+    }
+
+    @Test(timeout = 60000)
+    public void testUpdateLastDLSN() throws Exception {
+        String ledgerPath = "/testUpdateLastDLSN";
+        zkc.get().create(ledgerPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        // Create 1 completed log segment
+        LogSegmentMetadata completedLogSegment = DLMTestUtil.completedLogSegment(ledgerPath, 1L, 0L, 99L, 100, 1L, 99L, 0L);
+        completedLogSegment.write(zkc);
+        // Create 1 inprogress log segment
+        LogSegmentMetadata inprogressLogSegment = DLMTestUtil.inprogressLogSegment(ledgerPath, 2L, 100L, 2L);
+        inprogressLogSegment.write(zkc);
+
+        DLSN badLastDLSN = new DLSN(99L, 0L, 0L);
+        DLSN goodLastDLSN1 = new DLSN(1L, 100L, 0L);
+        DLSN goodLastDLSN2 = new DLSN(2L, 200L, 0L);
+
+        LogRecordWithDLSN badRecord = DLMTestUtil.getLogRecordWithDLSNInstance(badLastDLSN, 100L);
+        LogRecordWithDLSN goodRecord1 = DLMTestUtil.getLogRecordWithDLSNInstance(goodLastDLSN1, 100L);
+        LogRecordWithDLSN goodRecord2 = DLMTestUtil.getLogRecordWithDLSNInstance(goodLastDLSN2, 200L);
+
+        // Dryrun
+        MetadataUpdater dryrunUpdater = new DryrunLogSegmentMetadataStoreUpdater(conf, metadataStore);
+        try {
+            FutureUtils.result(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));
+            fail("Should fail on updating dlsn for an inprogress log segment");
+        } catch (IllegalStateException ise) {
+            // expected
+        }
+        LogSegmentMetadata updatedCompletedLogSegment =
+                FutureUtils.result(dryrunUpdater.updateLastRecord(completedLogSegment, goodRecord1));
+        assertEquals(goodLastDLSN1, updatedCompletedLogSegment.getLastDLSN());
+        assertEquals(goodRecord1.getTransactionId(), updatedCompletedLogSegment.getLastTxId());
+        assertTrue(updatedCompletedLogSegment.isRecordLastPositioninThisSegment(goodRecord1));
+
+        Map<Long, LogSegmentMetadata> segmentList = readLogSegments(ledgerPath);
+        assertEquals(2, segmentList.size());
+
+        LogSegmentMetadata readCompletedLogSegment = segmentList.get(1L);
+        assertNotNull(readCompletedLogSegment);
+        assertEquals(completedLogSegment, readCompletedLogSegment);
+
+        LogSegmentMetadata readInprogressLogSegment = segmentList.get(2L);
+        assertNotNull(readInprogressLogSegment);
+        assertEquals(inprogressLogSegment, readInprogressLogSegment);
+
+        // Fix the last dlsn
+        MetadataUpdater updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
+        try {
+            FutureUtils.result(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));
+            fail("Should fail on updating dlsn for an inprogress log segment");
+        } catch (IllegalStateException ise) {
+            // expected
+        }
+        updatedCompletedLogSegment = FutureUtils.result(updater.updateLastRecord(completedLogSegment, goodRecord1));
+        assertEquals(goodLastDLSN1, updatedCompletedLogSegment.getLastDLSN());
+        assertEquals(goodRecord1.getTransactionId(), updatedCompletedLogSegment.getLastTxId());
+        assertTrue(updatedCompletedLogSegment.isRecordLastPositioninThisSegment(goodRecord1));
+
+        segmentList = readLogSegments(ledgerPath);
+        assertEquals(2, segmentList.size());
+
+        readCompletedLogSegment = segmentList.get(1L);
+        assertNotNull(readCompletedLogSegment);
+        assertEquals(goodLastDLSN1, readCompletedLogSegment.getLastDLSN());
+        assertEquals(goodRecord1.getTransactionId(), readCompletedLogSegment.getLastTxId());
+        assertTrue(readCompletedLogSegment.isRecordLastPositioninThisSegment(goodRecord1));
+        assertEquals(updatedCompletedLogSegment, readCompletedLogSegment);
+        assertEquals(completedLogSegment.getCompletionTime(), readCompletedLogSegment.getCompletionTime());
+        assertEquals(completedLogSegment.getFirstTxId(), readCompletedLogSegment.getFirstTxId());
+        assertEquals(completedLogSegment.getLogSegmentId(), readCompletedLogSegment.getLogSegmentId());
+        assertEquals(completedLogSegment.getLogSegmentSequenceNumber(), readCompletedLogSegment.getLogSegmentSequenceNumber());
+        assertEquals(completedLogSegment.getRegionId(), readCompletedLogSegment.getRegionId());
+        assertEquals(completedLogSegment.getZkPath(), readCompletedLogSegment.getZkPath());
+        assertEquals(completedLogSegment.getZNodeName(), readCompletedLogSegment.getZNodeName());
+
+        readInprogressLogSegment = segmentList.get(2L);
+        assertNotNull(readInprogressLogSegment);
+        assertEquals(inprogressLogSegment, readInprogressLogSegment);
+    }
+
+    @Test(timeout = 60000)
+    public void testChangeTruncationStatus() throws Exception {
+        String ledgerPath = "/ledgers2";
+        zkc.get().create(ledgerPath, new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+        Map<Long, LogSegmentMetadata> completedLogSegments = new HashMap<Long, LogSegmentMetadata>();
+        // Create 5 completed log segments
+        for (int i = 1; i <= 5; i++) {
+            LogSegmentMetadata segment = DLMTestUtil.completedLogSegment(ledgerPath, i, (i - 1) * 100, i * 100 - 1, 100, i, 100, 0);
+            completedLogSegments.put(((long)i), segment);
+            LOG.info("Create completed segment {} : {}", segment.getZkPath(), segment);
+            segment.write(zkc);
+        }
+
+        Map<Long, LogSegmentMetadata> segmentList = readLogSegments(ledgerPath);
+        assertEquals(5, segmentList.size());
+
+        long segmentToModify = 1L;
+
+        // Dryrun
+        MetadataUpdater dryrunUpdater = new DryrunLogSegmentMetadataStoreUpdater(conf, metadataStore);
+        FutureUtils.result(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)));
+
+        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)));
+
+        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),
+                segmentList.get(segmentToModify).getFirstDLSN()));
+
+        segmentList = readLogSegments(ledgerPath);
+        assertEquals(false, segmentList.get(segmentToModify).isTruncated());
+        assertEquals(true, segmentList.get(segmentToModify).isPartiallyTruncated());
+
+        updater = LogSegmentMetadataStoreUpdater.createMetadataUpdater(conf, metadataStore);
+        FutureUtils.result(updater.setLogSegmentActive(segmentList.get(segmentToModify)));
+
+        segmentList = readLogSegments(ledgerPath);
+        assertEquals(false, segmentList.get(segmentToModify).isTruncated());
+        assertEquals(false, segmentList.get(segmentToModify).isPartiallyTruncated());
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestDistributedLogNamespaceBuilder.java b/distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestDistributedLogNamespaceBuilder.java
new file mode 100644
index 0000000..46a3a6f
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/namespace/TestDistributedLogNamespaceBuilder.java
@@ -0,0 +1,124 @@
+/**
+ * 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 org.apache.distributedlog.BKDistributedLogNamespace;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.TestDistributedLogBase;
+import org.junit.Test;
+
+import java.net.URI;
+
+import static org.apache.distributedlog.LocalDLMEmulator.DLOG_NAMESPACE;
+import static org.junit.Assert.assertTrue;
+
+/**
+ * Test Namespace Builder
+ */
+public class TestDistributedLogNamespaceBuilder extends TestDistributedLogBase {
+
+    @Test(timeout = 60000, expected = NullPointerException.class)
+    public void testEmptyBuilder() throws Exception {
+        DistributedLogNamespaceBuilder.newBuilder().build();
+    }
+
+    @Test(timeout = 60000, expected = NullPointerException.class)
+    public void testMissingUri() throws Exception {
+        DistributedLogNamespaceBuilder.newBuilder()
+                .conf(new DistributedLogConfiguration())
+                .build();
+    }
+
+    @Test(timeout = 60000, expected = NullPointerException.class)
+    public void testMissingSchemeInUri() throws Exception {
+        DistributedLogNamespaceBuilder.newBuilder()
+                .conf(new DistributedLogConfiguration())
+                .uri(new URI("/test"))
+                .build();
+    }
+
+    @Test(timeout = 60000, expected = IllegalArgumentException.class)
+    public void testInvalidSchemeInUri() throws Exception {
+        DistributedLogNamespaceBuilder.newBuilder()
+                .conf(new DistributedLogConfiguration())
+                .uri(new URI("dist://invalid/scheme/in/uri"))
+                .build();
+    }
+
+    @Test(timeout = 60000, expected = IllegalArgumentException.class)
+    public void testInvalidSchemeCorrectBackendInUri() throws Exception {
+        DistributedLogNamespaceBuilder.newBuilder()
+                .conf(new DistributedLogConfiguration())
+                .uri(new URI("dist-bk://invalid/scheme/in/uri"))
+                .build();
+    }
+
+    @Test(timeout = 60000, expected = IllegalArgumentException.class)
+    public void testUnknownBackendInUri() throws Exception {
+        DistributedLogNamespaceBuilder.newBuilder()
+                .conf(new DistributedLogConfiguration())
+                .uri(new URI("distributedlog-unknown://invalid/scheme/in/uri"))
+                .build();
+    }
+
+    @Test(timeout = 60000, expected = NullPointerException.class)
+    public void testNullStatsLogger() throws Exception {
+        DistributedLogNamespaceBuilder.newBuilder()
+                .conf(new DistributedLogConfiguration())
+                .uri(new URI("distributedlog-bk://localhost/distributedlog"))
+                .statsLogger(null)
+                .build();
+    }
+
+    @Test(timeout = 60000, expected = NullPointerException.class)
+    public void testNullClientId() throws Exception {
+        DistributedLogNamespaceBuilder.newBuilder()
+                .conf(new DistributedLogConfiguration())
+                .uri(new URI("distributedlog-bk://localhost/distributedlog"))
+                .clientId(null)
+                .build();
+    }
+
+    @Test(timeout = 60000)
+    public void testBuildBKDistributedLogNamespace() throws Exception {
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(new DistributedLogConfiguration())
+                .uri(new URI("distributedlog-bk://" + zkServers + DLOG_NAMESPACE + "/bknamespace"))
+                .build();
+        try {
+            assertTrue("distributedlog-bk:// should build bookkeeper based distributedlog namespace",
+                    namespace instanceof BKDistributedLogNamespace);
+        } finally {
+            namespace.close();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testBuildWhenMissingBackendInUri() throws Exception {
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(new DistributedLogConfiguration())
+                .uri(new URI("distributedlog://" + zkServers + DLOG_NAMESPACE + "/defaultnamespace"))
+                .build();
+        try {
+            assertTrue("distributedlog:// should build bookkeeper based distributedlog namespace",
+                    namespace instanceof BKDistributedLogNamespace);
+        } finally {
+            namespace.close();
+        }
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestDNSResolver.java b/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestDNSResolver.java
new file mode 100644
index 0000000..3a3a850
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestDNSResolver.java
@@ -0,0 +1,96 @@
+/**
+ * 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.net;
+
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+public class TestDNSResolver {
+
+    private static final String host1 = "r1-w1rack1-1111-2222.distributedlog.io";
+    private static final String host2 = "r2-w2rack2-3333-4444.distributedlog.io";
+
+    @Test(timeout = 20000)
+    public void testDNSResolverForRacks() {
+        DNSResolver dnsResolver = new DNSResolverForRacks("");
+
+        List<String> ipList = new ArrayList<String>();
+        ipList.add("192.0.0.1");
+        List<String> racks = dnsResolver.resolve(ipList);
+        assertEquals(DNSResolverForRacks.DEFAULT_RACK, racks.get(0));
+
+        List<String> unknownList = new ArrayList<String>();
+        unknownList.add("unknown");
+        racks = dnsResolver.resolve(unknownList);
+        assertEquals(DNSResolverForRacks.DEFAULT_RACK, racks.get(0));
+
+        List<String> r1List = new ArrayList<String>();
+        r1List.add(host1);
+        racks = dnsResolver.resolve(r1List);
+        assertEquals("/r1/w1rack1", racks.get(0));
+
+        List<String> r2List = new ArrayList<String>();
+        r2List.add(host2);
+        racks = dnsResolver.resolve(r2List);
+        assertEquals("/r2/w2rack2", racks.get(0));
+    }
+
+    @Test(timeout = 20000)
+    public void testDNSResolverForRows() {
+        DNSResolver dnsResolver = new DNSResolverForRows("");
+
+        List<String> ipList = new ArrayList<String>();
+        ipList.add("192.0.0.1");
+        List<String> rows = dnsResolver.resolve(ipList);
+        assertEquals(DNSResolverForRows.DEFAULT_ROW, rows.get(0));
+
+        List<String> unknownList = new ArrayList<String>();
+        unknownList.add("unknown");
+        rows = dnsResolver.resolve(unknownList);
+        assertEquals(DNSResolverForRows.DEFAULT_ROW, rows.get(0));
+
+        List<String> r1List = new ArrayList<String>();
+        r1List.add(host1);
+        rows = dnsResolver.resolve(r1List);
+        assertEquals("/r1/w1", rows.get(0));
+
+        List<String> r2List = new ArrayList<String>();
+        r2List.add(host2);
+        rows = dnsResolver.resolve(r2List);
+        assertEquals("/r2/w2", rows.get(0));
+    }
+
+    @Test(timeout = 20000)
+    public void testDNSResolverOverrides() {
+        DNSResolver dnsResolver = new DNSResolverForRacks("r1-w1rack1-1111-2222:r3;r2-w2rack2-3333-4444:r3");
+
+        List<String> r1List = new ArrayList<String>();
+        r1List.add(host1);
+        List<String> racks = dnsResolver.resolve(r1List);
+        assertEquals("/r3/w1rack1", racks.get(0));
+
+        List<String> r2List = new ArrayList<String>();
+        r2List.add(host2);
+        racks = dnsResolver.resolve(r2List);
+        assertEquals("/r3/w2rack2", racks.get(0));
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestNetUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestNetUtils.java
new file mode 100644
index 0000000..65ab9d9
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/net/TestNetUtils.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.net;
+
+import org.apache.bookkeeper.net.DNSToSwitchMapping;
+import org.junit.Test;
+
+import java.util.List;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test Cases of {@link NetUtils}.
+ */
+public class TestNetUtils {
+
+    static class DNSResolverWithDefaultConstructor implements DNSToSwitchMapping {
+
+        public DNSResolverWithDefaultConstructor() {}
+
+        @Override
+        public List<String> resolve(List<String> list) {
+            return list;
+        }
+
+        @Override
+        public void reloadCachedMappings() {
+            // no-op
+        }
+    }
+
+    static class DNSResolverWithUnknownConstructor implements DNSToSwitchMapping {
+
+        public DNSResolverWithUnknownConstructor(int var1, int var2, int var3) {}
+
+        @Override
+        public List<String> resolve(List<String> list) {
+            return list;
+        }
+
+        @Override
+        public void reloadCachedMappings() {
+            // no-op
+        }
+    }
+
+    @Test(timeout = 20000)
+    public void testGetDNSResolverWithOverrides() throws Exception {
+        DNSToSwitchMapping dnsResolver =
+                NetUtils.getDNSResolver(DNSResolverForRacks.class, "");
+        assertEquals("Should succeed to load " + DNSResolverForRacks.class,
+                dnsResolver.getClass(), DNSResolverForRacks.class);
+    }
+
+    @Test(timeout = 20000)
+    public void testGetDNSResolverWithDefaultConstructor() throws Exception {
+        DNSToSwitchMapping dnsResolver =
+                NetUtils.getDNSResolver(DNSResolverWithDefaultConstructor.class, "");
+        assertEquals("Should succeed to load " + DNSResolverWithDefaultConstructor.class,
+                dnsResolver.getClass(), DNSResolverWithDefaultConstructor.class);
+    }
+
+    @Test(timeout = 20000, expected = RuntimeException.class)
+    public void testGetDNSResolverWithUnknownConstructor() throws Exception {
+        NetUtils.getDNSResolver(DNSResolverWithUnknownConstructor.class, "");
+    }
+}
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
new file mode 100644
index 0000000..8949bec
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/rate/TestMovingAverageRate.java
@@ -0,0 +1,99 @@
+/**
+ * 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/selector/TestLogRecordSelectors.java b/distributedlog-core/src/test/java/org/apache/distributedlog/selector/TestLogRecordSelectors.java
new file mode 100644
index 0000000..53147bd
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/selector/TestLogRecordSelectors.java
@@ -0,0 +1,128 @@
+/**
+ * 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.selector;
+
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.DLSN;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+/**
+ * Test Case for {@link LogRecordSelector}s.
+ */
+public class TestLogRecordSelectors {
+
+    @Test(timeout = 60000)
+    public void testFirstRecordSelector() {
+        FirstRecordSelector selectorIncludeControlRecord =
+                new FirstRecordSelector(true);
+
+        for (int i = 0; i < 5; i++) {
+            selectorIncludeControlRecord.process(
+                    DLMTestUtil.getLogRecordWithDLSNInstance(
+                            new DLSN(1L, i * 2, 0L), i * 2, true));
+            selectorIncludeControlRecord.process(
+                    DLMTestUtil.getLogRecordWithDLSNInstance(
+                            new DLSN(1L, i * 2 + 1, 0L), i * 2 + 1));
+        }
+        assertEquals(new DLSN(1L, 0L, 0L), selectorIncludeControlRecord.result().getDlsn());
+
+        FirstRecordSelector selectorExcludeControlRecord =
+                new FirstRecordSelector(false);
+
+        for (int i = 0; i < 5; i++) {
+            selectorExcludeControlRecord.process(
+                    DLMTestUtil.getLogRecordWithDLSNInstance(
+                            new DLSN(1L, i * 2, 0L), i * 2, true));
+            selectorExcludeControlRecord.process(
+                    DLMTestUtil.getLogRecordWithDLSNInstance(
+                            new DLSN(1L, i * 2 + 1, 0L), i * 2 + 1));
+        }
+        assertEquals(new DLSN(1L, 1L, 0L), selectorExcludeControlRecord.result().getDlsn());
+    }
+
+    @Test(timeout = 60000)
+    public void testLastRecordSelector() {
+        LastRecordSelector selector = new LastRecordSelector();
+
+        for (int i = 0; i < 10; i++) {
+            selector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
+                    new DLSN(1L, i, 0L), i));
+        }
+        assertEquals(new DLSN(1L, 9L, 0L), selector.result().getDlsn());
+    }
+
+    @Test(timeout = 60000)
+    public void testFirstDLSNNotLessThanSelector() {
+        DLSN dlsn = new DLSN(5L, 5L, 0L);
+
+        FirstDLSNNotLessThanSelector largerSelector =
+                new FirstDLSNNotLessThanSelector(dlsn);
+        for (int i = 0; i < 10; i++) {
+            largerSelector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
+                    new DLSN(4L, i, 0L), i));
+        }
+        assertNull(largerSelector.result());
+
+        FirstDLSNNotLessThanSelector smallerSelector =
+                new FirstDLSNNotLessThanSelector(dlsn);
+        for (int i = 0; i < 10; i++) {
+            smallerSelector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
+                    new DLSN(6L, i, 0L), i));
+        }
+        assertEquals(new DLSN(6L, 0L, 0L), smallerSelector.result().getDlsn());
+
+        FirstDLSNNotLessThanSelector selector =
+                new FirstDLSNNotLessThanSelector(dlsn);
+        for (int i = 0; i < 10; i++) {
+            selector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
+                    new DLSN(5L, i, 0L), i));
+        }
+        assertEquals(dlsn, selector.result().getDlsn());
+    }
+
+    @Test(timeout = 60000)
+    public void testFirstTxIdNotLessThanSelector() {
+        long txId = 5 * 10 + 5;
+
+        FirstTxIdNotLessThanSelector largerSelector =
+                new FirstTxIdNotLessThanSelector(txId);
+        for (int i = 0; i < 10; i++) {
+            largerSelector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
+                    new DLSN(4L, i, 0L), 4 * 10 + i));
+        }
+        assertEquals(49, largerSelector.result().getTransactionId());
+
+        FirstTxIdNotLessThanSelector smallerSelector =
+                new FirstTxIdNotLessThanSelector(txId);
+        for (int i = 0; i < 10; i++) {
+            smallerSelector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
+                    new DLSN(6L, i, 0L), 6 * 10 + i));
+        }
+        assertEquals(6 * 10, smallerSelector.result().getTransactionId());
+
+        FirstTxIdNotLessThanSelector selector =
+                new FirstTxIdNotLessThanSelector(txId);
+        for (int i = 0; i < 10; i++) {
+            selector.process(DLMTestUtil.getLogRecordWithDLSNInstance(
+                    new DLSN(5L, i, 0L), 5 * 10 + i));
+        }
+        assertEquals(txId, selector.result().getTransactionId());
+    }
+}
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
new file mode 100644
index 0000000..47e2fae
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/tools/TestDistributedLogTool.java
@@ -0,0 +1,231 @@
+/**
+ * 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.tools;
+
+import java.net.URI;
+
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.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.exceptions.ZKException;
+import org.apache.distributedlog.tools.DistributedLogTool.*;
+import org.apache.bookkeeper.client.BKException.BKNoSuchLedgerExistsException;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.zookeeper.KeeperException;
+import org.junit.BeforeClass;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import static org.junit.Assert.*;
+
+public class TestDistributedLogTool extends TestDistributedLogBase {
+
+    static final Logger LOG = LoggerFactory.getLogger(TestDistributedLogTool.class);
+
+    static final String defaultLedgerPath = LocalDLMEmulator.getBkLedgerPath();
+    static final String defaultPath = "/test/namespace";
+    static final String defaultHost = "127.0.0.1";
+    static final String defaultPrivilegedZkAclId = "NathanielP";
+    static URI defaultUri = null;
+
+    static final String ADMIN_TOOL = org.apache.distributedlog.admin.DistributedLogAdmin.class.getName();
+
+    @BeforeClass
+    public static void setupDefaults() throws Exception {
+        defaultUri = DLMTestUtil.createDLMURI(zkPort, defaultPath);
+        DistributedLogManager dlm = DLMTestUtil.createNewDLM("DefaultStream", conf, defaultUri);
+        bindStream(defaultUri, defaultLedgerPath, defaultHost);
+        DLMTestUtil.generateCompletedLogSegments(dlm, conf, 3, 8192);
+        dlm.close();
+    }
+
+    private static int runTool(String[] args) throws Exception {
+        Tool tool = ReflectionUtils.newInstance(args[0], Tool.class);
+        String[] newArgs = new String[args.length - 1];
+        System.arraycopy(args, 1, newArgs, 0, newArgs.length);
+        int rc = tool.run(newArgs);
+        assertTrue(0 == rc);
+        return rc;
+    }
+
+    static void bindStream(URI uri, String ledgerPath, String zkHosts) throws Exception {
+        String[] args = new String[] { ADMIN_TOOL,
+            "bind",
+            "-c",
+            "-l", ledgerPath,
+            "-s", zkHosts,
+            "-f", uri.toString() };
+        runTool(args);
+    }
+
+    static void createStream(URI uri, String prefix, String expression, String zkAclId) throws Exception {
+        CreateCommand cmd = new CreateCommand();
+        cmd.setUri(defaultUri);
+        cmd.setPrefix(prefix);
+        cmd.setExpression(expression);
+        cmd.setForce(true);
+        cmd.setZkAclId(zkAclId);
+        assertEquals(0, cmd.runCmd());
+    }
+
+    void deleteStream(URI uri, String stream) throws Exception {
+        DeleteCommand cmd = new DeleteCommand();
+        cmd.setUri(defaultUri);
+        cmd.setStreamName(stream);
+        assertEquals(0, cmd.runCmd());
+    }
+
+    void list(URI uri) throws Exception {
+        ListCommand cmd = new ListCommand();
+        cmd.setUri(defaultUri);
+        assertEquals(0, cmd.runCmd());
+    }
+
+    @Test(timeout = 60000)
+    public void testToolCreate() throws Exception {
+        createStream(defaultUri, "0", "TestPrefix", null);
+    }
+
+    @Test(timeout = 60000)
+    public void testToolCreateZkAclId() throws Exception {
+        createStream(defaultUri, "0", "CreateAclStream", defaultPrivilegedZkAclId);
+        try {
+            DistributedLogManager dlm = DLMTestUtil.createNewDLM("0CreateAclStream", conf, defaultUri);
+            DLMTestUtil.generateCompletedLogSegments(dlm, conf, 3, 1000);
+            dlm.close();
+        } catch (ZKException ex) {
+            assertEquals(KeeperException.Code.NOAUTH, ex.getKeeperExceptionCode());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testToolDelete() throws Exception {
+        createStream(defaultUri, "1", "TestPrefix", null);
+        deleteStream(defaultUri, "1TestPrefix");
+    }
+
+    @Test(timeout = 60000)
+    public void testToolDeleteAllocPool() throws Exception {
+        try {
+            DeleteAllocatorPoolCommand cmd = new DeleteAllocatorPoolCommand();
+            cmd.setUri(defaultUri);
+            assertEquals(0, cmd.runCmd());
+            fail("should have failed");
+        } catch (org.apache.zookeeper.KeeperException.NoNodeException ex) {
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testToolList() throws Exception {
+        list(defaultUri);
+    }
+
+    @Test(timeout = 60000)
+    public void testToolDump() throws Exception {
+        DumpCommand cmd = new DumpCommand();
+        cmd.setUri(defaultUri);
+        cmd.setStreamName("DefaultStream");
+        cmd.setFromTxnId(Long.valueOf(0));
+        assertEquals(0, cmd.runCmd());
+    }
+
+    @Test(timeout = 60000)
+    public void testToolShow() throws Exception {
+        ShowCommand cmd = new ShowCommand();
+        cmd.setUri(defaultUri);
+        cmd.setStreamName("DefaultStream");
+        assertEquals(0, cmd.runCmd());
+    }
+
+    @Test(timeout = 60000)
+    public void testToolTruncate() throws Exception {
+        DistributedLogManager dlm = DLMTestUtil.createNewDLM("TruncateStream", conf, defaultUri);
+        DLMTestUtil.generateCompletedLogSegments(dlm, conf, 3, 1000);
+        dlm.close();
+
+        TruncateCommand cmd = new TruncateCommand();
+        cmd.setUri(defaultUri);
+        cmd.setFilter("TruncateStream");
+        cmd.setForce(true);
+        assertEquals(0, cmd.runCmd());
+    }
+
+    @Test(timeout = 60000)
+    public void testToolInspect() throws Exception {
+        InspectCommand cmd = new InspectCommand();
+        cmd.setUri(defaultUri);
+        cmd.setForce(true);
+        assertEquals(0, cmd.runCmd());
+    }
+
+    @Test(timeout = 60000)
+    public void testToolReadLastConfirmed() throws Exception {
+        ReadLastConfirmedCommand cmd = new ReadLastConfirmedCommand();
+        cmd.setUri(defaultUri);
+        cmd.setLedgerId(99999999);
+
+        // Too hard to predict ledger entry id. Settle for basicaly
+        // correct functionality.
+        try {
+            cmd.runCmd();
+        } catch (BKNoSuchLedgerExistsException ex) {
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testToolReadEntriesCommand() throws Exception {
+        ReadEntriesCommand cmd = new ReadEntriesCommand();
+        cmd.setUri(defaultUri);
+        cmd.setLedgerId(99999999);
+        try {
+            cmd.runCmd();
+        } catch (BKNoSuchLedgerExistsException ex) {
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testToolTruncateStream() throws Exception {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(conf);
+        confLocal.setLogSegmentCacheEnabled(false);
+        DistributedLogManager dlm = DLMTestUtil.createNewDLM("testToolTruncateStream", confLocal, defaultUri);
+        DLMTestUtil.generateCompletedLogSegments(dlm, confLocal, 3, 1000);
+
+        DLSN dlsn = new DLSN(2,1,0);
+        TruncateStreamCommand cmd = new TruncateStreamCommand();
+        cmd.setDlsn(dlsn);
+        cmd.setUri(defaultUri);
+        cmd.setStreamName("testToolTruncateStream");
+        cmd.setForce(true);
+
+        assertEquals(0, cmd.runCmd());
+
+        LogReader reader = dlm.getInputStream(0);
+        LogRecordWithDLSN record = reader.readNext(false);
+        assertEquals(dlsn, record.getDlsn());
+
+        reader.close();
+        dlm.close();
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestConfUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestConfUtils.java
new file mode 100644
index 0000000..df7270e
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestConfUtils.java
@@ -0,0 +1,53 @@
+/**
+ * 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.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.Configuration;
+import org.junit.Test;
+
+import static org.junit.Assert.*;
+
+public class TestConfUtils {
+
+    @Test(timeout = 60000)
+    public void testLoadConfiguration() {
+        Configuration conf1 = new CompositeConfiguration();
+        conf1.setProperty("key1", "value1");
+        conf1.setProperty("key2", "value2");
+        conf1.setProperty("key3", "value3");
+
+        Configuration conf2 = new CompositeConfiguration();
+        conf2.setProperty("bkc.key1", "bkc.value1");
+        conf2.setProperty("bkc.key4", "bkc.value4");
+
+        assertEquals("value1", conf1.getString("key1"));
+        assertEquals("value2", conf1.getString("key2"));
+        assertEquals("value3", conf1.getString("key3"));
+        assertEquals(null, conf1.getString("key4"));
+
+        ConfUtils.loadConfiguration(conf1, conf2, "bkc.");
+
+        assertEquals("bkc.value1", conf1.getString("key1"));
+        assertEquals("value2", conf1.getString("key2"));
+        assertEquals("value3", conf1.getString("key3"));
+        assertEquals("bkc.value4", conf1.getString("key4"));
+        assertEquals(null, conf1.getString("bkc.key1"));
+        assertEquals(null, conf1.getString("bkc.key4"));
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java
new file mode 100644
index 0000000..df49d49
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestDLUtils.java
@@ -0,0 +1,273 @@
+/**
+ * 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.collect.Lists;
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.LogSegmentMetadata.LogSegmentMetadataVersion;
+import org.apache.distributedlog.exceptions.UnexpectedException;
+import org.junit.Test;
+
+import java.util.List;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.*;
+
+/**
+ * Test Case for {@link DLUtils}
+ */
+public class TestDLUtils {
+
+    private static LogSegmentMetadata completedLogSegment(
+            long logSegmentSequenceNumber,
+            long fromTxnId,
+            long toTxnId) {
+        return completedLogSegment(
+                logSegmentSequenceNumber,
+                fromTxnId,
+                toTxnId,
+                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
+    }
+
+    private static LogSegmentMetadata completedLogSegment(
+        long logSegmentSequenceNumber,
+        long fromTxnId,
+        long toTxnId,
+        int version) {
+        return DLMTestUtil.completedLogSegment(
+                "/logsegment/" + fromTxnId,
+                fromTxnId,
+                fromTxnId,
+                toTxnId,
+                100,
+                logSegmentSequenceNumber,
+                999L,
+                0L,
+                version);
+    }
+
+    private static LogSegmentMetadata inprogressLogSegment(
+            long logSegmentSequenceNumber, long firstTxId) {
+        return inprogressLogSegment(
+                logSegmentSequenceNumber,
+                firstTxId,
+                LogSegmentMetadata.LEDGER_METADATA_CURRENT_LAYOUT_VERSION);
+    }
+
+    private static LogSegmentMetadata inprogressLogSegment(
+            long logSegmentSequenceNumber, long firstTxId, int version) {
+        return DLMTestUtil.inprogressLogSegment(
+                "/logsegment/" + firstTxId,
+                firstTxId,
+                firstTxId,
+                logSegmentSequenceNumber,
+                version);
+    }
+
+    @Test(timeout = 60000)
+    public void testFindLogSegmentNotLessThanTxnId() throws Exception {
+        long txnId = 999L;
+        // empty list
+        List<LogSegmentMetadata> emptyList = Lists.newArrayList();
+        assertEquals(-1, DLUtils.findLogSegmentNotLessThanTxnId(emptyList, txnId));
+
+        // list that all segment's txn id is larger than txn-id-to-search
+        List<LogSegmentMetadata> list1 = Lists.newArrayList(
+                completedLogSegment(1L, 1000L, 2000L));
+        assertEquals(-1, DLUtils.findLogSegmentNotLessThanTxnId(list1, txnId));
+
+        List<LogSegmentMetadata> list2 = Lists.newArrayList(
+                inprogressLogSegment(1L, 1000L));
+        assertEquals(-1, DLUtils.findLogSegmentNotLessThanTxnId(list2, txnId));
+
+        // the first log segment whose first txn id is less than txn-id-to-search
+        List<LogSegmentMetadata> list3 = Lists.newArrayList(
+                completedLogSegment(1L, 0L, 99L),
+                completedLogSegment(2L, 1000L, 2000L)
+        );
+        assertEquals(1, DLUtils.findLogSegmentNotLessThanTxnId(list3, txnId));
+
+        List<LogSegmentMetadata> list4 = Lists.newArrayList(
+                completedLogSegment(1L, 0L, 990L),
+                completedLogSegment(2L, 1000L, 2000L)
+        );
+        assertEquals(1, DLUtils.findLogSegmentNotLessThanTxnId(list4, txnId));
+
+        List<LogSegmentMetadata> list5 = Lists.newArrayList(
+                inprogressLogSegment(1L, 0L),
+                inprogressLogSegment(2L, 1000L)
+        );
+        assertEquals(0, DLUtils.findLogSegmentNotLessThanTxnId(list5, txnId));
+
+        // list that all segment's txn id is less than txn-id-to-search
+        List<LogSegmentMetadata> list6_0 = Lists.newArrayList(
+                completedLogSegment(1L, 100L, 200L));
+        assertEquals(0, DLUtils.findLogSegmentNotLessThanTxnId(list6_0, txnId));
+
+        List<LogSegmentMetadata> list6_1 = Lists.newArrayList(
+                completedLogSegment(1L, 100L, 199L),
+                completedLogSegment(2L, 200L, 299L));
+        assertEquals(1, DLUtils.findLogSegmentNotLessThanTxnId(list6_1, txnId));
+
+        List<LogSegmentMetadata> list7 = Lists.newArrayList(
+                inprogressLogSegment(1L, 100L));
+        assertEquals(0, DLUtils.findLogSegmentNotLessThanTxnId(list7, txnId));
+
+        // list that first segment's first txn id equals to txn-id-to-search
+        List<LogSegmentMetadata> list8 = Lists.newArrayList(
+                completedLogSegment(1L, 999L, 2000L));
+        assertEquals(0, DLUtils.findLogSegmentNotLessThanTxnId(list8, txnId));
+
+        List<LogSegmentMetadata> list9 = Lists.newArrayList(
+                inprogressLogSegment(1L, 999L));
+        assertEquals(0, DLUtils.findLogSegmentNotLessThanTxnId(list9, txnId));
+
+        List<LogSegmentMetadata> list10 = Lists.newArrayList(
+                completedLogSegment(1L, 0L, 999L),
+                completedLogSegment(2L, 999L, 2000L));
+        assertEquals(0, DLUtils.findLogSegmentNotLessThanTxnId(list10, txnId));
+
+        List<LogSegmentMetadata> list11 = Lists.newArrayList(
+                completedLogSegment(1L, 0L, 99L),
+                completedLogSegment(2L, 999L, 2000L));
+        assertEquals(1, DLUtils.findLogSegmentNotLessThanTxnId(list11, txnId));
+
+        List<LogSegmentMetadata> list12 = Lists.newArrayList(
+                inprogressLogSegment(1L, 0L),
+                inprogressLogSegment(2L, 999L));
+        assertEquals(1, DLUtils.findLogSegmentNotLessThanTxnId(list12, txnId));
+    }
+
+    @Test(timeout = 60000)
+    public void testNextLogSegmentSequenceNumber() throws Exception {
+        List<LogSegmentMetadata> v1List = Lists.newArrayList(
+                completedLogSegment(2L, 100L, 199L, LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value),
+                completedLogSegment(1L, 0L, 99L, LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value));
+        assertNull(DLUtils.nextLogSegmentSequenceNumber(v1List));
+
+        List<LogSegmentMetadata> afterV1List = Lists.newArrayList(
+                completedLogSegment(2L, 100L, 199L),
+                completedLogSegment(1L, 0L, 99L));
+        assertEquals((Long) 3L, DLUtils.nextLogSegmentSequenceNumber(afterV1List));
+
+        List<LogSegmentMetadata> mixList1 = Lists.newArrayList(
+                completedLogSegment(2L, 100L, 199L, LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value),
+                completedLogSegment(1L, 0L, 99L));
+        assertEquals((Long) 3L, DLUtils.nextLogSegmentSequenceNumber(mixList1));
+
+        List<LogSegmentMetadata> mixList2 = Lists.newArrayList(
+                completedLogSegment(2L, 100L, 199L),
+                completedLogSegment(1L, 0L, 99L, LogSegmentMetadataVersion.VERSION_V1_ORIGINAL.value));
+        assertEquals((Long) 3L, DLUtils.nextLogSegmentSequenceNumber(mixList2));
+    }
+
+    @Test(timeout = 60000, expected = UnexpectedException.class)
+    public void testUnexpectedExceptionOnComputeStartSequenceId() throws Exception {
+        List<LogSegmentMetadata> segments = Lists.newArrayList(
+                inprogressLogSegment(3L, 201L),
+                inprogressLogSegment(2L, 101L),
+                completedLogSegment(1L, 1L, 100L).mutator().setStartSequenceId(1L).build()
+        );
+        DLUtils.computeStartSequenceId(segments, segments.get(0));
+    }
+
+    @Test(timeout = 60000)
+    public void testComputeStartSequenceIdOnEmptyList() throws Exception {
+        List<LogSegmentMetadata> emptyList = Lists.newArrayList();
+        assertEquals(0L, DLUtils.computeStartSequenceId(emptyList, inprogressLogSegment(1L, 1L)));
+    }
+
+    @Test(timeout = 60000)
+    public void testComputeStartSequenceIdOnLowerSequenceNumberSegment() throws Exception {
+        List<LogSegmentMetadata> segments = Lists.newArrayList(
+                completedLogSegment(3L, 201L, 300L).mutator().setStartSequenceId(201L).build(),
+                completedLogSegment(2L, 101L, 200L).mutator().setStartSequenceId(101L).build()
+        );
+        assertEquals(0L, DLUtils.computeStartSequenceId(segments, inprogressLogSegment(1L, 1L)));
+    }
+
+    @Test(timeout = 60000)
+    public void testComputeStartSequenceIdOnHigherSequenceNumberSegment() throws Exception {
+        List<LogSegmentMetadata> segments = Lists.newArrayList(
+                completedLogSegment(3L, 201L, 300L).mutator().setStartSequenceId(201L).build(),
+                completedLogSegment(2L, 101L, 200L).mutator().setStartSequenceId(101L).build()
+        );
+        assertEquals(0L, DLUtils.computeStartSequenceId(segments, inprogressLogSegment(5L, 401L)));
+    }
+
+    @Test(timeout = 60000)
+    public void testComputeStartSequenceId() throws Exception {
+        List<LogSegmentMetadata> segments = Lists.newArrayList(
+                completedLogSegment(3L, 201L, 300L).mutator()
+                        .setStartSequenceId(201L).setRecordCount(100).build(),
+                completedLogSegment(2L, 101L, 200L).mutator()
+                        .setStartSequenceId(101L).setRecordCount(100).build()
+        );
+        assertEquals(301L, DLUtils.computeStartSequenceId(segments, inprogressLogSegment(4L, 301L)));
+    }
+
+    @Test(timeout = 60000)
+    public void testSerDeLogSegmentSequenceNumber() throws Exception {
+        long sn = 123456L;
+        byte[] snData = Long.toString(sn).getBytes(UTF_8);
+        assertEquals("Deserialization should succeed",
+                sn, DLUtils.deserializeLogSegmentSequenceNumber(snData));
+        assertArrayEquals("Serialization should succeed",
+                snData, DLUtils.serializeLogSegmentSequenceNumber(sn));
+    }
+
+    @Test(timeout = 60000, expected = NumberFormatException.class)
+    public void testDeserilizeInvalidLSSN() throws Exception {
+        byte[] corruptedData = "corrupted-lssn".getBytes(UTF_8);
+        DLUtils.deserializeLogSegmentSequenceNumber(corruptedData);
+    }
+
+    @Test(timeout = 60000)
+    public void testSerDeLogRecordTxnId() throws Exception {
+        long txnId = 123456L;
+        byte[] txnData = Long.toString(txnId).getBytes(UTF_8);
+        assertEquals("Deserialization should succeed",
+                txnId, DLUtils.deserializeTransactionId(txnData));
+        assertArrayEquals("Serialization should succeed",
+                txnData, DLUtils.serializeTransactionId(txnId));
+    }
+
+    @Test(timeout = 60000, expected = NumberFormatException.class)
+    public void testDeserilizeInvalidLogRecordTxnId() throws Exception {
+        byte[] corruptedData = "corrupted-txn-id".getBytes(UTF_8);
+        DLUtils.deserializeTransactionId(corruptedData);
+    }
+
+    @Test(timeout = 60000)
+    public void testSerDeLedgerId() throws Exception {
+        long ledgerId = 123456L;
+        byte[] ledgerIdData = Long.toString(ledgerId).getBytes(UTF_8);
+        assertEquals("Deserialization should succeed",
+                ledgerId, DLUtils.bytes2LogSegmentId(ledgerIdData));
+        assertArrayEquals("Serialization should succeed",
+                ledgerIdData, DLUtils.logSegmentId2Bytes(ledgerId));
+    }
+
+    @Test(timeout = 60000, expected = NumberFormatException.class)
+    public void testDeserializeInvalidLedgerId() throws Exception {
+        byte[] corruptedData = "corrupted-ledger-id".getBytes(UTF_8);
+        DLUtils.bytes2LogSegmentId(corruptedData);
+    }
+
+}
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
new file mode 100644
index 0000000..f9e4eb8
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestFutureUtils.java
@@ -0,0 +1,71 @@
+/**
+ * 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
new file mode 100644
index 0000000..802649d
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestPermitManager.java
@@ -0,0 +1,92 @@
+/**
+ * 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.distributedlog.zk.LimitedPermitManager;
+import org.junit.Test;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+
+import static org.junit.Assert.*;
+
+public class TestPermitManager {
+
+    @Test(timeout = 60000)
+    public void testUnlimitedPermitManager() {
+        PermitManager pm = PermitManager.UNLIMITED_PERMIT_MANAGER;
+        List<PermitManager.Permit> permits = new ArrayList<PermitManager.Permit>();
+        for (int i = 0; i < 10; i++) {
+            permits.add(pm.acquirePermit());
+        }
+        for (int i = 0; i < 10; i++) {
+            assertTrue(permits.get(i).isAllowed());
+            pm.releasePermit(permits.get(i));
+        }
+        PermitManager.Permit permit = pm.acquirePermit();
+        pm.disallowObtainPermits(permit);
+        pm.releasePermit(permit);
+
+        for (int i = 0; i < 10; i++) {
+            permits.add(pm.acquirePermit());
+        }
+        for (int i = 0; i < 10; i++) {
+            assertTrue(permits.get(i).isAllowed());
+            pm.releasePermit(permits.get(i));
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testLimitedPermitManager() {
+        ScheduledExecutorService executorService = Executors.newSingleThreadScheduledExecutor();
+        PermitManager pm = new LimitedPermitManager(1, 0, TimeUnit.SECONDS, executorService);
+        PermitManager.Permit permit1 = pm.acquirePermit();
+        PermitManager.Permit permit2 = pm.acquirePermit();
+        assertTrue(permit1.isAllowed());
+        assertFalse(permit2.isAllowed());
+        pm.releasePermit(permit2);
+        PermitManager.Permit permit3 = pm.acquirePermit();
+        assertFalse(permit3.isAllowed());
+        pm.releasePermit(permit3);
+        pm.releasePermit(permit1);
+        PermitManager.Permit permit4 = pm.acquirePermit();
+        assertTrue(permit4.isAllowed());
+        pm.releasePermit(permit4);
+
+        PermitManager pm2 = new LimitedPermitManager(2, 0, TimeUnit.SECONDS, executorService);
+
+        PermitManager.Permit permit5 = pm2.acquirePermit();
+        PermitManager.Permit permit6 = pm2.acquirePermit();
+        assertTrue(permit5.isAllowed());
+        assertTrue(permit6.isAllowed());
+        assertTrue(pm2.disallowObtainPermits(permit5));
+        assertFalse(pm2.disallowObtainPermits(permit6));
+        pm2.releasePermit(permit5);
+        pm2.releasePermit(permit6);
+        PermitManager.Permit permit7 = pm2.acquirePermit();
+        assertFalse(permit7.isAllowed());
+        pm2.releasePermit(permit7);
+        pm2.allowObtainPermits();
+        PermitManager.Permit permit8 = pm2.acquirePermit();
+        assertTrue(permit8.isAllowed());
+        pm2.releasePermit(permit2);
+    }
+}
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
new file mode 100644
index 0000000..7bfe5ed
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestSafeQueueingFuturePool.java
@@ -0,0 +1,205 @@
+/**
+ * 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/TestTimeSequencer.java b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestTimeSequencer.java
new file mode 100644
index 0000000..ccb82c5
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestTimeSequencer.java
@@ -0,0 +1,42 @@
+/**
+ * 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.junit.Test;
+import static org.junit.Assert.*;
+
+public class TestTimeSequencer {
+
+    @Test(timeout = 60000)
+    public void testNonDecreasingId() {
+        TimeSequencer sequencer = new TimeSequencer();
+        long lastId = System.currentTimeMillis() + 3600000;
+        sequencer.setLastId(lastId);
+        for (int i = 0; i < 10; i++) {
+            assertEquals(lastId, sequencer.nextId());
+        }
+        sequencer.setLastId(15);
+        long prevId = 15;
+        for (int i = 0; i < 10; i++) {
+            long newId = sequencer.nextId();
+            assertTrue("id should not decrease",
+                    newId >= prevId);
+            prevId = newId;
+        }
+    }
+}
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
new file mode 100644
index 0000000..3f1689b
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/util/TestUtils.java
@@ -0,0 +1,125 @@
+/**
+ * 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.Optional;
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.TestZooKeeperClientBuilder;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.ZooKeeperClusterTestCase;
+import org.apache.bookkeeper.meta.ZkVersion;
+import org.apache.bookkeeper.versioning.Versioned;
+import org.apache.zookeeper.AsyncCallback;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.ZooDefs;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.util.concurrent.CountDownLatch;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.*;
+
+/**
+ * Test Utils
+ */
+public class TestUtils extends ZooKeeperClusterTestCase {
+
+    private final static int sessionTimeoutMs = 30000;
+
+    private ZooKeeperClient zkc;
+
+    @Before
+    public void setup() throws Exception {
+        zkc = TestZooKeeperClientBuilder.newBuilder()
+                .name("zkc")
+                .uri(DLMTestUtil.createDLMURI(zkPort, "/"))
+                .sessionTimeoutMs(sessionTimeoutMs)
+                .build();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        zkc.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testZkAsyncCreateFulPathOptimisticRecursive() throws Exception {
+        String path1 = "/a/b/c/d";
+        Optional<String> parentPathShouldNotCreate = Optional.absent();
+        final CountDownLatch doneLatch1 = new CountDownLatch(1);
+        Utils.zkAsyncCreateFullPathOptimisticRecursive(zkc, path1, parentPathShouldNotCreate,
+                new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT,
+                new AsyncCallback.StringCallback() {
+                    @Override
+                    public void processResult(int rc, String path, Object ctx, String name) {
+                        doneLatch1.countDown();
+                    }
+                }, null);
+        doneLatch1.await();
+        assertNotNull(zkc.get().exists(path1, false));
+
+        String path2 = "/a/b/c/d/e/f/g";
+        parentPathShouldNotCreate = Optional.of("/a/b/c/d/e");
+        final CountDownLatch doneLatch2 = new CountDownLatch(1);
+        Utils.zkAsyncCreateFullPathOptimisticRecursive(zkc, path2, parentPathShouldNotCreate,
+                new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT,
+                new AsyncCallback.StringCallback() {
+                    @Override
+                    public void processResult(int rc, String path, Object ctx, String name) {
+                        doneLatch2.countDown();
+                    }
+                }, null);
+        doneLatch2.await();
+        assertNull(zkc.get().exists("/a/b/c/d/e", false));
+        assertNull(zkc.get().exists("/a/b/c/d/e/f", false));
+        assertNull(zkc.get().exists("/a/b/c/d/e/f/g", false));
+
+        parentPathShouldNotCreate = Optional.of("/a/b");
+        final CountDownLatch doneLatch3 = new CountDownLatch(1);
+        Utils.zkAsyncCreateFullPathOptimisticRecursive(zkc, path2, parentPathShouldNotCreate,
+                new byte[0], ZooDefs.Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT,
+                new AsyncCallback.StringCallback() {
+                    @Override
+                    public void processResult(int rc, String path, Object ctx, String name) {
+                        doneLatch3.countDown();
+                    }
+                }, null);
+        doneLatch3.await();
+        assertNotNull(zkc.get().exists(path2, false));
+    }
+
+    @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));
+        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,
+                zkc.getDefaultACL(), CreateMode.PERSISTENT));
+        data = FutureUtils.result(Utils.zkGetData(zkc.get(), path2, false));
+        assertArrayEquals("Data should return as written",
+                rawData, data.getValue());
+        assertEquals("Version should be zero",
+                0, ((ZkVersion) data.getVersion()).getZnodeVersion());
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKTransaction.java b/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKTransaction.java
new file mode 100644
index 0000000..bb1fbb6
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKTransaction.java
@@ -0,0 +1,97 @@
+/**
+ * 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.zk;
+
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.exceptions.DLIllegalStateException;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
+import org.junit.Test;
+
+import java.util.concurrent.CountDownLatch;
+import javax.annotation.Nullable;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+/**
+ * Test Case for zookeeper transaction
+ */
+public class TestZKTransaction {
+
+    static class CountDownZKOp extends ZKOp {
+
+        final CountDownLatch commitLatch;
+        final CountDownLatch abortLatch;
+
+        CountDownZKOp(CountDownLatch commitLatch,
+                      CountDownLatch abortLatch) {
+            super(mock(Op.class));
+            this.commitLatch = commitLatch;
+            this.abortLatch = abortLatch;
+        }
+
+        @Override
+        protected void commitOpResult(OpResult opResult) {
+            this.commitLatch.countDown();
+        }
+
+        @Override
+        protected void abortOpResult(Throwable t, @Nullable OpResult opResult) {
+            this.abortLatch.countDown();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testProcessNullResults() throws Exception {
+        ZooKeeperClient zkc = mock(ZooKeeperClient.class);
+        ZKTransaction transaction = new ZKTransaction(zkc);
+        int numOps = 3;
+        final CountDownLatch commitLatch = new CountDownLatch(numOps);
+        final CountDownLatch abortLatch = new CountDownLatch(numOps);
+        for (int i = 0; i < numOps; i++) {
+            transaction.addOp(new CountDownZKOp(commitLatch, abortLatch));
+        }
+        transaction.processResult(
+                KeeperException.Code.CONNECTIONLOSS.intValue(),
+                "test-path",
+                null,
+                null);
+        abortLatch.await();
+        assertEquals(0, abortLatch.getCount());
+        assertEquals(numOps, commitLatch.getCount());
+    }
+
+    @Test(timeout = 60000)
+    public void testAbortTransaction() throws Exception {
+        ZooKeeperClient zkc = mock(ZooKeeperClient.class);
+        ZKTransaction transaction = new ZKTransaction(zkc);
+        int numOps = 3;
+        final CountDownLatch commitLatch = new CountDownLatch(numOps);
+        final CountDownLatch abortLatch = new CountDownLatch(numOps);
+        for (int i = 0; i < numOps; i++) {
+            transaction.addOp(new CountDownZKOp(commitLatch, abortLatch));
+        }
+        transaction.abort(new DLIllegalStateException("Illegal State"));
+        abortLatch.await();
+        assertEquals(0, abortLatch.getCount());
+        assertEquals(numOps, commitLatch.getCount());
+    }
+
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKVersionedSetOp.java b/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKVersionedSetOp.java
new file mode 100644
index 0000000..064459e
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKVersionedSetOp.java
@@ -0,0 +1,86 @@
+/**
+ * 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.zk;
+
+import org.apache.distributedlog.util.Transaction;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Op;
+import org.apache.zookeeper.OpResult;
+import org.junit.Test;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.Assert.*;
+import static org.mockito.Mockito.*;
+
+/**
+ * Test Case for versioned set operation
+ */
+public class TestZKVersionedSetOp {
+
+    @Test(timeout = 60000)
+    public void testAbortNullOpResult() throws Exception {
+        final AtomicReference<Throwable> exception =
+                new AtomicReference<Throwable>();
+        final CountDownLatch latch = new CountDownLatch(1);
+        ZKVersionedSetOp versionedSetOp =
+                new ZKVersionedSetOp(mock(Op.class), new Transaction.OpListener<Version>() {
+                    @Override
+                    public void onCommit(Version r) {
+                        // no-op
+                    }
+
+                    @Override
+                    public void onAbort(Throwable t) {
+                        exception.set(t);
+                        latch.countDown();
+                    }
+                });
+        KeeperException ke = KeeperException.create(KeeperException.Code.SESSIONEXPIRED);
+        versionedSetOp.abortOpResult(ke, null);
+        latch.await();
+        assertTrue(ke == exception.get());
+    }
+
+    @Test(timeout = 60000)
+    public void testAbortOpResult() throws Exception {
+        final AtomicReference<Throwable> exception =
+                new AtomicReference<Throwable>();
+        final CountDownLatch latch = new CountDownLatch(1);
+        ZKVersionedSetOp versionedSetOp =
+                new ZKVersionedSetOp(mock(Op.class), new Transaction.OpListener<Version>() {
+                    @Override
+                    public void onCommit(Version r) {
+                        // no-op
+                    }
+
+                    @Override
+                    public void onAbort(Throwable t) {
+                        exception.set(t);
+                        latch.countDown();
+                    }
+                });
+        KeeperException ke = KeeperException.create(KeeperException.Code.SESSIONEXPIRED);
+        OpResult opResult = new OpResult.ErrorResult(KeeperException.Code.NONODE.intValue());
+        versionedSetOp.abortOpResult(ke, opResult);
+        latch.await();
+        assertTrue(exception.get() instanceof KeeperException.NoNodeException);
+    }
+}
diff --git a/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKWatcherManager.java b/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKWatcherManager.java
new file mode 100644
index 0000000..ad62e2f
--- /dev/null
+++ b/distributedlog-core/src/test/java/org/apache/distributedlog/zk/TestZKWatcherManager.java
@@ -0,0 +1,80 @@
+/**
+ * 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.zk;
+
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.junit.Test;
+
+import java.util.LinkedList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+
+import static org.junit.Assert.*;
+
+public class TestZKWatcherManager {
+
+    @Test(timeout = 60000)
+    public void testRegisterUnregisterWatcher() throws Exception {
+        ZKWatcherManager watcherManager = ZKWatcherManager.newBuilder()
+                .name("test-register-unregister-watcher")
+                .zkc(null)
+                .statsLogger(NullStatsLogger.INSTANCE)
+                .build();
+        String path = "/test-register-unregister-watcher";
+        final List<WatchedEvent> events = new LinkedList<WatchedEvent>();
+        final CountDownLatch latch = new CountDownLatch(2);
+        Watcher watcher = new Watcher() {
+            @Override
+            public void process(WatchedEvent event) {
+                events.add(event);
+                latch.countDown();
+            }
+        };
+        watcherManager.registerChildWatcher(path, watcher);
+
+        // fire the event
+        WatchedEvent event0 = new WatchedEvent(
+                Watcher.Event.EventType.NodeCreated,
+                Watcher.Event.KeeperState.SyncConnected,
+                path);
+        WatchedEvent event1 = new WatchedEvent(
+                Watcher.Event.EventType.None,
+                Watcher.Event.KeeperState.SyncConnected,
+                path);
+        WatchedEvent event2 = new WatchedEvent(
+                Watcher.Event.EventType.NodeChildrenChanged,
+                Watcher.Event.KeeperState.SyncConnected,
+                path);
+        watcher.process(event1);
+        watcher.process(event2);
+
+        latch.await();
+
+        assertEquals(2, events.size());
+        assertEquals(event1, events.get(0));
+        assertEquals(event2, events.get(1));
+
+        // unregister watcher
+        watcherManager.unregisterChildWatcher(path, watcher, true);
+        // unregister gauges
+        watcherManager.unregisterGauges();
+        assertEquals(0, watcherManager.childWatches.size());
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/DLSN.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/DLSN.java
deleted file mode 100644
index 57cc0ef..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/DLSN.java
+++ /dev/null
@@ -1,266 +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 com.twitter.distributedlog;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.common.annotations.VisibleForTesting;
-import java.nio.ByteBuffer;
-import org.apache.commons.codec.binary.Base64;
-import org.apache.commons.codec.binary.Hex;
-
-
-/**
- * DistributedLog Sequence Number (DLSN) is the system generated sequence number for log record.
- *
- * <p>DLSN is comprised with 3 components:
- * <ul>
- * <li>LogSegment Sequence Number: the sequence number of log segment that the record is written in
- * <li>Entry Id: the entry id of the entry that the record is batched in
- * <li>Slot Id: the slot id that the record is in the entry
- * </ul>
- *
- * @see LogRecordWithDLSN
- */
-public class DLSN implements Comparable<DLSN> {
-
-    public static final byte VERSION0 = (byte) 0;
-    public static final byte VERSION1 = (byte) 1;
-
-    // The initial DLSN that DL starts with
-    public static final DLSN InitialDLSN = new DLSN(1, 0 , 0);
-    // The non-inclusive lower bound DLSN
-    public static final DLSN NonInclusiveLowerBound = new DLSN(1, 0 , -1);
-    // Invalid DLSN
-    public static final DLSN InvalidDLSN = new DLSN(0, -1, -1);
-
-    static final byte CUR_VERSION = VERSION1;
-    static final int VERSION0_LEN = Long.SIZE * 3 + Byte.SIZE;
-    static final int VERSION1_LEN = Long.SIZE * 3 / Byte.SIZE + 1;
-
-    private final long logSegmentSequenceNo;
-    private final long entryId;
-    private final long slotId;
-
-    public DLSN(long logSegmentSequenceNo, long entryId, long slotId) {
-        this.logSegmentSequenceNo = logSegmentSequenceNo;
-        this.entryId = entryId;
-        this.slotId = slotId;
-    }
-
-    /**
-     * Return the sequence number of the log segment that the record is written to.
-     *
-     * @return sequence number of the log segment that the record is written to.
-     */
-    public long getLogSegmentSequenceNo() {
-        return logSegmentSequenceNo;
-    }
-
-    /**
-     * Return the entry id of the batch that the record is written to.
-     *
-     * @return entry id of the batch that the record is written to.
-     */
-    public long getEntryId() {
-        return entryId;
-    }
-
-    /**
-     * Return the slot id in the batch that the record is written to.
-     *
-     * @return slot id in the batch that the record is written to.
-     */
-    public long getSlotId() {
-        return slotId;
-    }
-
-    @Override
-    public int compareTo(DLSN that) {
-        if (this.logSegmentSequenceNo != that.logSegmentSequenceNo) {
-            return (this.logSegmentSequenceNo < that.logSegmentSequenceNo) ? -1 : 1;
-        } else if (this.entryId != that.entryId) {
-            return (this.entryId < that.entryId) ? -1 : 1;
-        } else {
-            return (this.slotId < that.slotId) ? -1 : (this.slotId == that.slotId) ? 0 : 1;
-        }
-    }
-
-    /**
-     * Serialize the DLSN into bytes with current version.
-     *
-     * @return the serialized bytes
-     */
-    public byte[] serializeBytes() {
-        return serializeBytes(CUR_VERSION);
-    }
-
-    /**
-     * Serialize the DLSN into bytes with given <code>version</code>.
-     *
-     * @param version
-     *          version to serialize the DLSN
-     * @return the serialized bytes
-     */
-    public byte[] serializeBytes(byte version) {
-        checkArgument(version <= CUR_VERSION && version >= VERSION0);
-        byte[] data = new byte[CUR_VERSION == version ? VERSION1_LEN : VERSION0_LEN];
-        ByteBuffer bb = ByteBuffer.wrap(data);
-        bb.put(version);
-        bb.putLong(logSegmentSequenceNo);
-        bb.putLong(entryId);
-        bb.putLong(slotId);
-        return data;
-    }
-
-    /**
-     * Serialize the DLSN into base64 encoded string.
-     *
-     * @return serialized base64 string
-     * @see #serializeBytes()
-     */
-    public String serialize() {
-        return serialize(CUR_VERSION);
-    }
-
-    /**
-     * Serialize the DLSN into base64 encoded string with given <code>version</code>.
-     *
-     * @param version
-     *          version to serialize the DLSN
-     * @return the serialized base64 string
-     * @see #serializeBytes(byte)
-     */
-    public String serialize(byte version) {
-        return Base64.encodeBase64String(serializeBytes(version));
-    }
-
-    /**
-     * Deserialize the DLSN from base64 encoded string <code>dlsn</code>.
-     *
-     * @param dlsn
-     *          base64 encoded string
-     * @return dlsn
-     */
-    public static DLSN deserialize(String dlsn) {
-        byte[] data = Base64.decodeBase64(dlsn);
-        return deserializeBytes(data);
-    }
-
-    /**
-     * Deserialize the DLSN from bytes array.
-     *
-     * @param data
-     *          serialized bytes
-     * @return dlsn
-     */
-    public static DLSN deserializeBytes(byte[] data) {
-        ByteBuffer bb = ByteBuffer.wrap(data);
-        byte version = bb.get();
-        if (VERSION0 == version) {
-            if (VERSION0_LEN != data.length) {
-                throw new IllegalArgumentException("Invalid version zero DLSN " + Hex.encodeHexString(data));
-            }
-        } else if (VERSION1 == version) {
-            if (VERSION1_LEN != data.length) {
-                throw new IllegalArgumentException("Invalid version one DLSN " + Hex.encodeHexString(data));
-            }
-        } else {
-            throw new IllegalArgumentException("Invalid DLSN : version = "
-                    + version + ", " + Hex.encodeHexString(data));
-        }
-        return new DLSN(bb.getLong(), bb.getLong(), bb.getLong());
-    }
-
-    // Keep original version0 logic for testing.
-    @VisibleForTesting
-    static DLSN deserialize0(String dlsn) {
-        byte[] data = Base64.decodeBase64(dlsn);
-        ByteBuffer bb = ByteBuffer.wrap(data);
-        byte version = bb.get();
-        if (VERSION0 != version || VERSION0_LEN != data.length) {
-            throw new IllegalArgumentException("Invalid DLSN " + dlsn);
-        }
-        return new DLSN(bb.getLong(), bb.getLong(), bb.getLong());
-    }
-
-    @Override
-    public String toString() {
-        return "DLSN{"
-            + "logSegmentSequenceNo=" + logSegmentSequenceNo
-            + ", entryId=" + entryId
-            + ", slotId=" + slotId
-            + '}';
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (!(o instanceof DLSN)) {
-            return false;
-        }
-
-        DLSN dlsn = (DLSN) o;
-
-        if (entryId != dlsn.entryId) {
-            return false;
-        }
-        if (logSegmentSequenceNo != dlsn.logSegmentSequenceNo) {
-            return false;
-        }
-        if (slotId != dlsn.slotId) {
-            return false;
-        }
-
-        return true;
-    }
-
-    @Override
-    public int hashCode() {
-        int result = (int) (logSegmentSequenceNo ^ (logSegmentSequenceNo >>> 32));
-        result = 31 * result + (int) (entryId ^ (entryId >>> 32));
-        result = 31 * result + (int) (slotId ^ (slotId >>> 32));
-        return result;
-    }
-
-    /**
-     * Positions to a DLSN greater than the current value.
-     *
-     * <p>This may not correspond to an actual LogRecord, its just used by the positioning logic
-     * to position the reader.
-     *
-     * @return the next DLSN
-     */
-    public DLSN getNextDLSN() {
-        return new DLSN(logSegmentSequenceNo, entryId, slotId + 1);
-    }
-
-    /**
-     * Positions to next log segment than the current value.
-     *
-     * <p>this may not correspond to an actual LogRecord, its just used by the positioning logic
-     * to position the reader.
-     *
-     * @return the next DLSN
-     */
-    public DLSN positionOnTheNextLedger() {
-        return new DLSN(logSegmentSequenceNo + 1 , 0, 0);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/EnvelopedRecordSetReader.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/EnvelopedRecordSetReader.java
deleted file mode 100644
index dae8403..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/EnvelopedRecordSetReader.java
+++ /dev/null
@@ -1,118 +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 com.twitter.distributedlog;
-
-import static com.twitter.distributedlog.LogRecordSet.COMPRESSION_CODEC_LZ4;
-import static com.twitter.distributedlog.LogRecordSet.METADATA_COMPRESSION_MASK;
-import static com.twitter.distributedlog.LogRecordSet.METADATA_VERSION_MASK;
-import static com.twitter.distributedlog.LogRecordSet.NULL_OP_STATS_LOGGER;
-import static com.twitter.distributedlog.LogRecordSet.VERSION;
-
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.distributedlog.io.CompressionUtils;
-import java.io.DataInputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.nio.ByteBuffer;
-
-
-/**
- * Record reader to read records from an enveloped entry buffer.
- */
-class EnvelopedRecordSetReader implements LogRecordSet.Reader {
-
-    private final long logSegmentSeqNo;
-    private final long entryId;
-    private final long transactionId;
-    private final long startSequenceId;
-    private int numRecords;
-    private final ByteBuffer reader;
-
-    // slot id
-    private long slotId;
-    private int position;
-
-    EnvelopedRecordSetReader(long logSegmentSeqNo,
-                             long entryId,
-                             long transactionId,
-                             long startSlotId,
-                             int startPositionWithinLogSegment,
-                             long startSequenceId,
-                             InputStream in)
-            throws IOException {
-        this.logSegmentSeqNo = logSegmentSeqNo;
-        this.entryId = entryId;
-        this.transactionId = transactionId;
-        this.slotId = startSlotId;
-        this.position = startPositionWithinLogSegment;
-        this.startSequenceId = startSequenceId;
-
-        // read data
-        DataInputStream src = new DataInputStream(in);
-        int metadata = src.readInt();
-        int version = metadata & METADATA_VERSION_MASK;
-        if (version != VERSION) {
-            throw new IOException(String.format("Version mismatch while reading. Received: %d,"
-                + " Required: %d", version, VERSION));
-        }
-        int codecCode = metadata & METADATA_COMPRESSION_MASK;
-        this.numRecords = src.readInt();
-        int originDataLen = src.readInt();
-        int actualDataLen = src.readInt();
-        byte[] compressedData = new byte[actualDataLen];
-        src.readFully(compressedData);
-
-        if (COMPRESSION_CODEC_LZ4 == codecCode) {
-            CompressionCodec codec = CompressionUtils.getCompressionCodec(CompressionCodec.Type.LZ4);
-            byte[] decompressedData = codec.decompress(compressedData, 0, actualDataLen,
-                    originDataLen, NULL_OP_STATS_LOGGER);
-            this.reader = ByteBuffer.wrap(decompressedData);
-        } else {
-            if (originDataLen != actualDataLen) {
-                throw new IOException("Inconsistent data length found for a non-compressed record set : original = "
-                        + originDataLen + ", actual = " + actualDataLen);
-            }
-            this.reader = ByteBuffer.wrap(compressedData);
-        }
-    }
-
-    @Override
-    public LogRecordWithDLSN nextRecord() throws IOException {
-        if (numRecords <= 0) {
-            return null;
-        }
-
-        int recordLen = reader.getInt();
-        byte[] recordData = new byte[recordLen];
-        reader.get(recordData);
-        DLSN dlsn = new DLSN(logSegmentSeqNo, entryId, slotId);
-
-        LogRecordWithDLSN record =
-                new LogRecordWithDLSN(dlsn, startSequenceId);
-        record.setPositionWithinLogSegment(position);
-        record.setTransactionId(transactionId);
-        record.setPayload(recordData);
-
-        ++slotId;
-        ++position;
-        --numRecords;
-
-        return record;
-    }
-
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/EnvelopedRecordSetWriter.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/EnvelopedRecordSetWriter.java
deleted file mode 100644
index 7f64a6d..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/EnvelopedRecordSetWriter.java
+++ /dev/null
@@ -1,197 +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 com.twitter.distributedlog;
-
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-import static com.twitter.distributedlog.LogRecordSet.COMPRESSION_CODEC_LZ4;
-import static com.twitter.distributedlog.LogRecordSet.COMPRESSION_CODEC_NONE;
-import static com.twitter.distributedlog.LogRecordSet.HEADER_LEN;
-import static com.twitter.distributedlog.LogRecordSet.METADATA_COMPRESSION_MASK;
-import static com.twitter.distributedlog.LogRecordSet.METADATA_VERSION_MASK;
-import static com.twitter.distributedlog.LogRecordSet.NULL_OP_STATS_LOGGER;
-import static com.twitter.distributedlog.LogRecordSet.VERSION;
-
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.io.Buffer;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.distributedlog.io.CompressionUtils;
-import com.twitter.util.Promise;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.Channels;
-import java.nio.channels.WritableByteChannel;
-import java.util.LinkedList;
-import java.util.List;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * {@link Buffer} based log record set writer.
- */
-class EnvelopedRecordSetWriter implements LogRecordSet.Writer {
-
-    private static final Logger logger = LoggerFactory.getLogger(EnvelopedRecordSetWriter.class);
-
-    private final Buffer buffer;
-    private final DataOutputStream writer;
-    private final WritableByteChannel writeChannel;
-    private final List<Promise<DLSN>> promiseList;
-    private final CompressionCodec.Type codec;
-    private final int codecCode;
-    private int count = 0;
-    private ByteBuffer recordSetBuffer = null;
-
-    EnvelopedRecordSetWriter(int initialBufferSize,
-                             CompressionCodec.Type codec) {
-        this.buffer = new Buffer(Math.max(initialBufferSize, HEADER_LEN));
-        this.promiseList = new LinkedList<Promise<DLSN>>();
-        this.codec = codec;
-        switch (codec) {
-            case LZ4:
-                this.codecCode = COMPRESSION_CODEC_LZ4;
-                break;
-            default:
-                this.codecCode = COMPRESSION_CODEC_NONE;
-                break;
-        }
-        this.writer = new DataOutputStream(buffer);
-        try {
-            this.writer.writeInt((VERSION & METADATA_VERSION_MASK)
-                    | (codecCode & METADATA_COMPRESSION_MASK));
-            this.writer.writeInt(0); // count
-            this.writer.writeInt(0); // original len
-            this.writer.writeInt(0); // actual len
-        } catch (IOException e) {
-            logger.warn("Failed to serialize the header to an enveloped record set", e);
-        }
-        this.writeChannel = Channels.newChannel(writer);
-    }
-
-    synchronized List<Promise<DLSN>> getPromiseList() {
-        return promiseList;
-    }
-
-    @Override
-    public synchronized void writeRecord(ByteBuffer record,
-                                         Promise<DLSN> transmitPromise)
-            throws LogRecordTooLongException, WriteException {
-        int logRecordSize = record.remaining();
-        if (logRecordSize > MAX_LOGRECORD_SIZE) {
-            throw new LogRecordTooLongException(
-                    "Log Record of size " + logRecordSize + " written when only "
-                            + MAX_LOGRECORD_SIZE + " is allowed");
-        }
-        try {
-            writer.writeInt(record.remaining());
-            writeChannel.write(record);
-            ++count;
-            promiseList.add(transmitPromise);
-        } catch (IOException e) {
-            logger.error("Failed to append record to record set", e);
-            throw new WriteException("", "Failed to append record to record set");
-        }
-    }
-
-    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));
-            nextSlotId++;
-        }
-        promiseList.clear();
-    }
-
-    private synchronized void cancelPromises(Throwable reason) {
-        for (Promise<DLSN> promise : promiseList) {
-            promise.setException(reason);
-        }
-        promiseList.clear();
-    }
-
-    @Override
-    public int getNumBytes() {
-        return buffer.size();
-    }
-
-    @Override
-    public synchronized int getNumRecords() {
-        return count;
-    }
-
-    @Override
-    public synchronized ByteBuffer getBuffer() {
-        if (null == recordSetBuffer) {
-            recordSetBuffer = createBuffer();
-        }
-        return recordSetBuffer.duplicate();
-    }
-
-    ByteBuffer createBuffer() {
-        byte[] data = buffer.getData();
-        int dataOffset = HEADER_LEN;
-        int dataLen = buffer.size() - HEADER_LEN;
-
-        if (COMPRESSION_CODEC_LZ4 != codecCode) {
-            ByteBuffer recordSetBuffer = ByteBuffer.wrap(data, 0, buffer.size());
-            // update count
-            recordSetBuffer.putInt(4, count);
-            // update data len
-            recordSetBuffer.putInt(8, dataLen);
-            recordSetBuffer.putInt(12, dataLen);
-            return recordSetBuffer;
-        }
-
-        // compression
-
-        CompressionCodec compressor =
-                    CompressionUtils.getCompressionCodec(codec);
-        byte[] compressed =
-                compressor.compress(data, dataOffset, dataLen, NULL_OP_STATS_LOGGER);
-
-        ByteBuffer recordSetBuffer;
-        if (compressed.length > dataLen) {
-            byte[] newData = new byte[HEADER_LEN + compressed.length];
-            System.arraycopy(data, 0, newData, 0, HEADER_LEN + dataLen);
-            recordSetBuffer = ByteBuffer.wrap(newData);
-        } else {
-            recordSetBuffer = ByteBuffer.wrap(data);
-        }
-        // version
-        recordSetBuffer.position(4);
-        // update count
-        recordSetBuffer.putInt(count);
-        // update data len
-        recordSetBuffer.putInt(dataLen);
-        recordSetBuffer.putInt(compressed.length);
-        recordSetBuffer.put(compressed);
-        recordSetBuffer.flip();
-        return recordSetBuffer;
-    }
-
-    @Override
-    public void completeTransmit(long lssn, long entryId, long startSlotId) {
-        satisfyPromises(lssn, entryId, startSlotId);
-    }
-
-    @Override
-    public void abortTransmit(Throwable reason) {
-        cancelPromises(reason);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/LogRecord.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/LogRecord.java
deleted file mode 100644
index 74df400..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/LogRecord.java
+++ /dev/null
@@ -1,606 +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 com.twitter.distributedlog;
-
-import com.google.common.annotations.VisibleForTesting;
-import java.io.ByteArrayInputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.EOFException;
-import java.io.IOException;
-import java.io.InputStream;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Log record is the basic element in a log.
- *
- * <p>A log is a sequence of log records. Each log record is a sequence of bytes.
- * Log records are written sequentially into a stream, and will be assigned with
- * an unique system generated sequence number {@link DLSN} (distributedlog sequence
- * number). Besides {@link DLSN}, application can assign its own sequence number
- * while constructing log records. The application defined sequence number is called
- * <code>TransactionID</code> (<i>txid</i>). Either {@link DLSN} or <code>TransactionId</code>
- * could be used to position readers to start from specific log records.
- *
- * <h3>User Record</h3>
- *
- * <p>User records are the records written by applications and read by applications. They
- * are constructed via {@link #LogRecord(long, byte[])} by applications and appended to
- * logs by writers. And they would be deserialized from bytes by the readers and return
- * to applications.
- *
- * <h3>Control Record</h3>
- *
- * <p>Control records are special records that written by distributedlog. They are invisible
- * to applications. They could be treated as <i>commit requests</i> as what people could find
- * in distributed consensus algorithms, since they are usually written by distributedlog to
- * commit application written records. <i>Commit</i> means making application written records
- * visible to readers to achieve consistent views among them.
- *
- * <p>They are named as 'Control Records' for controlling visibility of application written records.
- *
- * <p>The transaction id of 'Control Records' are assigned by distributedlog by inheriting from last
- * written user records. So we could indicate what user records that a control record is committing
- * by looking at its transaction id.
- *
- * <h4>EndOfStream Record</h4>
- *
- * <p><code>EoS</code>(EndOfStream) is a special control record that would be written by a writer
- * to seal a log. After a <i>EoS</i> record is written to a log, no writers could append any record
- * after that and readers will get {@link com.twitter.distributedlog.exceptions.EndOfStreamException}
- * when they reach EoS.
- *
- * <p>TransactionID of EoS is <code>Long.MAX_VALUE</code>.
- *
- * <h3>Serialization & Deserialization</h3>
- *
- * <p>Data type in brackets. Interpretation should be on the basis of data types and not individual
- * bytes to honor Endianness.
- *
- * <pre>
- * LogRecord structure:
- * -------------------
- * Bytes 0 - 7                      : Metadata (Long)
- * Bytes 8 - 15                     : TxId (Long)
- * Bytes 16 - 19                    : Payload length (Integer)
- * Bytes 20 - 20+payload.length-1   : Payload (Byte[])
- *
- * Metadata: 8 Bytes (Long)
- * --------
- *
- * 0x 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
- *            |_____________| |_____|
- *                   |           |
- *               position      flags
- *
- * Flags: 2 Bytes (least significant)
- * -----
- * Bit  0      : If set, control record, else record with payload.
- * Bit  1      : If set, end of stream.
- * Bits 2 - 15 : Unused
- * </pre>
- *
- * <h3>Sequence Numbers</h3>
- *
- * <p>A record is associated with three types of sequence numbers. They are generated
- * and used for different purposes. Check {@link LogRecordWithDLSN} for more details.
- *
- * @see LogRecordWithDLSN
- */
-public class LogRecord {
-
-    private static final Logger LOG = LoggerFactory.getLogger(LogRecord.class);
-
-    // Allow 4K overhead for metadata within the max transmission size
-    public static final int MAX_LOGRECORD_SIZE = 1024 * 1024 - 8 * 1024; //1MB - 8KB
-    // Allow 4K overhead for transmission overhead
-    public static final int MAX_LOGRECORDSET_SIZE = 1024 * 1024 - 4 * 1024; //1MB - 4KB
-
-    private static final int INPUTSTREAM_MARK_LIMIT = 16;
-
-    static final long LOGRECORD_METADATA_FLAGS_MASK = 0xffffL;
-    static final long LOGRECORD_METADATA_FLAGS_UMASK = 0xffffffffffff0000L;
-    static final long LOGRECORD_METADATA_POSITION_MASK = 0x0000ffffffff0000L;
-    static final long LOGRECORD_METADATA_POSITION_UMASK = 0xffff00000000ffffL;
-    static final int LOGRECORD_METADATA_POSITION_SHIFT = 16;
-    static final long LOGRECORD_METADATA_UNUSED_MASK = 0xffff000000000000L;
-
-    // TODO: Replace with EnumSet
-    static final long LOGRECORD_FLAGS_CONTROL_MESSAGE = 0x1;
-    static final long LOGRECORD_FLAGS_END_OF_STREAM = 0x2;
-    static final long LOGRECORD_FLAGS_RECORD_SET = 0x4;
-
-    private long metadata;
-    private long txid;
-    private byte[] payload;
-
-    /**
-     * Construct an uninitialized log record.
-     *
-     * <p>NOTE: only deserializer should call this constructor.
-     */
-    protected LogRecord() {
-        this.txid = 0;
-        this.metadata = 0;
-    }
-
-    /**
-     * Construct a log record with <i>TransactionId</i> and payload.
-     *
-     * <p>Usually writer would construct the log record for writing.
-     *
-     * @param txid
-     *          application defined transaction id.
-     * @param payload
-     *          record data
-     */
-    public LogRecord(long txid, byte[] payload) {
-        this.txid = txid;
-        this.payload = payload;
-        this.metadata = 0;
-    }
-
-    //
-    // Accessors
-    //
-
-    /**
-     * Return application defined transaction id.
-     *
-     * @return transacton id.
-     */
-    public long getTransactionId() {
-        return txid;
-    }
-
-    /**
-     * Set application defined transaction id.
-     *
-     * @param txid application defined transaction id.
-     */
-    protected void setTransactionId(long txid) {
-        this.txid = txid;
-    }
-
-    /**
-     * Return the payload of this log record.
-     *
-     * @return payload of this log record.
-     */
-    public byte[] getPayload() {
-        return payload;
-    }
-
-    /**
-     * Set payload for this log record.
-     *
-     * @param payload payload of this log record
-     */
-    void setPayload(byte[] payload) {
-        this.payload = payload;
-    }
-
-    /**
-     * Return the payload as an {@link InputStream}.
-     *
-     * @return payload as input stream
-     */
-    public InputStream getPayLoadInputStream() {
-        return new ByteArrayInputStream(payload);
-    }
-
-    //
-    // Metadata & Flags
-    //
-
-    protected void setMetadata(long metadata) {
-        this.metadata = metadata;
-    }
-
-    protected long getMetadata() {
-        return this.metadata;
-    }
-
-    /**
-     * Set the position in the log segment.
-     *
-     * @see #getPositionWithinLogSegment()
-     * @param positionWithinLogSegment position in the log segment.
-     */
-    void setPositionWithinLogSegment(int positionWithinLogSegment) {
-        assert(positionWithinLogSegment >= 0);
-        metadata = (metadata & LOGRECORD_METADATA_POSITION_UMASK)
-            | (((long) positionWithinLogSegment) << LOGRECORD_METADATA_POSITION_SHIFT);
-    }
-
-    /**
-     * The position in the log segment means how many records (inclusive) added to the log segment so far.
-     *
-     * @return position of the record in the log segment.
-     */
-    public int getPositionWithinLogSegment() {
-        long ret = (metadata & LOGRECORD_METADATA_POSITION_MASK) >> LOGRECORD_METADATA_POSITION_SHIFT;
-        if (ret < 0 || ret > Integer.MAX_VALUE) {
-            throw new IllegalArgumentException
-                (ret + " position should never exceed max integer value");
-        }
-        return (int) ret;
-    }
-
-    /**
-     * Get the last position of this record in the log segment.
-     *
-     * <p>If the record isn't record set, it would be same as {@link #getPositionWithinLogSegment()},
-     * otherwise, it would be {@link #getPositionWithinLogSegment()} + numRecords - 1. If the record set
-     * version is unknown, it would be same as {@link #getPositionWithinLogSegment()}.
-     *
-     * @return last position of this record in the log segment.
-     */
-    int getLastPositionWithinLogSegment() {
-        if (isRecordSet()) {
-            try {
-                return getPositionWithinLogSegment() + LogRecordSet.numRecords(this) - 1;
-            } catch (IOException e) {
-                // if it is unrecognized record set, we will return the position of this record set.
-                return getPositionWithinLogSegment();
-            }
-        } else {
-            return getPositionWithinLogSegment();
-        }
-    }
-
-    /**
-     * Set the record to represent a set of records.
-     *
-     * <p>The bytes in this record is the serialized format of {@link LogRecordSet}.
-     */
-    public void setRecordSet() {
-        metadata = metadata | LOGRECORD_FLAGS_RECORD_SET;
-    }
-
-    /**
-     * Check if the record represents a set of records.
-     *
-     * @return true if the record represents a set of records, otherwise false.
-     * @see #setRecordSet()
-     */
-    public boolean isRecordSet() {
-        return isRecordSet(metadata);
-    }
-
-    public static boolean isRecordSet(long metadata) {
-        return ((metadata & LOGRECORD_FLAGS_RECORD_SET) != 0);
-    }
-
-    @VisibleForTesting
-    public void setControl() {
-        metadata = metadata | LOGRECORD_FLAGS_CONTROL_MESSAGE;
-    }
-
-    /**
-     * Check if the record is a control record.
-     *
-     * @return true if the record is a control record, otherwise false.
-     */
-    public boolean isControl() {
-        return isControl(metadata);
-    }
-
-    /**
-     * Check flags to see if it indicates a control record.
-     *
-     * @param flags record flags
-     * @return true if the record is a control record, otherwise false.
-     */
-    public static boolean isControl(long flags) {
-        return ((flags & LOGRECORD_FLAGS_CONTROL_MESSAGE) != 0);
-    }
-
-    /**
-     * Set the record as <code>EoS</code> mark.
-     *
-     * @see #isEndOfStream()
-     */
-    void setEndOfStream() {
-        metadata = metadata | LOGRECORD_FLAGS_END_OF_STREAM;
-    }
-
-    /**
-     * Check if the record is a <code>EoS</code> mark.
-     *
-     * <p><code>EoS</code> mark is a special record that writer would
-     * add to seal a log. after <code>Eos</code> mark is written,
-     * writers can't write any more records and readers will get
-     * {@link com.twitter.distributedlog.exceptions.EndOfStreamException}
-     * when they reach <code>EoS</code>.
-     *
-     * @return true
-     */
-    boolean isEndOfStream() {
-        return ((metadata & LOGRECORD_FLAGS_END_OF_STREAM) != 0);
-    }
-
-    //
-    // Serialization & Deserialization
-    //
-
-    protected void readPayload(DataInputStream in) throws IOException {
-        int length = in.readInt();
-        if (length < 0) {
-            throw new EOFException("Log Record is corrupt: Negative length " + length);
-        }
-        payload = new byte[length];
-        in.readFully(payload);
-    }
-
-    private void writePayload(DataOutputStream out) throws IOException {
-        out.writeInt(payload.length);
-        out.write(payload);
-    }
-
-    private void writeToStream(DataOutputStream out) throws IOException {
-        out.writeLong(metadata);
-        out.writeLong(txid);
-        writePayload(out);
-    }
-
-    /**
-     * The size of the serialized log record.
-     *
-     * <p>This is used to estimate how much will be be appended to the in-memory buffer.
-     *
-     * @return serialized size
-     */
-    int getPersistentSize() {
-        // Flags + TxId + Payload-length + payload
-        return 2 * (Long.SIZE / 8) + Integer.SIZE / 8 + payload.length;
-    }
-
-    /**
-     * Writer class to write log records into an output {@code stream}.
-     */
-    public static class Writer {
-        private final DataOutputStream buf;
-
-        public Writer(DataOutputStream out) {
-            this.buf = out;
-        }
-
-        /**
-         * Write an operation to the output stream.
-         *
-         * @param record The operation to write
-         * @throws IOException if an error occurs during writing.
-         */
-        public void writeOp(LogRecord record) throws IOException {
-            record.writeToStream(buf);
-        }
-
-        public int getPendingBytes() {
-            return buf.size();
-        }
-    }
-
-    /**
-     * Reader class to read log records from an input {@code stream}.
-      */
-    public static class Reader {
-        private final RecordStream recordStream;
-        private final DataInputStream in;
-        private final long startSequenceId;
-        private final boolean deserializeRecordSet;
-        private static final int SKIP_BUFFER_SIZE = 512;
-        private LogRecordSet.Reader recordSetReader = null;
-        private LogRecordWithDLSN lastRecordSkipTo = null;
-
-        /**
-         * Construct the reader.
-         *
-         * @param recordStream the record stream for generating {@code DLSN}s.
-         * @param in The stream to read from.
-         * @param startSequenceId the start sequence id.
-         */
-        public Reader(RecordStream recordStream,
-                      DataInputStream in,
-                      long startSequenceId) {
-            this(recordStream, in, startSequenceId, true);
-        }
-
-        public Reader(RecordStream recordStream,
-                      DataInputStream in,
-                      long startSequenceId,
-                      boolean deserializeRecordSet) {
-            this.recordStream = recordStream;
-            this.in = in;
-            this.startSequenceId = startSequenceId;
-            this.deserializeRecordSet = deserializeRecordSet;
-        }
-
-        /**
-         * Read an log record from the input stream.
-         *
-         * <p/> Note that the objects returned from this method may be re-used by future
-         * calls to the same method.
-         *
-         * @return the operation read from the stream, or null at the end of the file
-         * @throws IOException on error.
-         */
-        public LogRecordWithDLSN readOp() throws IOException {
-            LogRecordWithDLSN nextRecordInStream;
-            while (true) {
-                if (lastRecordSkipTo != null) {
-                    nextRecordInStream = lastRecordSkipTo;
-                    recordStream.advance(1);
-                    lastRecordSkipTo = null;
-                    return nextRecordInStream;
-                }
-                if (recordSetReader != null) {
-                    nextRecordInStream = recordSetReader.nextRecord();
-                    if (null != nextRecordInStream) {
-                        recordStream.advance(1);
-                        return nextRecordInStream;
-                    } else {
-                        recordSetReader = null;
-                    }
-                }
-
-                try {
-                    long metadata = in.readLong();
-                    // Reading the first 8 bytes positions the record stream on the correct log record
-                    // By this time all components of the DLSN are valid so this is where we shoud
-                    // retrieve the currentDLSN and advance to the next
-                    // Given that there are 20 bytes following the read position of the previous call
-                    // to readLong, we should not have moved ahead in the stream.
-                    nextRecordInStream = new LogRecordWithDLSN(recordStream.getCurrentPosition(), startSequenceId);
-                    nextRecordInStream.setMetadata(metadata);
-                    nextRecordInStream.setTransactionId(in.readLong());
-                    nextRecordInStream.readPayload(in);
-                    if (LOG.isTraceEnabled()) {
-                        if (nextRecordInStream.isControl()) {
-                            LOG.trace("Reading {} Control DLSN {}",
-                                recordStream.getName(), nextRecordInStream.getDlsn());
-                        } else {
-                            LOG.trace("Reading {} Valid DLSN {}",
-                                recordStream.getName(), nextRecordInStream.getDlsn());
-                        }
-                    }
-
-                    int numRecords = 1;
-                    if (!deserializeRecordSet && nextRecordInStream.isRecordSet()) {
-                        numRecords = LogRecordSet.numRecords(nextRecordInStream);
-                    }
-
-                    if (deserializeRecordSet && nextRecordInStream.isRecordSet()) {
-                        recordSetReader = LogRecordSet.of(nextRecordInStream);
-                    } else {
-                        recordStream.advance(numRecords);
-                        return nextRecordInStream;
-                    }
-                } catch (EOFException eof) {
-                    // Expected
-                    break;
-                }
-            }
-            return null;
-        }
-
-        public boolean skipTo(long txId, boolean skipControl) throws IOException {
-            return skipTo(txId, null, skipControl);
-        }
-
-        public boolean skipTo(DLSN dlsn) throws IOException {
-            return skipTo(null, dlsn, false);
-        }
-
-        private boolean skipTo(Long txId, DLSN dlsn, boolean skipControl) throws IOException {
-            LOG.debug("SkipTo");
-            byte[] skipBuffer = null;
-            boolean found = false;
-            while (true) {
-                try {
-                    long flags;
-                    long currTxId;
-
-                    // if there is not record set, read next record
-                    if (null == recordSetReader) {
-                        in.mark(INPUTSTREAM_MARK_LIMIT);
-                        flags = in.readLong();
-                        currTxId = in.readLong();
-                    } else {
-                        // check record set until reach end of record set
-                        lastRecordSkipTo = recordSetReader.nextRecord();
-                        if (null == lastRecordSkipTo) {
-                            // reach end of record set
-                            recordSetReader = null;
-                            continue;
-                        }
-                        flags = lastRecordSkipTo.getMetadata();
-                        currTxId = lastRecordSkipTo.getTransactionId();
-                    }
-
-                    if ((null != dlsn) && (recordStream.getCurrentPosition().compareTo(dlsn) >= 0)) {
-                        if (LOG.isTraceEnabled()) {
-                            LOG.trace("Found position {} beyond {}", recordStream.getCurrentPosition(), dlsn);
-                        }
-                        if (null == lastRecordSkipTo) {
-                            in.reset();
-                        }
-                        found = true;
-                        break;
-                    }
-                    if ((null != txId) && (currTxId >= txId)) {
-                        if (!skipControl || !isControl(flags)) {
-                            if (LOG.isTraceEnabled()) {
-                                LOG.trace("Found position {} beyond {}", currTxId, txId);
-                            }
-                            if (null == lastRecordSkipTo) {
-                                in.reset();
-                            }
-                            found = true;
-                            break;
-                        }
-                    }
-
-                    if (null != lastRecordSkipTo) {
-                        recordStream.advance(1);
-                        continue;
-                    }
-
-                    // get the num of records to skip
-                    if (isRecordSet(flags)) {
-                        // read record set
-                        LogRecordWithDLSN record =
-                            new LogRecordWithDLSN(recordStream.getCurrentPosition(), startSequenceId);
-                        record.setMetadata(flags);
-                        record.setTransactionId(currTxId);
-                        record.readPayload(in);
-                        recordSetReader = LogRecordSet.of(record);
-                    } else {
-                        int length = in.readInt();
-                        if (length < 0) {
-                            // We should never really see this as we only write complete entries to
-                            // BK and BK client has logic to detect torn writes (through checksum)
-                            LOG.info("Encountered Record with negative length at TxId: {}", currTxId);
-                            break;
-                        }
-                        // skip single record
-                        if (null == skipBuffer) {
-                            skipBuffer = new byte[SKIP_BUFFER_SIZE];
-                        }
-                        int read = 0;
-                        while (read < length) {
-                            int bytesToRead = Math.min(length - read, SKIP_BUFFER_SIZE);
-                            in.readFully(skipBuffer, 0, bytesToRead);
-                            read += bytesToRead;
-                        }
-                        if (LOG.isTraceEnabled()) {
-                            LOG.trace("Skipped Record with TxId {} DLSN {}",
-                                currTxId, recordStream.getCurrentPosition());
-                        }
-                        recordStream.advance(1);
-                    }
-                } catch (EOFException eof) {
-                    LOG.debug("Skip encountered end of file Exception", eof);
-                    break;
-                }
-            }
-            return found;
-        }
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/LogRecordSet.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/LogRecordSet.java
deleted file mode 100644
index 52970c7..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/LogRecordSet.java
+++ /dev/null
@@ -1,155 +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 com.twitter.distributedlog;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.distributedlog.exceptions.WriteException;
-import com.twitter.distributedlog.io.CompressionCodec;
-import com.twitter.util.Promise;
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-
-/**
- * A set of {@link LogRecord}s.
- *
- * <pre>
- * Structure:
- * Bytes 0  -  4                : Metadata (version + flags)
- * Bytes 4  - 7                 : Number of Records
- * Bytes 8  - 11                : Original Payload Length
- * Bytes 12 - 15                : Actual Payload Length
- * Bytes 16 - 16+length-1       : Payload
- * ------------------------------------------------------
- *
- * Metadata: Version and Flags // 32 Bits
- * --------------------------------------
- * 0 ... 0 0 0 0 0
- *             |_|
- *              |
- *    Compression Codec: // 2 Bits (Least significant)
- *    -----------------
- *    00        : No Compression
- *    01        : LZ4 Compression
- *    10        : Unused
- *    11        : Unused
- *
- * </pre>
- */
-public class LogRecordSet {
-
-    static final OpStatsLogger NULL_OP_STATS_LOGGER =
-            NullStatsLogger.INSTANCE.getOpStatsLogger("");
-
-    public static final int HEADER_LEN =
-            4 /* Metadata */
-          + 4 /* Count */
-          + 8 /* Lengths */
-            ;
-
-    // Version
-    static final int VERSION = 0x1000;
-
-    // Metadata
-    static final int METADATA_VERSION_MASK = 0xf000;
-    static final int METADATA_COMPRESSION_MASK = 0x3;
-
-    // Compression Codec
-    static final int COMPRESSION_CODEC_NONE = 0x0;
-    static final int COMPRESSION_CODEC_LZ4 = 0X1;
-
-    public static int numRecords(LogRecord record) throws IOException {
-        checkArgument(record.isRecordSet(),
-                "record is not a recordset");
-        byte[] data = record.getPayload();
-        return numRecords(data);
-    }
-
-    public static int numRecords(byte[] data) throws IOException {
-        ByteBuffer buffer = ByteBuffer.wrap(data);
-        int metadata = buffer.getInt();
-        int version = (metadata & METADATA_VERSION_MASK);
-        if (version != VERSION) {
-            throw new IOException(String.format("Version mismatch while reading. Received: %d,"
-                + " Required: %d", version, VERSION));
-        }
-        return buffer.getInt();
-    }
-
-    public static Writer newWriter(int initialBufferSize,
-                                   CompressionCodec.Type codec) {
-        return new EnvelopedRecordSetWriter(initialBufferSize, codec);
-    }
-
-    public static Reader of(LogRecordWithDLSN record) throws IOException {
-        checkArgument(record.isRecordSet(),
-                "record is not a recordset");
-        byte[] data = record.getPayload();
-        DLSN dlsn = record.getDlsn();
-        int startPosition = record.getPositionWithinLogSegment();
-        long startSequenceId = record.getStartSequenceIdOfCurrentSegment();
-
-        return new EnvelopedRecordSetReader(
-                dlsn.getLogSegmentSequenceNo(),
-                dlsn.getEntryId(),
-                record.getTransactionId(),
-                dlsn.getSlotId(),
-                startPosition,
-                startSequenceId,
-                new ByteArrayInputStream(data));
-    }
-
-    /**
-     * Writer to append {@link LogRecord}s to {@link LogRecordSet}.
-     */
-    public interface Writer extends LogRecordSetBuffer {
-
-        /**
-         * Write a {@link LogRecord} to this record set.
-         *
-         * @param record
-         *          record to write
-         * @param transmitPromise
-         *          callback for transmit result. the promise is only
-         *          satisfied when this record set is transmitted.
-         * @throws LogRecordTooLongException if the record is too long
-         * @throws WriteException when encountered exception writing the record
-         */
-        void writeRecord(ByteBuffer record, Promise<DLSN> transmitPromise)
-                throws LogRecordTooLongException, WriteException;
-    }
-
-    /**
-     * Reader to read {@link LogRecord}s from this record set.
-     */
-    public interface Reader {
-
-        /**
-         * Read next log record from this record set.
-         *
-         * @return next log record from this record set.
-         */
-        LogRecordWithDLSN nextRecord() throws IOException;
-
-    }
-
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/LogRecordSetBuffer.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/LogRecordSetBuffer.java
deleted file mode 100644
index 8591e4e..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/LogRecordSetBuffer.java
+++ /dev/null
@@ -1,65 +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 com.twitter.distributedlog;
-
-import java.nio.ByteBuffer;
-
-/**
- * Write representation of a {@link LogRecordSet}.
- * It is a buffer of log record set, used for transmission.
- */
-public interface LogRecordSetBuffer {
-
-    /**
-     * Return number of records in current record set.
-     *
-     * @return number of records in current record set.
-     */
-    int getNumRecords();
-
-    /**
-     * Return number of bytes in current record set.
-     *
-     * @return number of bytes in current record set.
-     */
-    int getNumBytes();
-
-    /**
-     * Get the buffer to transmit.
-     *
-     * @return the buffer to transmit.
-     */
-    ByteBuffer getBuffer();
-
-    /**
-     * Complete transmit.
-     *
-     * @param lssn log segment sequence number
-     * @param entryId entry id
-     * @param startSlotId start slot id
-     */
-    void completeTransmit(long lssn, long entryId, long startSlotId);
-
-    /**
-     * Abort transmit.
-     *
-     * @param reason reason to abort.
-     */
-    void abortTransmit(Throwable reason);
-
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/LogRecordWithDLSN.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/LogRecordWithDLSN.java
deleted file mode 100644
index 14c8e76..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/LogRecordWithDLSN.java
+++ /dev/null
@@ -1,107 +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 com.twitter.distributedlog;
-
-import com.google.common.annotations.VisibleForTesting;
-
-/**
- * Log record with {@link DLSN} and <code>SequenceId</code>.
- *
- * <h3>Sequence Numbers</h3>
- *
- * <p>A log record will be assigned with an unique system generated sequence number {@link DLSN} when it is
- * written to a log. At the mean time, a 64-bits long number is assigned to the record indicates its position
- * within a log, which is called <code>SequenceId</code>. Besides {@link DLSN} and <code>SequenceID</code>,
- * application can assign its own sequence number (called <code>TransactionID</code>) to the log record while
- * writing it.
- *
- * <h4>Transaction ID</h4>
- *
- * <p>Transaction ID is a positive 64-bits long number that is assigned by the application. It is a very helpful
- * field when application wants to organize the records and position the readers using their own sequencing method.
- * A typical use case of <code>TransactionID</code> is DistributedLog Write Proxy. It assigns the non-decreasing
- * timestamps to log records, which the timestamps could be used as `physical time` to implement `TTL` in a strong
- * consistent database to achieve consistent `TTL` over replicas.
- *
- * <h4>DLSN</h4>
- *
- * <p>DistributedLog Sequence Number (<i>DLSN</i>) is the sequence number generated during written time.
- * It is comparable and could be used to figure out the order between records. The DLSN is comprised with 3 components.
- * They are <i>Log Segment Sequence Number</i>, <i>Entry Id</i> and <i>Slot Id</i>. (See {@link DLSN} for more details).
- * The DLSN is usually used for comparison, positioning and truncation.
- *
- * <h4>Sequence ID</h4>
- *
- * <p>Sequence ID is introduced to address the drawback of <code>DLSN</code>, in favor of answering questions like
- * `how many records written between two DLSNs`. It is a 64-bits monotonic increasing number (starting from zero).
- * Sequence ids are only accessible by readers. That means writers don't know the sequence ids of records after they
- * wrote them.
- */
-public class LogRecordWithDLSN extends LogRecord {
-    private final DLSN dlsn;
-    private final long startSequenceIdOfCurrentSegment;
-
-    /**
-     * This empty constructor can only be called from Reader#readOp.
-     */
-    LogRecordWithDLSN(DLSN dlsn, long startSequenceIdOfCurrentSegment) {
-        super();
-        this.dlsn = dlsn;
-        this.startSequenceIdOfCurrentSegment = startSequenceIdOfCurrentSegment;
-    }
-
-    @VisibleForTesting
-    public LogRecordWithDLSN(DLSN dlsn, long txid, byte[] data, long startSequenceIdOfCurrentSegment) {
-        super(txid, data);
-        this.dlsn = dlsn;
-        this.startSequenceIdOfCurrentSegment = startSequenceIdOfCurrentSegment;
-    }
-
-    long getStartSequenceIdOfCurrentSegment() {
-        return startSequenceIdOfCurrentSegment;
-    }
-
-    /**
-     * Get the sequence id of the record in the log.
-     *
-     * @return sequence id of the record in the log.
-     */
-    public long getSequenceId() {
-        return startSequenceIdOfCurrentSegment + getPositionWithinLogSegment() - 1;
-    }
-
-    /**
-     * Get the DLSN of the record in the log.
-     *
-     * @return DLSN of the record in the log.
-     */
-    public DLSN getDlsn() {
-        return dlsn;
-    }
-
-    @Override
-    public String toString() {
-        return "LogRecordWithDLSN{"
-            + "dlsn=" + dlsn
-            + ", txid=" + getTransactionId()
-            + ", position=" + getPositionWithinLogSegment()
-            + ", isControl=" + isControl()
-            + ", isEndOfStream=" + isEndOfStream()
-            + '}';
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/RecordStream.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/RecordStream.java
deleted file mode 100644
index d920ea1..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/RecordStream.java
+++ /dev/null
@@ -1,42 +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 com.twitter.distributedlog;
-
-/**
- * A iterator to iterate records in the stream.
- */
-public interface RecordStream {
-    /**
-     * advance <i>numRecords</i> records.
-     */
-    void advance(int numRecords);
-
-    /**
-     * Get position of current record in the stream.
-     *
-     * @return position of current record
-     */
-    DLSN getCurrentPosition();
-
-    /**
-     * Get the name of the stream.
-     *
-     * @return the name of the stream
-     */
-    String getName();
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/annotations/DistributedLogAnnotations.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/annotations/DistributedLogAnnotations.java
deleted file mode 100644
index 2859008..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/annotations/DistributedLogAnnotations.java
+++ /dev/null
@@ -1,35 +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 com.twitter.distributedlog.annotations;
-
-/**
- * Common annotation types.
- */
-public class DistributedLogAnnotations {
-    /**
-     * Annotation to identify flaky tests in DistributedLog.
-     * As and when we find that a test is flaky, we'll add this annotation to it for reference.
-     */
-    public @interface FlakyTest {}
-
-    /**
-     * Annotation to specify the occurrence of a compression operation. These are CPU intensive
-     * and should be avoided in low-latency paths.
-     */
-    public @interface Compression {}
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/annotations/package-info.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/annotations/package-info.java
deleted file mode 100644
index 24bb36b..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/annotations/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Defines annotations used across distributedlog project.
- */
-package com.twitter.distributedlog.annotations;
\ No newline at end of file
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/AlreadyClosedException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/AlreadyClosedException.java
deleted file mode 100644
index abbce5f..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/AlreadyClosedException.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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Thrown when any distributedlog resources have already been closed.
- *
- * <p>For example, it might be thrown when the connections have been torn down.
- */
-public class AlreadyClosedException extends DLException {
-
-    private static final long serialVersionUID = -4721864322739563725L;
-
-    public AlreadyClosedException(String message) {
-        super(StatusCode.ALREADY_CLOSED, message);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/AlreadyTruncatedTransactionException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/AlreadyTruncatedTransactionException.java
deleted file mode 100644
index 788b4d8..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/AlreadyTruncatedTransactionException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Thrown when the transaction Id specified in the API is in the range that has already been truncated.
- */
-public class AlreadyTruncatedTransactionException extends DLException {
-
-    private static final long serialVersionUID = 4287238797065959977L;
-
-    public AlreadyTruncatedTransactionException(String message) {
-        super(StatusCode.TRUNCATED_TRANSACTION, message);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/BKTransmitException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/BKTransmitException.java
deleted file mode 100644
index 8f29628..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/BKTransmitException.java
+++ /dev/null
@@ -1,42 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Thrown when the send to bookkeeper fails.
- *
- * <p>This is thrown by the next attempt to write, send or flush
- */
-public class BKTransmitException extends DLException {
-
-    private static final long serialVersionUID = -5796100450432076091L;
-
-    final int bkRc;
-
-    public BKTransmitException(String message, int bkRc) {
-        super(StatusCode.BK_TRANSMIT_ERROR, message + " : " + bkRc);
-        this.bkRc = bkRc;
-    }
-
-    public int getBKResultCode() {
-        return this.bkRc;
-    }
-
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/ChecksumFailedException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/ChecksumFailedException.java
deleted file mode 100644
index 4b67299..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/ChecksumFailedException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception thrown when checksum failures occurred.
- */
-public class ChecksumFailedException extends DLException {
-
-    private static final long serialVersionUID = 288438128880378812L;
-
-    public ChecksumFailedException() {
-        super(StatusCode.CHECKSUM_FAILED, "Checksum failed");
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/DLClientClosedException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/DLClientClosedException.java
deleted file mode 100644
index 2f506ba..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/DLClientClosedException.java
+++ /dev/null
@@ -1,36 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exceptions thrown when a distributedlog client is closed.
- */
-public class DLClientClosedException extends DLException {
-
-    private static final long serialVersionUID = -8876218750540927584L;
-
-    public DLClientClosedException(String msg) {
-        super(StatusCode.CLIENT_CLOSED, msg);
-    }
-
-    public DLClientClosedException(String msg, Throwable t) {
-        super(StatusCode.CLIENT_CLOSED, msg, t);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/DLException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/DLException.java
deleted file mode 100644
index 23af0c5..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/DLException.java
+++ /dev/null
@@ -1,81 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.ResponseHeader;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import java.io.IOException;
-
-/**
- * The basic exception of all distributedlog exceptions.
- */
-public class DLException extends IOException {
-    private static final long serialVersionUID = -4485775468586114393L;
-    protected final StatusCode code;
-
-    protected DLException(StatusCode code) {
-        super();
-        this.code = code;
-    }
-
-    protected DLException(StatusCode code, String msg) {
-        super(msg);
-        this.code = code;
-    }
-
-    protected DLException(StatusCode code, Throwable t) {
-        super(t);
-        this.code = code;
-    }
-
-    protected DLException(StatusCode code, String msg, Throwable t) {
-        super(msg, t);
-        this.code = code;
-    }
-
-    /**
-     * Return the status code representing the exception.
-     *
-     * @return status code representing the exception.
-     */
-    public StatusCode getCode() {
-        return code;
-    }
-
-    public static DLException of(ResponseHeader response) {
-        String errMsg;
-        switch (response.getCode()) {
-            case FOUND:
-                if (response.isSetErrMsg()) {
-                    errMsg = response.getErrMsg();
-                } else {
-                    errMsg = "Request is redirected to " + response.getLocation();
-                }
-                return new OwnershipAcquireFailedException(errMsg, response.getLocation());
-            case SUCCESS:
-                throw new IllegalArgumentException("Can't instantiate an exception for success response.");
-            default:
-                if (response.isSetErrMsg()) {
-                    errMsg = response.getErrMsg();
-                } else {
-                    errMsg = response.getCode().name();
-                }
-                return new DLException(response.getCode(), errMsg);
-        }
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/DLIllegalStateException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/DLIllegalStateException.java
deleted file mode 100644
index c80b9db..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/DLIllegalStateException.java
+++ /dev/null
@@ -1,36 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Signals that a method has been invoked at an illegal or inappropriate time.
- */
-public class DLIllegalStateException extends DLException {
-
-    private static final long serialVersionUID = -6721471104777747420L;
-
-    public DLIllegalStateException(String msg) {
-        super(StatusCode.ILLEGAL_STATE, msg);
-    }
-
-    public DLIllegalStateException(String msg, Throwable t) {
-        super(StatusCode.ILLEGAL_STATE, msg, t);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/DLInterruptedException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/DLInterruptedException.java
deleted file mode 100644
index a1e3d1b..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/DLInterruptedException.java
+++ /dev/null
@@ -1,36 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * An interrupted exception wrapper indicates dl operations are interrupted.
- */
-public class DLInterruptedException extends DLException {
-
-    private static final long serialVersionUID = 3023394251107286496L;
-
-    public DLInterruptedException(String msg) {
-        super(StatusCode.INTERRUPTED, msg);
-    }
-
-    public DLInterruptedException(String msg, Throwable t) {
-        super(StatusCode.INTERRUPTED, msg, t);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/EndOfLogSegmentException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/EndOfLogSegmentException.java
deleted file mode 100644
index ae4aa45..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/EndOfLogSegmentException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception thrown when reach end of the log segment.
- */
-public class EndOfLogSegmentException extends DLException {
-
-    private static final long serialVersionUID = 6060419315910178451L;
-
-    public EndOfLogSegmentException(String logSegmentName) {
-        super(StatusCode.END_OF_LOG_SEGMENT, "end of log segment " + logSegmentName);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/EndOfStreamException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/EndOfStreamException.java
deleted file mode 100644
index a62e629..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/EndOfStreamException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception thrown when a reader reaches end of a sealed log stream.
- */
-public class EndOfStreamException extends DLException {
-
-    private static final long serialVersionUID = -6398949401860680263L;
-
-    public EndOfStreamException(String message) {
-        super(StatusCode.END_OF_STREAM, message);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/FlushException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/FlushException.java
deleted file mode 100644
index f984b6c..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/FlushException.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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception indicates that errors occurred on flushing data.
- */
-public class FlushException extends DLException {
-
-    private final long lastTxIdWritten;
-    private final long lastTxIdAcknowledged;
-
-    private static final long serialVersionUID = -9060360360261130489L;
-
-    public FlushException(String message, long lastTxIdWritten, long lastTxIdAcknowledged) {
-        super(StatusCode.FLUSH_TIMEOUT, message);
-        this.lastTxIdWritten = lastTxIdWritten;
-        this.lastTxIdAcknowledged = lastTxIdAcknowledged;
-    }
-
-    public FlushException(String message, long lastTxIdWritten, long lastTxIdAcknowledged, Throwable cause) {
-        super(StatusCode.FLUSH_TIMEOUT, message, cause);
-        this.lastTxIdWritten = lastTxIdWritten;
-        this.lastTxIdAcknowledged = lastTxIdAcknowledged;
-    }
-
-    public long getLastTxIdWritten() {
-        return lastTxIdWritten;
-    }
-
-    public long getLastTxIdAcknowledged() {
-        return lastTxIdAcknowledged;
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/IdleReaderException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/IdleReaderException.java
deleted file mode 100644
index 1f6923a..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/IdleReaderException.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 com.twitter.distributedlog.exceptions;
-
-import java.io.IOException;
-
-/**
- * An exception is thrown when a log reader becomes idle.
- *
- * <p>A log reader can become idle when there is no records written to the stream.
- */
-public class IdleReaderException extends IOException {
-
-    private static final long serialVersionUID = 4882278671396434834L;
-
-    public IdleReaderException(String message) {
-        super(message);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/InternalServerException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/InternalServerException.java
deleted file mode 100644
index 2aea152..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/InternalServerException.java
+++ /dev/null
@@ -1,40 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception indicates that there is an internal error at distributedlog service side.
- */
-public class InternalServerException extends DLException {
-
-    private static final long serialVersionUID = 288438028880978802L;
-
-    public InternalServerException(String msg) {
-        super(StatusCode.INTERNAL_SERVER_ERROR, msg);
-    }
-
-    public InternalServerException(Throwable t) {
-        super(StatusCode.INTERNAL_SERVER_ERROR, t);
-    }
-
-    public InternalServerException(String msg, Throwable t) {
-        super(StatusCode.INTERNAL_SERVER_ERROR, msg, t);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/InvalidEnvelopedEntryException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/InvalidEnvelopedEntryException.java
deleted file mode 100644
index 09e146c..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/InvalidEnvelopedEntryException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception thrown when encounter invalid enveloped entry.
- */
-public class InvalidEnvelopedEntryException extends DLException {
-
-    private static final long serialVersionUID = -9190621788978573862L;
-
-    public InvalidEnvelopedEntryException(String msg) {
-        super(StatusCode.INVALID_ENVELOPED_ENTRY, msg);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/InvalidStreamNameException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/InvalidStreamNameException.java
deleted file mode 100644
index 1e8f445..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/InvalidStreamNameException.java
+++ /dev/null
@@ -1,36 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception is thrown when encountered invalid log stream.
- */
-public class InvalidStreamNameException extends DLException {
-
-    private static final long serialVersionUID = 6393315766140568100L;
-
-    public InvalidStreamNameException(String streamName) {
-        super(StatusCode.INVALID_STREAM_NAME, "Invalid stream name : '" + streamName + "'");
-    }
-
-    public InvalidStreamNameException(String streamName, String reason) {
-        super(StatusCode.INVALID_STREAM_NAME, "Invalid stream name : '" + streamName + "' : " + reason);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LockCancelledException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LockCancelledException.java
deleted file mode 100644
index 98131ee..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LockCancelledException.java
+++ /dev/null
@@ -1,31 +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 com.twitter.distributedlog.exceptions;
-
-/**
- * A lock is cancelled when waiting for locking a stream.
- */
-public class LockCancelledException extends LockingException {
-
-    private static final long serialVersionUID = -148795017092861106L;
-
-    public LockCancelledException(String lockPath, String message, Throwable cause) {
-        super(lockPath, message, cause);
-    }
-
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LockingException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LockingException.java
deleted file mode 100644
index 326a1cd..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LockingException.java
+++ /dev/null
@@ -1,44 +0,0 @@
-/**
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *     http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-package com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception when a log writer attempts to acquire a lock to write data to the stream.
- */
-public class LockingException extends DLException {
-
-    private static final long serialVersionUID = -4960278188448464473L;
-
-    public LockingException(String lockPath, String message) {
-        this(StatusCode.LOCKING_EXCEPTION, lockPath, message);
-    }
-
-    public LockingException(String lockPath, String message, Throwable cause) {
-        this(StatusCode.LOCKING_EXCEPTION, lockPath, message, cause);
-    }
-
-    protected LockingException(StatusCode code, String lockPath, String message) {
-        super(code, String.format("LockPath - %s: %s", lockPath, message));
-    }
-
-    protected LockingException(StatusCode code, String lockPath, String message, Throwable cause) {
-        super(code, String.format("LockPath - %s: %s", lockPath, message), cause);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogEmptyException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogEmptyException.java
deleted file mode 100644
index 7dcd644..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogEmptyException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exceptions are thrown when attempt to read a log stream that doesn't have any records.
- */
-public class LogEmptyException extends DLException {
-
-    private static final long serialVersionUID = -1106184127178002282L;
-
-    public LogEmptyException(String message) {
-        super(StatusCode.LOG_EMPTY, message);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogExistsException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogExistsException.java
deleted file mode 100644
index e7a0d6d..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogExistsException.java
+++ /dev/null
@@ -1,31 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception thrown on creating a log stream but the log stream already exists.
- */
-public class LogExistsException extends DLException {
-    private static final long serialVersionUID = 1794053581673506784L;
-
-    public LogExistsException(String msg) {
-        super(StatusCode.LOG_EXISTS, msg);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogNotFoundException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogNotFoundException.java
deleted file mode 100644
index e2c93e8..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogNotFoundException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception is thrown when a reader attempts to read a log stream that doesn't exist.
- */
-public class LogNotFoundException extends DLException {
-
-    private static final long serialVersionUID = 871435700699403164L;
-
-    public LogNotFoundException(String message) {
-        super(StatusCode.LOG_NOT_FOUND, message);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogReadException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogReadException.java
deleted file mode 100644
index 6b08f53..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogReadException.java
+++ /dev/null
@@ -1,33 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Thrown when there's a failure to read an edit log op from disk when loading
- * edits.
- */
-public class LogReadException extends DLException {
-
-    private static final long serialVersionUID = 302945709106128177L;
-
-    public LogReadException(String message) {
-        super(StatusCode.LOG_READ_ERROR, message);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogRecordTooLongException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogRecordTooLongException.java
deleted file mode 100644
index 76a1aa2..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogRecordTooLongException.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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception is thrown when attempting to write a record whose size is too larger.
- *
- * <p>The size limit of a log record is {@link com.twitter.distributedlog.LogRecord#MAX_LOGRECORD_SIZE}.
- */
-public class LogRecordTooLongException extends DLException {
-
-    private static final long serialVersionUID = 2788274084603111386L;
-
-    public LogRecordTooLongException(String message) {
-        super(StatusCode.TOO_LARGE_RECORD, message);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogSegmentIsTruncatedException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogSegmentIsTruncatedException.java
deleted file mode 100644
index 2b56550..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogSegmentIsTruncatedException.java
+++ /dev/null
@@ -1,33 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception is thrown when reading data from a truncated log segment.
- */
-public class LogSegmentIsTruncatedException extends DLException {
-
-    private static final long serialVersionUID = -218506870918498062L;
-
-    public LogSegmentIsTruncatedException(String logSegmentName) {
-        super(StatusCode.LOG_SEGMENT_IS_TRUNCATED, "Log Segment '"
-                + logSegmentName + "'" + " is already truncated");
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogSegmentNotFoundException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogSegmentNotFoundException.java
deleted file mode 100644
index 292d135..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/LogSegmentNotFoundException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception on log segment not found.
- */
-public class LogSegmentNotFoundException extends DLException {
-
-    private static final long serialVersionUID = -2482324226595903864L;
-
-    public LogSegmentNotFoundException(String logSegmentPath) {
-        super(StatusCode.LOG_SEGMENT_NOT_FOUND, "Log Segment " + logSegmentPath + " not found");
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/MetadataException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/MetadataException.java
deleted file mode 100644
index cb62e7c..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/MetadataException.java
+++ /dev/null
@@ -1,36 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception is thrown when encountering metadata errors.
- */
-public class MetadataException extends DLException {
-
-    private static final long serialVersionUID = 6683578078046016125L;
-
-    public MetadataException(String msg) {
-        super(StatusCode.METADATA_EXCEPTION, msg);
-    }
-
-    public MetadataException(String msg, Throwable t) {
-        super(StatusCode.METADATA_EXCEPTION, msg, t);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/NotYetImplementedException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/NotYetImplementedException.java
deleted file mode 100644
index 04edb01..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/NotYetImplementedException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception is thrown when a method is not implemented yet.
- */
-public class NotYetImplementedException extends DLException {
-
-    private static final long serialVersionUID = -6002036746792556106L;
-
-    public NotYetImplementedException(String method) {
-        super(StatusCode.NOT_IMPLEMENTED, method + "is not supported by the current version");
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/OverCapacityException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/OverCapacityException.java
deleted file mode 100644
index d564758..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/OverCapacityException.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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception is thrown when the system is over capacity.
- *
- * <p>When this exception is received, the client (either writer or reader) should take actions to not
- * overwhelm the system. For example, backoff sending any requests.
- */
-public class OverCapacityException extends DLException {
-
-    private static final long serialVersionUID = -6398949404860680263L;
-
-    public OverCapacityException(String message) {
-        super(StatusCode.OVER_CAPACITY, message);
-    }
-
-    public OverCapacityException(StatusCode code, String message) {
-        super(code, message);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/OwnershipAcquireFailedException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/OwnershipAcquireFailedException.java
deleted file mode 100644
index 7816333..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/OwnershipAcquireFailedException.java
+++ /dev/null
@@ -1,40 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception is thrown when a log writer attempt to acquire a lock.
- *
- * <p>It is typically thrown when the lock is already acquired by another writer.
- */
-public class OwnershipAcquireFailedException extends LockingException {
-    private static final long serialVersionUID = 8176056926552748001L;
-    private final String currentOwner;
-
-    public OwnershipAcquireFailedException(String lockPath, String currentOwner) {
-        super(StatusCode.FOUND, lockPath,
-              String.format("Lock acquisition failed, the current owner is %s", currentOwner));
-        this.currentOwner = currentOwner;
-    }
-
-    public String getCurrentOwner() {
-        return currentOwner;
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/ReadCancelledException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/ReadCancelledException.java
deleted file mode 100644
index 84d9271..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/ReadCancelledException.java
+++ /dev/null
@@ -1,33 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Signals that a read request has been cancelled.
- */
-public class ReadCancelledException extends DLException {
-
-    private static final long serialVersionUID = -6273430297547510262L;
-
-    public ReadCancelledException(String stream, String reason) {
-        super(StatusCode.READ_CANCELLED_EXCEPTION,
-              "Read cancelled on stream " + stream + " : " + reason);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/RegionUnavailableException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/RegionUnavailableException.java
deleted file mode 100644
index 440d86e..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/RegionUnavailableException.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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception indicates that the service is not available in one region.
- *
- * <p>The client should retry the other regions when encountering this exception.
- */
-public class RegionUnavailableException extends DLException {
-
-    private static final long serialVersionUID = 5727337162533143957L;
-
-    public RegionUnavailableException(String msg) {
-        super(StatusCode.REGION_UNAVAILABLE, msg);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/RequestDeniedException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/RequestDeniedException.java
deleted file mode 100644
index 0a1e1c5..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/RequestDeniedException.java
+++ /dev/null
@@ -1,33 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Signals that a request has been denied at the server.
- */
-public class RequestDeniedException extends DLException {
-
-    private static final long serialVersionUID = 7338220414584728216L;
-
-    public RequestDeniedException(String stream, String operation) {
-        super(StatusCode.REQUEST_DENIED,
-                operation + " request to stream " + stream + " is denied");
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/RetryableReadException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/RetryableReadException.java
deleted file mode 100644
index 5224564..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/RetryableReadException.java
+++ /dev/null
@@ -1,36 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Signals that a read request can be retried..
- */
-public class RetryableReadException extends DLException {
-
-    private static final long serialVersionUID = 2803207702150642330L;
-
-    public RetryableReadException (String streamName, String message) {
-        super(StatusCode.RETRYABLE_READ, String.format("Reader on %s failed with %s", streamName, message));
-    }
-
-    public RetryableReadException (String streamName, String message, Throwable cause) {
-        super(StatusCode.RETRYABLE_READ, String.format("Reader on %s failed with %s", streamName, message), cause);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/ServiceUnavailableException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/ServiceUnavailableException.java
deleted file mode 100644
index 29e20d8..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/ServiceUnavailableException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception indicates that the service is unavailable at the server side.
- */
-public class ServiceUnavailableException extends DLException {
-
-    private static final long serialVersionUID = 6317900286881665746L;
-
-    public ServiceUnavailableException(String msg) {
-        super(StatusCode.SERVICE_UNAVAILABLE, msg);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/StreamNotReadyException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/StreamNotReadyException.java
deleted file mode 100644
index 6cac468..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/StreamNotReadyException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception is thrown when a log stream is not ready on server side for serving the write requests.
- */
-public class StreamNotReadyException extends DLException {
-
-    private static final long serialVersionUID = 684211282036293028L;
-
-    public StreamNotReadyException(String msg) {
-        super(StatusCode.STREAM_NOT_READY, msg);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/StreamUnavailableException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/StreamUnavailableException.java
deleted file mode 100644
index 9872fdc..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/StreamUnavailableException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception thrown when a stream is not available for serving traffic.
- */
-public class StreamUnavailableException extends DLException {
-
-    private static final long serialVersionUID = 684211282036993028L;
-
-    public StreamUnavailableException(String msg) {
-        super(StatusCode.STREAM_UNAVAILABLE, msg);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/TooManyStreamsException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/TooManyStreamsException.java
deleted file mode 100644
index 6ee405d..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/TooManyStreamsException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Signals that a server has been serving too many streams.
- */
-public class TooManyStreamsException extends OverCapacityException {
-
-    private static final long serialVersionUID = -6391941401860180163L;
-
-    public TooManyStreamsException(String message) {
-        super(StatusCode.TOO_MANY_STREAMS, message);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/TransactionIdOutOfOrderException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/TransactionIdOutOfOrderException.java
deleted file mode 100644
index 915525c..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/TransactionIdOutOfOrderException.java
+++ /dev/null
@@ -1,47 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * An exception is thrown when a log writer attempts to write a record with out-of-order transaction id.
- */
-public class TransactionIdOutOfOrderException extends DLException {
-
-    private static final long serialVersionUID = -6239322552103630036L;
-    // TODO: copied from DistributedLogConstants (we should think about how to separated common constants)
-    public static final long INVALID_TXID = -999;
-    private final long lastTxnId;
-
-    public TransactionIdOutOfOrderException(long smallerTxnId, long lastTxnId) {
-        super(StatusCode.TRANSACTION_OUT_OF_ORDER,
-              "Received smaller txn id " + smallerTxnId + ", last txn id is " + lastTxnId);
-        this.lastTxnId = lastTxnId;
-    }
-
-    public TransactionIdOutOfOrderException(long invalidTxnId) {
-        super(StatusCode.TRANSACTION_OUT_OF_ORDER,
-            "The txn id " + invalidTxnId + " is invalid and will break the sequence");
-        lastTxnId = INVALID_TXID;
-    }
-
-    public long getLastTxnId() {
-        return lastTxnId;
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/UnexpectedException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/UnexpectedException.java
deleted file mode 100644
index 4454cd0..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/UnexpectedException.java
+++ /dev/null
@@ -1,40 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * An {@code UnexpectedException} is thrown when encountering unexpected conditions.
- */
-public class UnexpectedException extends DLException {
-
-    private static final long serialVersionUID = 903763128422774055L;
-
-    public UnexpectedException() {
-        super(StatusCode.UNEXPECTED);
-    }
-
-    public UnexpectedException(String msg) {
-        super(StatusCode.UNEXPECTED, msg);
-    }
-
-    public UnexpectedException(String msg, Throwable t) {
-        super(StatusCode.UNEXPECTED, msg, t);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/UnsupportedMetadataVersionException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/UnsupportedMetadataVersionException.java
deleted file mode 100644
index 47c361a..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/UnsupportedMetadataVersionException.java
+++ /dev/null
@@ -1,32 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Exception is thrown when found unsupported metadata version.
- */
-public class UnsupportedMetadataVersionException extends DLException {
-
-    private static final long serialVersionUID = 4980892659955478446L;
-
-    public UnsupportedMetadataVersionException(String message) {
-        super(StatusCode.UNSUPPORTED_METADATA_VERSION, String.format(message));
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/WriteCancelledException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/WriteCancelledException.java
deleted file mode 100644
index f3adac2..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/WriteCancelledException.java
+++ /dev/null
@@ -1,45 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * Signals when a write request is cancelled.
- */
-public class WriteCancelledException extends DLException {
-
-    private static final long serialVersionUID = -1836146493496072122L;
-
-    public WriteCancelledException(String stream, Throwable t) {
-        super(StatusCode.WRITE_CANCELLED_EXCEPTION,
-            "Write cancelled on stream "
-                + stream + " due to an earlier error", t);
-    }
-
-    public WriteCancelledException(String stream, String reason) {
-        super(StatusCode.WRITE_CANCELLED_EXCEPTION,
-                "Write cancelled on stream " + stream + " due to : " + reason);
-    }
-
-    public WriteCancelledException(String stream) {
-        super(StatusCode.WRITE_CANCELLED_EXCEPTION,
-            "Write cancelled on stream "
-                + stream + " due to an earlier error");
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/WriteException.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/WriteException.java
deleted file mode 100644
index 77c18d4..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/WriteException.java
+++ /dev/null
@@ -1,33 +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 com.twitter.distributedlog.exceptions;
-
-import com.twitter.distributedlog.thrift.service.StatusCode;
-
-/**
- * An exception on writing log records.
- */
-public class WriteException extends DLException {
-
-    private static final long serialVersionUID = -1836146493446072122L;
-
-    public WriteException(String stream, String transmitError) {
-        super(StatusCode.WRITE_EXCEPTION,
-            "Write rejected because stream " + stream + " has encountered an error : " + transmitError);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/package-info.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/package-info.java
deleted file mode 100644
index ad027d8..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/exceptions/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Defines exceptions used across distributedlog project.
- */
-package com.twitter.distributedlog.exceptions;
\ No newline at end of file
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/Buffer.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/Buffer.java
deleted file mode 100644
index 4c9b23f..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/Buffer.java
+++ /dev/null
@@ -1,33 +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 com.twitter.distributedlog.io;
-
-import java.io.ByteArrayOutputStream;
-
-/**
- * {@link ByteArrayOutputStream} based buffer.
- */
-public class Buffer extends ByteArrayOutputStream {
-    public Buffer(int initialCapacity) {
-        super(initialCapacity);
-    }
-
-    public byte[] getData() {
-        return buf;
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/CompressionCodec.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/CompressionCodec.java
deleted file mode 100644
index 0f1753a..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/CompressionCodec.java
+++ /dev/null
@@ -1,80 +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 com.twitter.distributedlog.io;
-
-import org.apache.bookkeeper.stats.OpStatsLogger;
-
-/**
- * Common interface for compression/decompression operations using different compression codecs.
- */
-public interface CompressionCodec {
-    /**
-     * Enum specifying the currently supported compression types.
-     */
-    enum Type {
-        NONE, LZ4, UNKNOWN
-    }
-
-    /**
-     * Return the compressed data as a byte array.
-     * @param data
-     *          The data to be compressed
-     * @param offset
-     *          The offset in the bytes of data to compress
-     * @param length
-     *          The number of bytes of data to compress
-     * @param compressionStat
-     *          The stat to use for timing the compression operation
-     * @return
-     *          The compressed data
-     *          The returned byte array is sized to the length of the compressed data
-     */
-    byte[] compress(byte[] data, int offset, int length, OpStatsLogger compressionStat);
-
-    /**
-     * Return the decompressed data as a byte array.
-     * @param data
-     *          The data to be decompressed
-     * @param offset
-     *          The offset in the bytes of data to decompress
-     * @param length
-     *          The number of bytes of data to decompress
-     * @param decompressionStat
-     *          The stat to use for timing the decompression operation
-     * @return
-     *          The decompressed data
-     */
-    byte[] decompress(byte[] data, int offset, int length, OpStatsLogger decompressionStat);
-
-    /**
-     * Return the decompressed data as a byte array.
-     * @param data
-     *          The data to the decompressed
-     * @param offset
-     *          The offset in the bytes of data to decompress
-     * @param length
-     *          The number of bytes of data to decompress
-     * @param decompressedSize
-     *          The exact size of the decompressed data
-     * @param decompressionStat
-     *          The stat to use for timing the decompression operation
-     * @return
-     *          The decompressed data
-     */
-    byte[] decompress(byte[] data, int offset, int length, int decompressedSize, OpStatsLogger decompressionStat);
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/CompressionUtils.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/CompressionUtils.java
deleted file mode 100644
index 2552786..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/CompressionUtils.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 com.twitter.distributedlog.io;
-
-/**
- * Utils for compression related operations.
- */
-public class CompressionUtils {
-
-    public static final String LZ4 = "lz4";
-    public static final String NONE = "none";
-
-    private static final CompressionCodec IDENTITY_CODEC = new IdentityCompressionCodec();
-    private static final CompressionCodec LZ4_CODEC = new LZ4CompressionCodec();
-
-    /**
-     * Get a cached compression codec instance for the specified type.
-     * @param type compression codec type
-     * @return compression codec instance
-     */
-    public static CompressionCodec getCompressionCodec(CompressionCodec.Type type) {
-        if (type == CompressionCodec.Type.LZ4) {
-            return LZ4_CODEC;
-        }
-        // No Compression
-        return IDENTITY_CODEC;
-    }
-
-    /**
-     * Compression type value from string.
-     * @param compressionString compression codec presentation in string
-     * @return compression codec type
-     */
-    public static CompressionCodec.Type stringToType(String compressionString) {
-        if (compressionString.equals(LZ4)) {
-            return CompressionCodec.Type.LZ4;
-        } else if (compressionString.equals(NONE)) {
-            return CompressionCodec.Type.NONE;
-        } else {
-            return CompressionCodec.Type.UNKNOWN;
-        }
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/IdentityCompressionCodec.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/IdentityCompressionCodec.java
deleted file mode 100644
index 39952c3..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/IdentityCompressionCodec.java
+++ /dev/null
@@ -1,50 +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 com.twitter.distributedlog.io;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import java.util.Arrays;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-
-/**
- * An identity compression codec implementation for {@link CompressionCodec}.
- */
-public class IdentityCompressionCodec implements CompressionCodec {
-    @Override
-    public byte[] compress(byte[] data, int offset, int length, OpStatsLogger compressionStat) {
-        checkNotNull(data);
-        checkArgument(length >= 0);
-        return Arrays.copyOfRange(data, offset, offset + length);
-    }
-
-    @Override
-    public byte[] decompress(byte[] data, int offset, int length, OpStatsLogger decompressionStat) {
-        checkNotNull(data);
-        return Arrays.copyOfRange(data, offset, offset + length);
-    }
-
-    @Override
-    // Decompressed size is the same as the length of the data because this is an
-    // Identity compressor
-    public byte[] decompress(byte[] data, int offset, int length,
-                             int decompressedSize, OpStatsLogger decompressionStat) {
-        return decompress(data, offset, length, decompressionStat);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/LZ4CompressionCodec.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/LZ4CompressionCodec.java
deleted file mode 100644
index 12d6ac4..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/LZ4CompressionCodec.java
+++ /dev/null
@@ -1,101 +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 com.twitter.distributedlog.io;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.google.common.base.Stopwatch;
-import java.util.concurrent.TimeUnit;
-import net.jpountz.lz4.LZ4Compressor;
-import net.jpountz.lz4.LZ4Exception;
-import net.jpountz.lz4.LZ4Factory;
-import net.jpountz.lz4.LZ4FastDecompressor;
-import net.jpountz.lz4.LZ4SafeDecompressor;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-
-/**
- * An {@code lz4} based {@link CompressionCodec} implementation.
- *
- * <p>All functions are thread safe.
- */
-public class LZ4CompressionCodec implements CompressionCodec {
-
-    // Used for compression
-    private final LZ4Compressor compressor;
-    // Used to decompress when the size of the output is known
-    private final LZ4FastDecompressor fastDecompressor;
-    // Used to decompress when the size of the output is not known
-    private final LZ4SafeDecompressor safeDecompressor;
-
-    public LZ4CompressionCodec() {
-        this.compressor = LZ4Factory.fastestInstance().fastCompressor();
-        this.fastDecompressor = LZ4Factory.fastestInstance().fastDecompressor();
-        this.safeDecompressor = LZ4Factory.fastestInstance().safeDecompressor();
-    }
-
-    @Override
-    public byte[] compress(byte[] data, int offset, int length, OpStatsLogger compressionStat) {
-        checkNotNull(data);
-        checkArgument(offset >= 0 && offset < data.length);
-        checkArgument(length >= 0);
-        checkNotNull(compressionStat);
-
-        Stopwatch watch = Stopwatch.createStarted();
-        byte[] compressed = compressor.compress(data, offset, length);
-        compressionStat.registerSuccessfulEvent(watch.elapsed(TimeUnit.MICROSECONDS));
-        return compressed;
-    }
-
-    @Override
-    public byte[] decompress(byte[] data, int offset, int length, OpStatsLogger decompressionStat) {
-        checkNotNull(data);
-        checkArgument(offset >= 0 && offset < data.length);
-        checkArgument(length >= 0);
-        checkNotNull(decompressionStat);
-
-        Stopwatch watch = Stopwatch.createStarted();
-        // Assume that we have a compression ratio of 1/3.
-        int outLength = length * 3;
-        while (true) {
-            try {
-                byte[] decompressed = safeDecompressor.decompress(data, offset, length, outLength);
-                decompressionStat.registerSuccessfulEvent(watch.elapsed(TimeUnit.MICROSECONDS));
-                return decompressed;
-            } catch (LZ4Exception e) {
-                outLength *= 2;
-            }
-        }
-    }
-
-    @Override
-    // length parameter is ignored here because of the way the fastDecompressor works.
-    public byte[] decompress(byte[] data, int offset, int length, int decompressedSize,
-                             OpStatsLogger decompressionStat) {
-        checkNotNull(data);
-        checkArgument(offset >= 0 && offset < data.length);
-        checkArgument(length >= 0);
-        checkArgument(decompressedSize >= 0);
-        checkNotNull(decompressionStat);
-
-        Stopwatch watch = Stopwatch.createStarted();
-        byte[] decompressed = fastDecompressor.decompress(data, offset, decompressedSize);
-        decompressionStat.registerSuccessfulEvent(watch.elapsed(TimeUnit.MICROSECONDS));
-        return decompressed;
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/TransmitListener.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/TransmitListener.java
deleted file mode 100644
index 4196912..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/TransmitListener.java
+++ /dev/null
@@ -1,56 +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 com.twitter.distributedlog.io;
-
-import com.twitter.distributedlog.DLSN;
-
-/**
- * Listener on transmit results.
- */
-public interface TransmitListener {
-
-    /**
-     * Finalize the transmit result and result the last
-     * {@link DLSN} in this transmit.
-     *
-     * @param lssn
-     *          log segment sequence number
-     * @param entryId
-     *          entry id
-     * @return last dlsn in this transmit
-     */
-    DLSN finalizeTransmit(long lssn, long entryId);
-
-    /**
-     * Complete the whole transmit.
-     *
-     * @param lssn
-     *          log segment sequence number
-     * @param entryId
-     *          entry id
-     */
-    void completeTransmit(long lssn, long entryId);
-
-    /**
-     * Abort the transmit.
-     *
-     * @param reason
-     *          reason to abort transmit
-     */
-    void abortTransmit(Throwable reason);
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/package-info.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/package-info.java
deleted file mode 100644
index bbd9528..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/io/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * I/O related utilities used across the project.
- */
-package com.twitter.distributedlog.io;
\ No newline at end of file
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/package-info.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/package-info.java
deleted file mode 100644
index 8fc526d..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Provides the core data structures used across multiple modules.
- */
-package com.twitter.distributedlog;
\ No newline at end of file
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/util/BitMaskUtils.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/util/BitMaskUtils.java
deleted file mode 100644
index 07d9d26..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/util/BitMaskUtils.java
+++ /dev/null
@@ -1,57 +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 com.twitter.distributedlog.util;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-/**
- * Utils for bit mask operations.
- */
-public class BitMaskUtils {
-
-    /**
-     * 1) Unset all bits where value in mask is set.
-     * 2) Set these bits to value specified by newValue.
-     *
-     * <p>e.g.
-     * if oldValue = 1010, mask = 0011, newValue = 0001
-     * 1) 1010 -> 1000
-     * 2) 1000 -> 1001
-     *
-     * @param oldValue expected old value
-     * @param mask the mask of the value for updates
-     * @param newValue new value to set
-     * @return updated value
-     */
-    public static long set(long oldValue, long mask, long newValue) {
-        checkArgument(oldValue >= 0L && mask >= 0L && newValue >= 0L);
-        return ((oldValue & (~mask)) | (newValue & mask));
-    }
-
-    /**
-     * Get the bits where mask is 1.
-     *
-     * @param value value
-     * @param mask mask of the value
-     * @return the bit of the mask
-     */
-    public static long get(long value, long mask) {
-        checkArgument(value >= 0L && mask >= 0L);
-        return (value & mask);
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/util/ProtocolUtils.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/util/ProtocolUtils.java
deleted file mode 100644
index bac5cdd..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/util/ProtocolUtils.java
+++ /dev/null
@@ -1,79 +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 com.twitter.distributedlog.util;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-import com.twitter.distributedlog.DLSN;
-import java.util.zip.CRC32;
-
-/**
- * With CRC embedded in the application, we have to keep track of per api crc. Ideally this
- * would be done by thrift.
- */
-public class ProtocolUtils {
-
-    // For request payload checksum
-    private static final ThreadLocal<CRC32> requestCRC = new ThreadLocal<CRC32>() {
-        @Override
-        protected CRC32 initialValue() {
-            return new CRC32();
-        }
-    };
-
-    /**
-     * Generate crc32 for WriteOp.
-     */
-    public static Long writeOpCRC32(String stream, byte[] payload) {
-        CRC32 crc = requestCRC.get();
-        try {
-            crc.update(stream.getBytes(UTF_8));
-            crc.update(payload);
-            return crc.getValue();
-        } finally {
-            crc.reset();
-        }
-    }
-
-    /**
-     * Generate crc32 for TruncateOp.
-     */
-    public static Long truncateOpCRC32(String stream, DLSN dlsn) {
-        CRC32 crc = requestCRC.get();
-        try {
-            crc.update(stream.getBytes(UTF_8));
-            crc.update(dlsn.serializeBytes());
-            return crc.getValue();
-        } finally {
-            crc.reset();
-        }
-    }
-
-    /**
-     * Generate crc32 for any op which only passes a stream name.
-     */
-    public static Long streamOpCRC32(String stream) {
-        CRC32 crc = requestCRC.get();
-        try {
-            crc.update(stream.getBytes(UTF_8));
-            return crc.getValue();
-        } finally {
-            crc.reset();
-        }
-    }
-}
diff --git a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/util/package-info.java b/distributedlog-protocol/src/main/java/com/twitter/distributedlog/util/package-info.java
deleted file mode 100644
index f95e666..0000000
--- a/distributedlog-protocol/src/main/java/com/twitter/distributedlog/util/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * defines the utilities used across the project.
- */
-package com.twitter.distributedlog.util;
\ No newline at end of file
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/DLSN.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/DLSN.java
new file mode 100644
index 0000000..8a496ab
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/DLSN.java
@@ -0,0 +1,266 @@
+/**
+ * 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 com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.annotations.VisibleForTesting;
+import java.nio.ByteBuffer;
+import org.apache.commons.codec.binary.Base64;
+import org.apache.commons.codec.binary.Hex;
+
+
+/**
+ * DistributedLog Sequence Number (DLSN) is the system generated sequence number for log record.
+ *
+ * <p>DLSN is comprised with 3 components:
+ * <ul>
+ * <li>LogSegment Sequence Number: the sequence number of log segment that the record is written in
+ * <li>Entry Id: the entry id of the entry that the record is batched in
+ * <li>Slot Id: the slot id that the record is in the entry
+ * </ul>
+ *
+ * @see LogRecordWithDLSN
+ */
+public class DLSN implements Comparable<DLSN> {
+
+    public static final byte VERSION0 = (byte) 0;
+    public static final byte VERSION1 = (byte) 1;
+
+    // The initial DLSN that DL starts with
+    public static final DLSN InitialDLSN = new DLSN(1, 0 , 0);
+    // The non-inclusive lower bound DLSN
+    public static final DLSN NonInclusiveLowerBound = new DLSN(1, 0 , -1);
+    // Invalid DLSN
+    public static final DLSN InvalidDLSN = new DLSN(0, -1, -1);
+
+    static final byte CUR_VERSION = VERSION1;
+    static final int VERSION0_LEN = Long.SIZE * 3 + Byte.SIZE;
+    static final int VERSION1_LEN = Long.SIZE * 3 / Byte.SIZE + 1;
+
+    private final long logSegmentSequenceNo;
+    private final long entryId;
+    private final long slotId;
+
+    public DLSN(long logSegmentSequenceNo, long entryId, long slotId) {
+        this.logSegmentSequenceNo = logSegmentSequenceNo;
+        this.entryId = entryId;
+        this.slotId = slotId;
+    }
+
+    /**
+     * Return the sequence number of the log segment that the record is written to.
+     *
+     * @return sequence number of the log segment that the record is written to.
+     */
+    public long getLogSegmentSequenceNo() {
+        return logSegmentSequenceNo;
+    }
+
+    /**
+     * Return the entry id of the batch that the record is written to.
+     *
+     * @return entry id of the batch that the record is written to.
+     */
+    public long getEntryId() {
+        return entryId;
+    }
+
+    /**
+     * Return the slot id in the batch that the record is written to.
+     *
+     * @return slot id in the batch that the record is written to.
+     */
+    public long getSlotId() {
+        return slotId;
+    }
+
+    @Override
+    public int compareTo(DLSN that) {
+        if (this.logSegmentSequenceNo != that.logSegmentSequenceNo) {
+            return (this.logSegmentSequenceNo < that.logSegmentSequenceNo) ? -1 : 1;
+        } else if (this.entryId != that.entryId) {
+            return (this.entryId < that.entryId) ? -1 : 1;
+        } else {
+            return (this.slotId < that.slotId) ? -1 : (this.slotId == that.slotId) ? 0 : 1;
+        }
+    }
+
+    /**
+     * Serialize the DLSN into bytes with current version.
+     *
+     * @return the serialized bytes
+     */
+    public byte[] serializeBytes() {
+        return serializeBytes(CUR_VERSION);
+    }
+
+    /**
+     * Serialize the DLSN into bytes with given <code>version</code>.
+     *
+     * @param version
+     *          version to serialize the DLSN
+     * @return the serialized bytes
+     */
+    public byte[] serializeBytes(byte version) {
+        checkArgument(version <= CUR_VERSION && version >= VERSION0);
+        byte[] data = new byte[CUR_VERSION == version ? VERSION1_LEN : VERSION0_LEN];
+        ByteBuffer bb = ByteBuffer.wrap(data);
+        bb.put(version);
+        bb.putLong(logSegmentSequenceNo);
+        bb.putLong(entryId);
+        bb.putLong(slotId);
+        return data;
+    }
+
+    /**
+     * Serialize the DLSN into base64 encoded string.
+     *
+     * @return serialized base64 string
+     * @see #serializeBytes()
+     */
+    public String serialize() {
+        return serialize(CUR_VERSION);
+    }
+
+    /**
+     * Serialize the DLSN into base64 encoded string with given <code>version</code>.
+     *
+     * @param version
+     *          version to serialize the DLSN
+     * @return the serialized base64 string
+     * @see #serializeBytes(byte)
+     */
+    public String serialize(byte version) {
+        return Base64.encodeBase64String(serializeBytes(version));
+    }
+
+    /**
+     * Deserialize the DLSN from base64 encoded string <code>dlsn</code>.
+     *
+     * @param dlsn
+     *          base64 encoded string
+     * @return dlsn
+     */
+    public static DLSN deserialize(String dlsn) {
+        byte[] data = Base64.decodeBase64(dlsn);
+        return deserializeBytes(data);
+    }
+
+    /**
+     * Deserialize the DLSN from bytes array.
+     *
+     * @param data
+     *          serialized bytes
+     * @return dlsn
+     */
+    public static DLSN deserializeBytes(byte[] data) {
+        ByteBuffer bb = ByteBuffer.wrap(data);
+        byte version = bb.get();
+        if (VERSION0 == version) {
+            if (VERSION0_LEN != data.length) {
+                throw new IllegalArgumentException("Invalid version zero DLSN " + Hex.encodeHexString(data));
+            }
+        } else if (VERSION1 == version) {
+            if (VERSION1_LEN != data.length) {
+                throw new IllegalArgumentException("Invalid version one DLSN " + Hex.encodeHexString(data));
+            }
+        } else {
+            throw new IllegalArgumentException("Invalid DLSN : version = "
+                    + version + ", " + Hex.encodeHexString(data));
+        }
+        return new DLSN(bb.getLong(), bb.getLong(), bb.getLong());
+    }
+
+    // Keep original version0 logic for testing.
+    @VisibleForTesting
+    static DLSN deserialize0(String dlsn) {
+        byte[] data = Base64.decodeBase64(dlsn);
+        ByteBuffer bb = ByteBuffer.wrap(data);
+        byte version = bb.get();
+        if (VERSION0 != version || VERSION0_LEN != data.length) {
+            throw new IllegalArgumentException("Invalid DLSN " + dlsn);
+        }
+        return new DLSN(bb.getLong(), bb.getLong(), bb.getLong());
+    }
+
+    @Override
+    public String toString() {
+        return "DLSN{"
+            + "logSegmentSequenceNo=" + logSegmentSequenceNo
+            + ", entryId=" + entryId
+            + ", slotId=" + slotId
+            + '}';
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (!(o instanceof DLSN)) {
+            return false;
+        }
+
+        DLSN dlsn = (DLSN) o;
+
+        if (entryId != dlsn.entryId) {
+            return false;
+        }
+        if (logSegmentSequenceNo != dlsn.logSegmentSequenceNo) {
+            return false;
+        }
+        if (slotId != dlsn.slotId) {
+            return false;
+        }
+
+        return true;
+    }
+
+    @Override
+    public int hashCode() {
+        int result = (int) (logSegmentSequenceNo ^ (logSegmentSequenceNo >>> 32));
+        result = 31 * result + (int) (entryId ^ (entryId >>> 32));
+        result = 31 * result + (int) (slotId ^ (slotId >>> 32));
+        return result;
+    }
+
+    /**
+     * Positions to a DLSN greater than the current value.
+     *
+     * <p>This may not correspond to an actual LogRecord, its just used by the positioning logic
+     * to position the reader.
+     *
+     * @return the next DLSN
+     */
+    public DLSN getNextDLSN() {
+        return new DLSN(logSegmentSequenceNo, entryId, slotId + 1);
+    }
+
+    /**
+     * Positions to next log segment than the current value.
+     *
+     * <p>this may not correspond to an actual LogRecord, its just used by the positioning logic
+     * to position the reader.
+     *
+     * @return the next DLSN
+     */
+    public DLSN positionOnTheNextLedger() {
+        return new DLSN(logSegmentSequenceNo + 1 , 0, 0);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetReader.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetReader.java
new file mode 100644
index 0000000..1648c6d
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetReader.java
@@ -0,0 +1,118 @@
+/**
+ * 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.LogRecordSet.COMPRESSION_CODEC_LZ4;
+import static org.apache.distributedlog.LogRecordSet.METADATA_COMPRESSION_MASK;
+import static org.apache.distributedlog.LogRecordSet.METADATA_VERSION_MASK;
+import static org.apache.distributedlog.LogRecordSet.NULL_OP_STATS_LOGGER;
+import static org.apache.distributedlog.LogRecordSet.VERSION;
+
+import org.apache.distributedlog.io.CompressionCodec;
+import org.apache.distributedlog.io.CompressionUtils;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.nio.ByteBuffer;
+
+
+/**
+ * Record reader to read records from an enveloped entry buffer.
+ */
+class EnvelopedRecordSetReader implements LogRecordSet.Reader {
+
+    private final long logSegmentSeqNo;
+    private final long entryId;
+    private final long transactionId;
+    private final long startSequenceId;
+    private int numRecords;
+    private final ByteBuffer reader;
+
+    // slot id
+    private long slotId;
+    private int position;
+
+    EnvelopedRecordSetReader(long logSegmentSeqNo,
+                             long entryId,
+                             long transactionId,
+                             long startSlotId,
+                             int startPositionWithinLogSegment,
+                             long startSequenceId,
+                             InputStream in)
+            throws IOException {
+        this.logSegmentSeqNo = logSegmentSeqNo;
+        this.entryId = entryId;
+        this.transactionId = transactionId;
+        this.slotId = startSlotId;
+        this.position = startPositionWithinLogSegment;
+        this.startSequenceId = startSequenceId;
+
+        // read data
+        DataInputStream src = new DataInputStream(in);
+        int metadata = src.readInt();
+        int version = metadata & METADATA_VERSION_MASK;
+        if (version != VERSION) {
+            throw new IOException(String.format("Version mismatch while reading. Received: %d,"
+                + " Required: %d", version, VERSION));
+        }
+        int codecCode = metadata & METADATA_COMPRESSION_MASK;
+        this.numRecords = src.readInt();
+        int originDataLen = src.readInt();
+        int actualDataLen = src.readInt();
+        byte[] compressedData = new byte[actualDataLen];
+        src.readFully(compressedData);
+
+        if (COMPRESSION_CODEC_LZ4 == codecCode) {
+            CompressionCodec codec = CompressionUtils.getCompressionCodec(CompressionCodec.Type.LZ4);
+            byte[] decompressedData = codec.decompress(compressedData, 0, actualDataLen,
+                    originDataLen, NULL_OP_STATS_LOGGER);
+            this.reader = ByteBuffer.wrap(decompressedData);
+        } else {
+            if (originDataLen != actualDataLen) {
+                throw new IOException("Inconsistent data length found for a non-compressed record set : original = "
+                        + originDataLen + ", actual = " + actualDataLen);
+            }
+            this.reader = ByteBuffer.wrap(compressedData);
+        }
+    }
+
+    @Override
+    public LogRecordWithDLSN nextRecord() throws IOException {
+        if (numRecords <= 0) {
+            return null;
+        }
+
+        int recordLen = reader.getInt();
+        byte[] recordData = new byte[recordLen];
+        reader.get(recordData);
+        DLSN dlsn = new DLSN(logSegmentSeqNo, entryId, slotId);
+
+        LogRecordWithDLSN record =
+                new LogRecordWithDLSN(dlsn, startSequenceId);
+        record.setPositionWithinLogSegment(position);
+        record.setTransactionId(transactionId);
+        record.setPayload(recordData);
+
+        ++slotId;
+        ++position;
+        --numRecords;
+
+        return record;
+    }
+
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetWriter.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetWriter.java
new file mode 100644
index 0000000..9d2d7a7
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/EnvelopedRecordSetWriter.java
@@ -0,0 +1,197 @@
+/**
+ * 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.LogRecord.MAX_LOGRECORD_SIZE;
+import static org.apache.distributedlog.LogRecordSet.COMPRESSION_CODEC_LZ4;
+import static org.apache.distributedlog.LogRecordSet.COMPRESSION_CODEC_NONE;
+import static org.apache.distributedlog.LogRecordSet.HEADER_LEN;
+import static org.apache.distributedlog.LogRecordSet.METADATA_COMPRESSION_MASK;
+import static org.apache.distributedlog.LogRecordSet.METADATA_VERSION_MASK;
+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;
+import java.nio.channels.Channels;
+import java.nio.channels.WritableByteChannel;
+import java.util.LinkedList;
+import java.util.List;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * {@link Buffer} based log record set writer.
+ */
+class EnvelopedRecordSetWriter implements LogRecordSet.Writer {
+
+    private static final Logger logger = LoggerFactory.getLogger(EnvelopedRecordSetWriter.class);
+
+    private final Buffer buffer;
+    private final DataOutputStream writer;
+    private final WritableByteChannel writeChannel;
+    private final List<Promise<DLSN>> promiseList;
+    private final CompressionCodec.Type codec;
+    private final int codecCode;
+    private int count = 0;
+    private ByteBuffer recordSetBuffer = null;
+
+    EnvelopedRecordSetWriter(int initialBufferSize,
+                             CompressionCodec.Type codec) {
+        this.buffer = new Buffer(Math.max(initialBufferSize, HEADER_LEN));
+        this.promiseList = new LinkedList<Promise<DLSN>>();
+        this.codec = codec;
+        switch (codec) {
+            case LZ4:
+                this.codecCode = COMPRESSION_CODEC_LZ4;
+                break;
+            default:
+                this.codecCode = COMPRESSION_CODEC_NONE;
+                break;
+        }
+        this.writer = new DataOutputStream(buffer);
+        try {
+            this.writer.writeInt((VERSION & METADATA_VERSION_MASK)
+                    | (codecCode & METADATA_COMPRESSION_MASK));
+            this.writer.writeInt(0); // count
+            this.writer.writeInt(0); // original len
+            this.writer.writeInt(0); // actual len
+        } catch (IOException e) {
+            logger.warn("Failed to serialize the header to an enveloped record set", e);
+        }
+        this.writeChannel = Channels.newChannel(writer);
+    }
+
+    synchronized List<Promise<DLSN>> getPromiseList() {
+        return promiseList;
+    }
+
+    @Override
+    public synchronized void writeRecord(ByteBuffer record,
+                                         Promise<DLSN> transmitPromise)
+            throws LogRecordTooLongException, WriteException {
+        int logRecordSize = record.remaining();
+        if (logRecordSize > MAX_LOGRECORD_SIZE) {
+            throw new LogRecordTooLongException(
+                    "Log Record of size " + logRecordSize + " written when only "
+                            + MAX_LOGRECORD_SIZE + " is allowed");
+        }
+        try {
+            writer.writeInt(record.remaining());
+            writeChannel.write(record);
+            ++count;
+            promiseList.add(transmitPromise);
+        } catch (IOException e) {
+            logger.error("Failed to append record to record set", e);
+            throw new WriteException("", "Failed to append record to record set");
+        }
+    }
+
+    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));
+            nextSlotId++;
+        }
+        promiseList.clear();
+    }
+
+    private synchronized void cancelPromises(Throwable reason) {
+        for (Promise<DLSN> promise : promiseList) {
+            promise.setException(reason);
+        }
+        promiseList.clear();
+    }
+
+    @Override
+    public int getNumBytes() {
+        return buffer.size();
+    }
+
+    @Override
+    public synchronized int getNumRecords() {
+        return count;
+    }
+
+    @Override
+    public synchronized ByteBuffer getBuffer() {
+        if (null == recordSetBuffer) {
+            recordSetBuffer = createBuffer();
+        }
+        return recordSetBuffer.duplicate();
+    }
+
+    ByteBuffer createBuffer() {
+        byte[] data = buffer.getData();
+        int dataOffset = HEADER_LEN;
+        int dataLen = buffer.size() - HEADER_LEN;
+
+        if (COMPRESSION_CODEC_LZ4 != codecCode) {
+            ByteBuffer recordSetBuffer = ByteBuffer.wrap(data, 0, buffer.size());
+            // update count
+            recordSetBuffer.putInt(4, count);
+            // update data len
+            recordSetBuffer.putInt(8, dataLen);
+            recordSetBuffer.putInt(12, dataLen);
+            return recordSetBuffer;
+        }
+
+        // compression
+
+        CompressionCodec compressor =
+                    CompressionUtils.getCompressionCodec(codec);
+        byte[] compressed =
+                compressor.compress(data, dataOffset, dataLen, NULL_OP_STATS_LOGGER);
+
+        ByteBuffer recordSetBuffer;
+        if (compressed.length > dataLen) {
+            byte[] newData = new byte[HEADER_LEN + compressed.length];
+            System.arraycopy(data, 0, newData, 0, HEADER_LEN + dataLen);
+            recordSetBuffer = ByteBuffer.wrap(newData);
+        } else {
+            recordSetBuffer = ByteBuffer.wrap(data);
+        }
+        // version
+        recordSetBuffer.position(4);
+        // update count
+        recordSetBuffer.putInt(count);
+        // update data len
+        recordSetBuffer.putInt(dataLen);
+        recordSetBuffer.putInt(compressed.length);
+        recordSetBuffer.put(compressed);
+        recordSetBuffer.flip();
+        return recordSetBuffer;
+    }
+
+    @Override
+    public void completeTransmit(long lssn, long entryId, long startSlotId) {
+        satisfyPromises(lssn, entryId, startSlotId);
+    }
+
+    @Override
+    public void abortTransmit(Throwable reason) {
+        cancelPromises(reason);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecord.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecord.java
new file mode 100644
index 0000000..f2a19fc
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecord.java
@@ -0,0 +1,606 @@
+/**
+ * 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.ByteArrayInputStream;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.InputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Log record is the basic element in a log.
+ *
+ * <p>A log is a sequence of log records. Each log record is a sequence of bytes.
+ * Log records are written sequentially into a stream, and will be assigned with
+ * an unique system generated sequence number {@link DLSN} (distributedlog sequence
+ * number). Besides {@link DLSN}, application can assign its own sequence number
+ * while constructing log records. The application defined sequence number is called
+ * <code>TransactionID</code> (<i>txid</i>). Either {@link DLSN} or <code>TransactionId</code>
+ * could be used to position readers to start from specific log records.
+ *
+ * <h3>User Record</h3>
+ *
+ * <p>User records are the records written by applications and read by applications. They
+ * are constructed via {@link #LogRecord(long, byte[])} by applications and appended to
+ * logs by writers. And they would be deserialized from bytes by the readers and return
+ * to applications.
+ *
+ * <h3>Control Record</h3>
+ *
+ * <p>Control records are special records that written by distributedlog. They are invisible
+ * to applications. They could be treated as <i>commit requests</i> as what people could find
+ * in distributed consensus algorithms, since they are usually written by distributedlog to
+ * commit application written records. <i>Commit</i> means making application written records
+ * visible to readers to achieve consistent views among them.
+ *
+ * <p>They are named as 'Control Records' for controlling visibility of application written records.
+ *
+ * <p>The transaction id of 'Control Records' are assigned by distributedlog by inheriting from last
+ * written user records. So we could indicate what user records that a control record is committing
+ * by looking at its transaction id.
+ *
+ * <h4>EndOfStream Record</h4>
+ *
+ * <p><code>EoS</code>(EndOfStream) is a special control record that would be written by a writer
+ * to seal a log. After a <i>EoS</i> record is written to a log, no writers could append any record
+ * after that and readers will get {@link org.apache.distributedlog.exceptions.EndOfStreamException}
+ * when they reach EoS.
+ *
+ * <p>TransactionID of EoS is <code>Long.MAX_VALUE</code>.
+ *
+ * <h3>Serialization & Deserialization</h3>
+ *
+ * <p>Data type in brackets. Interpretation should be on the basis of data types and not individual
+ * bytes to honor Endianness.
+ *
+ * <pre>
+ * LogRecord structure:
+ * -------------------
+ * Bytes 0 - 7                      : Metadata (Long)
+ * Bytes 8 - 15                     : TxId (Long)
+ * Bytes 16 - 19                    : Payload length (Integer)
+ * Bytes 20 - 20+payload.length-1   : Payload (Byte[])
+ *
+ * Metadata: 8 Bytes (Long)
+ * --------
+ *
+ * 0x 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0 0
+ *            |_____________| |_____|
+ *                   |           |
+ *               position      flags
+ *
+ * Flags: 2 Bytes (least significant)
+ * -----
+ * Bit  0      : If set, control record, else record with payload.
+ * Bit  1      : If set, end of stream.
+ * Bits 2 - 15 : Unused
+ * </pre>
+ *
+ * <h3>Sequence Numbers</h3>
+ *
+ * <p>A record is associated with three types of sequence numbers. They are generated
+ * and used for different purposes. Check {@link LogRecordWithDLSN} for more details.
+ *
+ * @see LogRecordWithDLSN
+ */
+public class LogRecord {
+
+    private static final Logger LOG = LoggerFactory.getLogger(LogRecord.class);
+
+    // Allow 4K overhead for metadata within the max transmission size
+    public static final int MAX_LOGRECORD_SIZE = 1024 * 1024 - 8 * 1024; //1MB - 8KB
+    // Allow 4K overhead for transmission overhead
+    public static final int MAX_LOGRECORDSET_SIZE = 1024 * 1024 - 4 * 1024; //1MB - 4KB
+
+    private static final int INPUTSTREAM_MARK_LIMIT = 16;
+
+    static final long LOGRECORD_METADATA_FLAGS_MASK = 0xffffL;
+    static final long LOGRECORD_METADATA_FLAGS_UMASK = 0xffffffffffff0000L;
+    static final long LOGRECORD_METADATA_POSITION_MASK = 0x0000ffffffff0000L;
+    static final long LOGRECORD_METADATA_POSITION_UMASK = 0xffff00000000ffffL;
+    static final int LOGRECORD_METADATA_POSITION_SHIFT = 16;
+    static final long LOGRECORD_METADATA_UNUSED_MASK = 0xffff000000000000L;
+
+    // TODO: Replace with EnumSet
+    static final long LOGRECORD_FLAGS_CONTROL_MESSAGE = 0x1;
+    static final long LOGRECORD_FLAGS_END_OF_STREAM = 0x2;
+    static final long LOGRECORD_FLAGS_RECORD_SET = 0x4;
+
+    private long metadata;
+    private long txid;
+    private byte[] payload;
+
+    /**
+     * Construct an uninitialized log record.
+     *
+     * <p>NOTE: only deserializer should call this constructor.
+     */
+    protected LogRecord() {
+        this.txid = 0;
+        this.metadata = 0;
+    }
+
+    /**
+     * Construct a log record with <i>TransactionId</i> and payload.
+     *
+     * <p>Usually writer would construct the log record for writing.
+     *
+     * @param txid
+     *          application defined transaction id.
+     * @param payload
+     *          record data
+     */
+    public LogRecord(long txid, byte[] payload) {
+        this.txid = txid;
+        this.payload = payload;
+        this.metadata = 0;
+    }
+
+    //
+    // Accessors
+    //
+
+    /**
+     * Return application defined transaction id.
+     *
+     * @return transacton id.
+     */
+    public long getTransactionId() {
+        return txid;
+    }
+
+    /**
+     * Set application defined transaction id.
+     *
+     * @param txid application defined transaction id.
+     */
+    protected void setTransactionId(long txid) {
+        this.txid = txid;
+    }
+
+    /**
+     * Return the payload of this log record.
+     *
+     * @return payload of this log record.
+     */
+    public byte[] getPayload() {
+        return payload;
+    }
+
+    /**
+     * Set payload for this log record.
+     *
+     * @param payload payload of this log record
+     */
+    void setPayload(byte[] payload) {
+        this.payload = payload;
+    }
+
+    /**
+     * Return the payload as an {@link InputStream}.
+     *
+     * @return payload as input stream
+     */
+    public InputStream getPayLoadInputStream() {
+        return new ByteArrayInputStream(payload);
+    }
+
+    //
+    // Metadata & Flags
+    //
+
+    protected void setMetadata(long metadata) {
+        this.metadata = metadata;
+    }
+
+    protected long getMetadata() {
+        return this.metadata;
+    }
+
+    /**
+     * Set the position in the log segment.
+     *
+     * @see #getPositionWithinLogSegment()
+     * @param positionWithinLogSegment position in the log segment.
+     */
+    void setPositionWithinLogSegment(int positionWithinLogSegment) {
+        assert(positionWithinLogSegment >= 0);
+        metadata = (metadata & LOGRECORD_METADATA_POSITION_UMASK)
+            | (((long) positionWithinLogSegment) << LOGRECORD_METADATA_POSITION_SHIFT);
+    }
+
+    /**
+     * The position in the log segment means how many records (inclusive) added to the log segment so far.
+     *
+     * @return position of the record in the log segment.
+     */
+    public int getPositionWithinLogSegment() {
+        long ret = (metadata & LOGRECORD_METADATA_POSITION_MASK) >> LOGRECORD_METADATA_POSITION_SHIFT;
+        if (ret < 0 || ret > Integer.MAX_VALUE) {
+            throw new IllegalArgumentException
+                (ret + " position should never exceed max integer value");
+        }
+        return (int) ret;
+    }
+
+    /**
+     * Get the last position of this record in the log segment.
+     *
+     * <p>If the record isn't record set, it would be same as {@link #getPositionWithinLogSegment()},
+     * otherwise, it would be {@link #getPositionWithinLogSegment()} + numRecords - 1. If the record set
+     * version is unknown, it would be same as {@link #getPositionWithinLogSegment()}.
+     *
+     * @return last position of this record in the log segment.
+     */
+    int getLastPositionWithinLogSegment() {
+        if (isRecordSet()) {
+            try {
+                return getPositionWithinLogSegment() + LogRecordSet.numRecords(this) - 1;
+            } catch (IOException e) {
+                // if it is unrecognized record set, we will return the position of this record set.
+                return getPositionWithinLogSegment();
+            }
+        } else {
+            return getPositionWithinLogSegment();
+        }
+    }
+
+    /**
+     * Set the record to represent a set of records.
+     *
+     * <p>The bytes in this record is the serialized format of {@link LogRecordSet}.
+     */
+    public void setRecordSet() {
+        metadata = metadata | LOGRECORD_FLAGS_RECORD_SET;
+    }
+
+    /**
+     * Check if the record represents a set of records.
+     *
+     * @return true if the record represents a set of records, otherwise false.
+     * @see #setRecordSet()
+     */
+    public boolean isRecordSet() {
+        return isRecordSet(metadata);
+    }
+
+    public static boolean isRecordSet(long metadata) {
+        return ((metadata & LOGRECORD_FLAGS_RECORD_SET) != 0);
+    }
+
+    @VisibleForTesting
+    public void setControl() {
+        metadata = metadata | LOGRECORD_FLAGS_CONTROL_MESSAGE;
+    }
+
+    /**
+     * Check if the record is a control record.
+     *
+     * @return true if the record is a control record, otherwise false.
+     */
+    public boolean isControl() {
+        return isControl(metadata);
+    }
+
+    /**
+     * Check flags to see if it indicates a control record.
+     *
+     * @param flags record flags
+     * @return true if the record is a control record, otherwise false.
+     */
+    public static boolean isControl(long flags) {
+        return ((flags & LOGRECORD_FLAGS_CONTROL_MESSAGE) != 0);
+    }
+
+    /**
+     * Set the record as <code>EoS</code> mark.
+     *
+     * @see #isEndOfStream()
+     */
+    void setEndOfStream() {
+        metadata = metadata | LOGRECORD_FLAGS_END_OF_STREAM;
+    }
+
+    /**
+     * Check if the record is a <code>EoS</code> mark.
+     *
+     * <p><code>EoS</code> mark is a special record that writer would
+     * add to seal a log. after <code>Eos</code> mark is written,
+     * writers can't write any more records and readers will get
+     * {@link org.apache.distributedlog.exceptions.EndOfStreamException}
+     * when they reach <code>EoS</code>.
+     *
+     * @return true
+     */
+    boolean isEndOfStream() {
+        return ((metadata & LOGRECORD_FLAGS_END_OF_STREAM) != 0);
+    }
+
+    //
+    // Serialization & Deserialization
+    //
+
+    protected void readPayload(DataInputStream in) throws IOException {
+        int length = in.readInt();
+        if (length < 0) {
+            throw new EOFException("Log Record is corrupt: Negative length " + length);
+        }
+        payload = new byte[length];
+        in.readFully(payload);
+    }
+
+    private void writePayload(DataOutputStream out) throws IOException {
+        out.writeInt(payload.length);
+        out.write(payload);
+    }
+
+    private void writeToStream(DataOutputStream out) throws IOException {
+        out.writeLong(metadata);
+        out.writeLong(txid);
+        writePayload(out);
+    }
+
+    /**
+     * The size of the serialized log record.
+     *
+     * <p>This is used to estimate how much will be be appended to the in-memory buffer.
+     *
+     * @return serialized size
+     */
+    int getPersistentSize() {
+        // Flags + TxId + Payload-length + payload
+        return 2 * (Long.SIZE / 8) + Integer.SIZE / 8 + payload.length;
+    }
+
+    /**
+     * Writer class to write log records into an output {@code stream}.
+     */
+    public static class Writer {
+        private final DataOutputStream buf;
+
+        public Writer(DataOutputStream out) {
+            this.buf = out;
+        }
+
+        /**
+         * Write an operation to the output stream.
+         *
+         * @param record The operation to write
+         * @throws IOException if an error occurs during writing.
+         */
+        public void writeOp(LogRecord record) throws IOException {
+            record.writeToStream(buf);
+        }
+
+        public int getPendingBytes() {
+            return buf.size();
+        }
+    }
+
+    /**
+     * Reader class to read log records from an input {@code stream}.
+      */
+    public static class Reader {
+        private final RecordStream recordStream;
+        private final DataInputStream in;
+        private final long startSequenceId;
+        private final boolean deserializeRecordSet;
+        private static final int SKIP_BUFFER_SIZE = 512;
+        private LogRecordSet.Reader recordSetReader = null;
+        private LogRecordWithDLSN lastRecordSkipTo = null;
+
+        /**
+         * Construct the reader.
+         *
+         * @param recordStream the record stream for generating {@code DLSN}s.
+         * @param in The stream to read from.
+         * @param startSequenceId the start sequence id.
+         */
+        public Reader(RecordStream recordStream,
+                      DataInputStream in,
+                      long startSequenceId) {
+            this(recordStream, in, startSequenceId, true);
+        }
+
+        public Reader(RecordStream recordStream,
+                      DataInputStream in,
+                      long startSequenceId,
+                      boolean deserializeRecordSet) {
+            this.recordStream = recordStream;
+            this.in = in;
+            this.startSequenceId = startSequenceId;
+            this.deserializeRecordSet = deserializeRecordSet;
+        }
+
+        /**
+         * Read an log record from the input stream.
+         *
+         * <p/> Note that the objects returned from this method may be re-used by future
+         * calls to the same method.
+         *
+         * @return the operation read from the stream, or null at the end of the file
+         * @throws IOException on error.
+         */
+        public LogRecordWithDLSN readOp() throws IOException {
+            LogRecordWithDLSN nextRecordInStream;
+            while (true) {
+                if (lastRecordSkipTo != null) {
+                    nextRecordInStream = lastRecordSkipTo;
+                    recordStream.advance(1);
+                    lastRecordSkipTo = null;
+                    return nextRecordInStream;
+                }
+                if (recordSetReader != null) {
+                    nextRecordInStream = recordSetReader.nextRecord();
+                    if (null != nextRecordInStream) {
+                        recordStream.advance(1);
+                        return nextRecordInStream;
+                    } else {
+                        recordSetReader = null;
+                    }
+                }
+
+                try {
+                    long metadata = in.readLong();
+                    // Reading the first 8 bytes positions the record stream on the correct log record
+                    // By this time all components of the DLSN are valid so this is where we shoud
+                    // retrieve the currentDLSN and advance to the next
+                    // Given that there are 20 bytes following the read position of the previous call
+                    // to readLong, we should not have moved ahead in the stream.
+                    nextRecordInStream = new LogRecordWithDLSN(recordStream.getCurrentPosition(), startSequenceId);
+                    nextRecordInStream.setMetadata(metadata);
+                    nextRecordInStream.setTransactionId(in.readLong());
+                    nextRecordInStream.readPayload(in);
+                    if (LOG.isTraceEnabled()) {
+                        if (nextRecordInStream.isControl()) {
+                            LOG.trace("Reading {} Control DLSN {}",
+                                recordStream.getName(), nextRecordInStream.getDlsn());
+                        } else {
+                            LOG.trace("Reading {} Valid DLSN {}",
+                                recordStream.getName(), nextRecordInStream.getDlsn());
+                        }
+                    }
+
+                    int numRecords = 1;
+                    if (!deserializeRecordSet && nextRecordInStream.isRecordSet()) {
+                        numRecords = LogRecordSet.numRecords(nextRecordInStream);
+                    }
+
+                    if (deserializeRecordSet && nextRecordInStream.isRecordSet()) {
+                        recordSetReader = LogRecordSet.of(nextRecordInStream);
+                    } else {
+                        recordStream.advance(numRecords);
+                        return nextRecordInStream;
+                    }
+                } catch (EOFException eof) {
+                    // Expected
+                    break;
+                }
+            }
+            return null;
+        }
+
+        public boolean skipTo(long txId, boolean skipControl) throws IOException {
+            return skipTo(txId, null, skipControl);
+        }
+
+        public boolean skipTo(DLSN dlsn) throws IOException {
+            return skipTo(null, dlsn, false);
+        }
+
+        private boolean skipTo(Long txId, DLSN dlsn, boolean skipControl) throws IOException {
+            LOG.debug("SkipTo");
+            byte[] skipBuffer = null;
+            boolean found = false;
+            while (true) {
+                try {
+                    long flags;
+                    long currTxId;
+
+                    // if there is not record set, read next record
+                    if (null == recordSetReader) {
+                        in.mark(INPUTSTREAM_MARK_LIMIT);
+                        flags = in.readLong();
+                        currTxId = in.readLong();
+                    } else {
+                        // check record set until reach end of record set
+                        lastRecordSkipTo = recordSetReader.nextRecord();
+                        if (null == lastRecordSkipTo) {
+                            // reach end of record set
+                            recordSetReader = null;
+                            continue;
+                        }
+                        flags = lastRecordSkipTo.getMetadata();
+                        currTxId = lastRecordSkipTo.getTransactionId();
+                    }
+
+                    if ((null != dlsn) && (recordStream.getCurrentPosition().compareTo(dlsn) >= 0)) {
+                        if (LOG.isTraceEnabled()) {
+                            LOG.trace("Found position {} beyond {}", recordStream.getCurrentPosition(), dlsn);
+                        }
+                        if (null == lastRecordSkipTo) {
+                            in.reset();
+                        }
+                        found = true;
+                        break;
+                    }
+                    if ((null != txId) && (currTxId >= txId)) {
+                        if (!skipControl || !isControl(flags)) {
+                            if (LOG.isTraceEnabled()) {
+                                LOG.trace("Found position {} beyond {}", currTxId, txId);
+                            }
+                            if (null == lastRecordSkipTo) {
+                                in.reset();
+                            }
+                            found = true;
+                            break;
+                        }
+                    }
+
+                    if (null != lastRecordSkipTo) {
+                        recordStream.advance(1);
+                        continue;
+                    }
+
+                    // get the num of records to skip
+                    if (isRecordSet(flags)) {
+                        // read record set
+                        LogRecordWithDLSN record =
+                            new LogRecordWithDLSN(recordStream.getCurrentPosition(), startSequenceId);
+                        record.setMetadata(flags);
+                        record.setTransactionId(currTxId);
+                        record.readPayload(in);
+                        recordSetReader = LogRecordSet.of(record);
+                    } else {
+                        int length = in.readInt();
+                        if (length < 0) {
+                            // We should never really see this as we only write complete entries to
+                            // BK and BK client has logic to detect torn writes (through checksum)
+                            LOG.info("Encountered Record with negative length at TxId: {}", currTxId);
+                            break;
+                        }
+                        // skip single record
+                        if (null == skipBuffer) {
+                            skipBuffer = new byte[SKIP_BUFFER_SIZE];
+                        }
+                        int read = 0;
+                        while (read < length) {
+                            int bytesToRead = Math.min(length - read, SKIP_BUFFER_SIZE);
+                            in.readFully(skipBuffer, 0, bytesToRead);
+                            read += bytesToRead;
+                        }
+                        if (LOG.isTraceEnabled()) {
+                            LOG.trace("Skipped Record with TxId {} DLSN {}",
+                                currTxId, recordStream.getCurrentPosition());
+                        }
+                        recordStream.advance(1);
+                    }
+                } catch (EOFException eof) {
+                    LOG.debug("Skip encountered end of file Exception", eof);
+                    break;
+                }
+            }
+            return found;
+        }
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordSet.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordSet.java
new file mode 100644
index 0000000..375ed3f
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordSet.java
@@ -0,0 +1,155 @@
+/**
+ * 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 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 org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+
+/**
+ * A set of {@link LogRecord}s.
+ *
+ * <pre>
+ * Structure:
+ * Bytes 0  -  4                : Metadata (version + flags)
+ * Bytes 4  - 7                 : Number of Records
+ * Bytes 8  - 11                : Original Payload Length
+ * Bytes 12 - 15                : Actual Payload Length
+ * Bytes 16 - 16+length-1       : Payload
+ * ------------------------------------------------------
+ *
+ * Metadata: Version and Flags // 32 Bits
+ * --------------------------------------
+ * 0 ... 0 0 0 0 0
+ *             |_|
+ *              |
+ *    Compression Codec: // 2 Bits (Least significant)
+ *    -----------------
+ *    00        : No Compression
+ *    01        : LZ4 Compression
+ *    10        : Unused
+ *    11        : Unused
+ *
+ * </pre>
+ */
+public class LogRecordSet {
+
+    static final OpStatsLogger NULL_OP_STATS_LOGGER =
+            NullStatsLogger.INSTANCE.getOpStatsLogger("");
+
+    public static final int HEADER_LEN =
+            4 /* Metadata */
+          + 4 /* Count */
+          + 8 /* Lengths */
+            ;
+
+    // Version
+    static final int VERSION = 0x1000;
+
+    // Metadata
+    static final int METADATA_VERSION_MASK = 0xf000;
+    static final int METADATA_COMPRESSION_MASK = 0x3;
+
+    // Compression Codec
+    static final int COMPRESSION_CODEC_NONE = 0x0;
+    static final int COMPRESSION_CODEC_LZ4 = 0X1;
+
+    public static int numRecords(LogRecord record) throws IOException {
+        checkArgument(record.isRecordSet(),
+                "record is not a recordset");
+        byte[] data = record.getPayload();
+        return numRecords(data);
+    }
+
+    public static int numRecords(byte[] data) throws IOException {
+        ByteBuffer buffer = ByteBuffer.wrap(data);
+        int metadata = buffer.getInt();
+        int version = (metadata & METADATA_VERSION_MASK);
+        if (version != VERSION) {
+            throw new IOException(String.format("Version mismatch while reading. Received: %d,"
+                + " Required: %d", version, VERSION));
+        }
+        return buffer.getInt();
+    }
+
+    public static Writer newWriter(int initialBufferSize,
+                                   CompressionCodec.Type codec) {
+        return new EnvelopedRecordSetWriter(initialBufferSize, codec);
+    }
+
+    public static Reader of(LogRecordWithDLSN record) throws IOException {
+        checkArgument(record.isRecordSet(),
+                "record is not a recordset");
+        byte[] data = record.getPayload();
+        DLSN dlsn = record.getDlsn();
+        int startPosition = record.getPositionWithinLogSegment();
+        long startSequenceId = record.getStartSequenceIdOfCurrentSegment();
+
+        return new EnvelopedRecordSetReader(
+                dlsn.getLogSegmentSequenceNo(),
+                dlsn.getEntryId(),
+                record.getTransactionId(),
+                dlsn.getSlotId(),
+                startPosition,
+                startSequenceId,
+                new ByteArrayInputStream(data));
+    }
+
+    /**
+     * Writer to append {@link LogRecord}s to {@link LogRecordSet}.
+     */
+    public interface Writer extends LogRecordSetBuffer {
+
+        /**
+         * Write a {@link LogRecord} to this record set.
+         *
+         * @param record
+         *          record to write
+         * @param transmitPromise
+         *          callback for transmit result. the promise is only
+         *          satisfied when this record set is transmitted.
+         * @throws LogRecordTooLongException if the record is too long
+         * @throws WriteException when encountered exception writing the record
+         */
+        void writeRecord(ByteBuffer record, Promise<DLSN> transmitPromise)
+                throws LogRecordTooLongException, WriteException;
+    }
+
+    /**
+     * Reader to read {@link LogRecord}s from this record set.
+     */
+    public interface Reader {
+
+        /**
+         * Read next log record from this record set.
+         *
+         * @return next log record from this record set.
+         */
+        LogRecordWithDLSN nextRecord() throws IOException;
+
+    }
+
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordSetBuffer.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordSetBuffer.java
new file mode 100644
index 0000000..1fac817
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordSetBuffer.java
@@ -0,0 +1,65 @@
+/**
+ * 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 java.nio.ByteBuffer;
+
+/**
+ * Write representation of a {@link LogRecordSet}.
+ * It is a buffer of log record set, used for transmission.
+ */
+public interface LogRecordSetBuffer {
+
+    /**
+     * Return number of records in current record set.
+     *
+     * @return number of records in current record set.
+     */
+    int getNumRecords();
+
+    /**
+     * Return number of bytes in current record set.
+     *
+     * @return number of bytes in current record set.
+     */
+    int getNumBytes();
+
+    /**
+     * Get the buffer to transmit.
+     *
+     * @return the buffer to transmit.
+     */
+    ByteBuffer getBuffer();
+
+    /**
+     * Complete transmit.
+     *
+     * @param lssn log segment sequence number
+     * @param entryId entry id
+     * @param startSlotId start slot id
+     */
+    void completeTransmit(long lssn, long entryId, long startSlotId);
+
+    /**
+     * Abort transmit.
+     *
+     * @param reason reason to abort.
+     */
+    void abortTransmit(Throwable reason);
+
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordWithDLSN.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordWithDLSN.java
new file mode 100644
index 0000000..3279a5a
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/LogRecordWithDLSN.java
@@ -0,0 +1,107 @@
+/**
+ * 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;
+
+/**
+ * Log record with {@link DLSN} and <code>SequenceId</code>.
+ *
+ * <h3>Sequence Numbers</h3>
+ *
+ * <p>A log record will be assigned with an unique system generated sequence number {@link DLSN} when it is
+ * written to a log. At the mean time, a 64-bits long number is assigned to the record indicates its position
+ * within a log, which is called <code>SequenceId</code>. Besides {@link DLSN} and <code>SequenceID</code>,
+ * application can assign its own sequence number (called <code>TransactionID</code>) to the log record while
+ * writing it.
+ *
+ * <h4>Transaction ID</h4>
+ *
+ * <p>Transaction ID is a positive 64-bits long number that is assigned by the application. It is a very helpful
+ * field when application wants to organize the records and position the readers using their own sequencing method.
+ * A typical use case of <code>TransactionID</code> is DistributedLog Write Proxy. It assigns the non-decreasing
+ * timestamps to log records, which the timestamps could be used as `physical time` to implement `TTL` in a strong
+ * consistent database to achieve consistent `TTL` over replicas.
+ *
+ * <h4>DLSN</h4>
+ *
+ * <p>DistributedLog Sequence Number (<i>DLSN</i>) is the sequence number generated during written time.
+ * It is comparable and could be used to figure out the order between records. The DLSN is comprised with 3 components.
+ * They are <i>Log Segment Sequence Number</i>, <i>Entry Id</i> and <i>Slot Id</i>. (See {@link DLSN} for more details).
+ * The DLSN is usually used for comparison, positioning and truncation.
+ *
+ * <h4>Sequence ID</h4>
+ *
+ * <p>Sequence ID is introduced to address the drawback of <code>DLSN</code>, in favor of answering questions like
+ * `how many records written between two DLSNs`. It is a 64-bits monotonic increasing number (starting from zero).
+ * Sequence ids are only accessible by readers. That means writers don't know the sequence ids of records after they
+ * wrote them.
+ */
+public class LogRecordWithDLSN extends LogRecord {
+    private final DLSN dlsn;
+    private final long startSequenceIdOfCurrentSegment;
+
+    /**
+     * This empty constructor can only be called from Reader#readOp.
+     */
+    LogRecordWithDLSN(DLSN dlsn, long startSequenceIdOfCurrentSegment) {
+        super();
+        this.dlsn = dlsn;
+        this.startSequenceIdOfCurrentSegment = startSequenceIdOfCurrentSegment;
+    }
+
+    @VisibleForTesting
+    public LogRecordWithDLSN(DLSN dlsn, long txid, byte[] data, long startSequenceIdOfCurrentSegment) {
+        super(txid, data);
+        this.dlsn = dlsn;
+        this.startSequenceIdOfCurrentSegment = startSequenceIdOfCurrentSegment;
+    }
+
+    long getStartSequenceIdOfCurrentSegment() {
+        return startSequenceIdOfCurrentSegment;
+    }
+
+    /**
+     * Get the sequence id of the record in the log.
+     *
+     * @return sequence id of the record in the log.
+     */
+    public long getSequenceId() {
+        return startSequenceIdOfCurrentSegment + getPositionWithinLogSegment() - 1;
+    }
+
+    /**
+     * Get the DLSN of the record in the log.
+     *
+     * @return DLSN of the record in the log.
+     */
+    public DLSN getDlsn() {
+        return dlsn;
+    }
+
+    @Override
+    public String toString() {
+        return "LogRecordWithDLSN{"
+            + "dlsn=" + dlsn
+            + ", txid=" + getTransactionId()
+            + ", position=" + getPositionWithinLogSegment()
+            + ", isControl=" + isControl()
+            + ", isEndOfStream=" + isEndOfStream()
+            + '}';
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/RecordStream.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/RecordStream.java
new file mode 100644
index 0000000..a2614c0
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/RecordStream.java
@@ -0,0 +1,42 @@
+/**
+ * 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;
+
+/**
+ * A iterator to iterate records in the stream.
+ */
+public interface RecordStream {
+    /**
+     * advance <i>numRecords</i> records.
+     */
+    void advance(int numRecords);
+
+    /**
+     * Get position of current record in the stream.
+     *
+     * @return position of current record
+     */
+    DLSN getCurrentPosition();
+
+    /**
+     * Get the name of the stream.
+     *
+     * @return the name of the stream
+     */
+    String getName();
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/annotations/DistributedLogAnnotations.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/annotations/DistributedLogAnnotations.java
new file mode 100644
index 0000000..f8cdea4
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/annotations/DistributedLogAnnotations.java
@@ -0,0 +1,35 @@
+/**
+ * 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.annotations;
+
+/**
+ * Common annotation types.
+ */
+public class DistributedLogAnnotations {
+    /**
+     * Annotation to identify flaky tests in DistributedLog.
+     * As and when we find that a test is flaky, we'll add this annotation to it for reference.
+     */
+    public @interface FlakyTest {}
+
+    /**
+     * Annotation to specify the occurrence of a compression operation. These are CPU intensive
+     * and should be avoided in low-latency paths.
+     */
+    public @interface Compression {}
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/annotations/package-info.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/annotations/package-info.java
new file mode 100644
index 0000000..0922f14
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/annotations/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Defines annotations used across distributedlog project.
+ */
+package org.apache.distributedlog.annotations;
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyClosedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyClosedException.java
new file mode 100644
index 0000000..ecb9180
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyClosedException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Thrown when any distributedlog resources have already been closed.
+ *
+ * <p>For example, it might be thrown when the connections have been torn down.
+ */
+public class AlreadyClosedException extends DLException {
+
+    private static final long serialVersionUID = -4721864322739563725L;
+
+    public AlreadyClosedException(String message) {
+        super(StatusCode.ALREADY_CLOSED, message);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyTruncatedTransactionException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyTruncatedTransactionException.java
new file mode 100644
index 0000000..9f8d5bc
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/AlreadyTruncatedTransactionException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Thrown when the transaction Id specified in the API is in the range that has already been truncated.
+ */
+public class AlreadyTruncatedTransactionException extends DLException {
+
+    private static final long serialVersionUID = 4287238797065959977L;
+
+    public AlreadyTruncatedTransactionException(String message) {
+        super(StatusCode.TRUNCATED_TRANSACTION, message);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/BKTransmitException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/BKTransmitException.java
new file mode 100644
index 0000000..7d0a5b3
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/BKTransmitException.java
@@ -0,0 +1,42 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Thrown when the send to bookkeeper fails.
+ *
+ * <p>This is thrown by the next attempt to write, send or flush
+ */
+public class BKTransmitException extends DLException {
+
+    private static final long serialVersionUID = -5796100450432076091L;
+
+    final int bkRc;
+
+    public BKTransmitException(String message, int bkRc) {
+        super(StatusCode.BK_TRANSMIT_ERROR, message + " : " + bkRc);
+        this.bkRc = bkRc;
+    }
+
+    public int getBKResultCode() {
+        return this.bkRc;
+    }
+
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ChecksumFailedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ChecksumFailedException.java
new file mode 100644
index 0000000..36c7bd9
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ChecksumFailedException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception thrown when checksum failures occurred.
+ */
+public class ChecksumFailedException extends DLException {
+
+    private static final long serialVersionUID = 288438128880378812L;
+
+    public ChecksumFailedException() {
+        super(StatusCode.CHECKSUM_FAILED, "Checksum failed");
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLClientClosedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLClientClosedException.java
new file mode 100644
index 0000000..a5e4faa
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLClientClosedException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exceptions thrown when a distributedlog client is closed.
+ */
+public class DLClientClosedException extends DLException {
+
+    private static final long serialVersionUID = -8876218750540927584L;
+
+    public DLClientClosedException(String msg) {
+        super(StatusCode.CLIENT_CLOSED, msg);
+    }
+
+    public DLClientClosedException(String msg, Throwable t) {
+        super(StatusCode.CLIENT_CLOSED, msg, t);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLException.java
new file mode 100644
index 0000000..e974d46
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLException.java
@@ -0,0 +1,81 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import java.io.IOException;
+
+/**
+ * The basic exception of all distributedlog exceptions.
+ */
+public class DLException extends IOException {
+    private static final long serialVersionUID = -4485775468586114393L;
+    protected final StatusCode code;
+
+    protected DLException(StatusCode code) {
+        super();
+        this.code = code;
+    }
+
+    protected DLException(StatusCode code, String msg) {
+        super(msg);
+        this.code = code;
+    }
+
+    protected DLException(StatusCode code, Throwable t) {
+        super(t);
+        this.code = code;
+    }
+
+    protected DLException(StatusCode code, String msg, Throwable t) {
+        super(msg, t);
+        this.code = code;
+    }
+
+    /**
+     * Return the status code representing the exception.
+     *
+     * @return status code representing the exception.
+     */
+    public StatusCode getCode() {
+        return code;
+    }
+
+    public static DLException of(ResponseHeader response) {
+        String errMsg;
+        switch (response.getCode()) {
+            case FOUND:
+                if (response.isSetErrMsg()) {
+                    errMsg = response.getErrMsg();
+                } else {
+                    errMsg = "Request is redirected to " + response.getLocation();
+                }
+                return new OwnershipAcquireFailedException(errMsg, response.getLocation());
+            case SUCCESS:
+                throw new IllegalArgumentException("Can't instantiate an exception for success response.");
+            default:
+                if (response.isSetErrMsg()) {
+                    errMsg = response.getErrMsg();
+                } else {
+                    errMsg = response.getCode().name();
+                }
+                return new DLException(response.getCode(), errMsg);
+        }
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLIllegalStateException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLIllegalStateException.java
new file mode 100644
index 0000000..7aa8c39
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLIllegalStateException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Signals that a method has been invoked at an illegal or inappropriate time.
+ */
+public class DLIllegalStateException extends DLException {
+
+    private static final long serialVersionUID = -6721471104777747420L;
+
+    public DLIllegalStateException(String msg) {
+        super(StatusCode.ILLEGAL_STATE, msg);
+    }
+
+    public DLIllegalStateException(String msg, Throwable t) {
+        super(StatusCode.ILLEGAL_STATE, msg, t);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLInterruptedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLInterruptedException.java
new file mode 100644
index 0000000..6eaf8ac
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/DLInterruptedException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * An interrupted exception wrapper indicates dl operations are interrupted.
+ */
+public class DLInterruptedException extends DLException {
+
+    private static final long serialVersionUID = 3023394251107286496L;
+
+    public DLInterruptedException(String msg) {
+        super(StatusCode.INTERRUPTED, msg);
+    }
+
+    public DLInterruptedException(String msg, Throwable t) {
+        super(StatusCode.INTERRUPTED, msg, t);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfLogSegmentException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfLogSegmentException.java
new file mode 100644
index 0000000..356dde8
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfLogSegmentException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception thrown when reach end of the log segment.
+ */
+public class EndOfLogSegmentException extends DLException {
+
+    private static final long serialVersionUID = 6060419315910178451L;
+
+    public EndOfLogSegmentException(String logSegmentName) {
+        super(StatusCode.END_OF_LOG_SEGMENT, "end of log segment " + logSegmentName);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfStreamException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfStreamException.java
new file mode 100644
index 0000000..4b8a931
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/EndOfStreamException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception thrown when a reader reaches end of a sealed log stream.
+ */
+public class EndOfStreamException extends DLException {
+
+    private static final long serialVersionUID = -6398949401860680263L;
+
+    public EndOfStreamException(String message) {
+        super(StatusCode.END_OF_STREAM, message);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/FlushException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/FlushException.java
new file mode 100644
index 0000000..7225970
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/FlushException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception indicates that errors occurred on flushing data.
+ */
+public class FlushException extends DLException {
+
+    private final long lastTxIdWritten;
+    private final long lastTxIdAcknowledged;
+
+    private static final long serialVersionUID = -9060360360261130489L;
+
+    public FlushException(String message, long lastTxIdWritten, long lastTxIdAcknowledged) {
+        super(StatusCode.FLUSH_TIMEOUT, message);
+        this.lastTxIdWritten = lastTxIdWritten;
+        this.lastTxIdAcknowledged = lastTxIdAcknowledged;
+    }
+
+    public FlushException(String message, long lastTxIdWritten, long lastTxIdAcknowledged, Throwable cause) {
+        super(StatusCode.FLUSH_TIMEOUT, message, cause);
+        this.lastTxIdWritten = lastTxIdWritten;
+        this.lastTxIdAcknowledged = lastTxIdAcknowledged;
+    }
+
+    public long getLastTxIdWritten() {
+        return lastTxIdWritten;
+    }
+
+    public long getLastTxIdAcknowledged() {
+        return lastTxIdAcknowledged;
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/IdleReaderException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/IdleReaderException.java
new file mode 100644
index 0000000..048a763
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/IdleReaderException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.exceptions;
+
+import java.io.IOException;
+
+/**
+ * An exception is thrown when a log reader becomes idle.
+ *
+ * <p>A log reader can become idle when there is no records written to the stream.
+ */
+public class IdleReaderException extends IOException {
+
+    private static final long serialVersionUID = 4882278671396434834L;
+
+    public IdleReaderException(String message) {
+        super(message);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InternalServerException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InternalServerException.java
new file mode 100644
index 0000000..9032c2b
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InternalServerException.java
@@ -0,0 +1,40 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception indicates that there is an internal error at distributedlog service side.
+ */
+public class InternalServerException extends DLException {
+
+    private static final long serialVersionUID = 288438028880978802L;
+
+    public InternalServerException(String msg) {
+        super(StatusCode.INTERNAL_SERVER_ERROR, msg);
+    }
+
+    public InternalServerException(Throwable t) {
+        super(StatusCode.INTERNAL_SERVER_ERROR, t);
+    }
+
+    public InternalServerException(String msg, Throwable t) {
+        super(StatusCode.INTERNAL_SERVER_ERROR, msg, t);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidEnvelopedEntryException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidEnvelopedEntryException.java
new file mode 100644
index 0000000..d46d14e
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidEnvelopedEntryException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception thrown when encounter invalid enveloped entry.
+ */
+public class InvalidEnvelopedEntryException extends DLException {
+
+    private static final long serialVersionUID = -9190621788978573862L;
+
+    public InvalidEnvelopedEntryException(String msg) {
+        super(StatusCode.INVALID_ENVELOPED_ENTRY, msg);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidStreamNameException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidStreamNameException.java
new file mode 100644
index 0000000..29e2663
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/InvalidStreamNameException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception is thrown when encountered invalid log stream.
+ */
+public class InvalidStreamNameException extends DLException {
+
+    private static final long serialVersionUID = 6393315766140568100L;
+
+    public InvalidStreamNameException(String streamName) {
+        super(StatusCode.INVALID_STREAM_NAME, "Invalid stream name : '" + streamName + "'");
+    }
+
+    public InvalidStreamNameException(String streamName, String reason) {
+        super(StatusCode.INVALID_STREAM_NAME, "Invalid stream name : '" + streamName + "' : " + reason);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LockCancelledException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LockCancelledException.java
new file mode 100644
index 0000000..7f66232
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LockCancelledException.java
@@ -0,0 +1,31 @@
+/**
+ * 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.exceptions;
+
+/**
+ * A lock is cancelled when waiting for locking a stream.
+ */
+public class LockCancelledException extends LockingException {
+
+    private static final long serialVersionUID = -148795017092861106L;
+
+    public LockCancelledException(String lockPath, String message, Throwable cause) {
+        super(lockPath, message, cause);
+    }
+
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LockingException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LockingException.java
new file mode 100644
index 0000000..63f6b0f
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LockingException.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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception when a log writer attempts to acquire a lock to write data to the stream.
+ */
+public class LockingException extends DLException {
+
+    private static final long serialVersionUID = -4960278188448464473L;
+
+    public LockingException(String lockPath, String message) {
+        this(StatusCode.LOCKING_EXCEPTION, lockPath, message);
+    }
+
+    public LockingException(String lockPath, String message, Throwable cause) {
+        this(StatusCode.LOCKING_EXCEPTION, lockPath, message, cause);
+    }
+
+    protected LockingException(StatusCode code, String lockPath, String message) {
+        super(code, String.format("LockPath - %s: %s", lockPath, message));
+    }
+
+    protected LockingException(StatusCode code, String lockPath, String message, Throwable cause) {
+        super(code, String.format("LockPath - %s: %s", lockPath, message), cause);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogEmptyException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogEmptyException.java
new file mode 100644
index 0000000..ba3545d
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogEmptyException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exceptions are thrown when attempt to read a log stream that doesn't have any records.
+ */
+public class LogEmptyException extends DLException {
+
+    private static final long serialVersionUID = -1106184127178002282L;
+
+    public LogEmptyException(String message) {
+        super(StatusCode.LOG_EMPTY, message);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogExistsException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogExistsException.java
new file mode 100644
index 0000000..3ecb80f
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogExistsException.java
@@ -0,0 +1,31 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception thrown on creating a log stream but the log stream already exists.
+ */
+public class LogExistsException extends DLException {
+    private static final long serialVersionUID = 1794053581673506784L;
+
+    public LogExistsException(String msg) {
+        super(StatusCode.LOG_EXISTS, msg);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogNotFoundException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogNotFoundException.java
new file mode 100644
index 0000000..3795c5a
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogNotFoundException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception is thrown when a reader attempts to read a log stream that doesn't exist.
+ */
+public class LogNotFoundException extends DLException {
+
+    private static final long serialVersionUID = 871435700699403164L;
+
+    public LogNotFoundException(String message) {
+        super(StatusCode.LOG_NOT_FOUND, message);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogReadException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogReadException.java
new file mode 100644
index 0000000..91cf5c9
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogReadException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Thrown when there's a failure to read an edit log op from disk when loading
+ * edits.
+ */
+public class LogReadException extends DLException {
+
+    private static final long serialVersionUID = 302945709106128177L;
+
+    public LogReadException(String message) {
+        super(StatusCode.LOG_READ_ERROR, message);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogRecordTooLongException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogRecordTooLongException.java
new file mode 100644
index 0000000..2cb9085
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogRecordTooLongException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception is thrown when attempting to write a record whose size is too larger.
+ *
+ * <p>The size limit of a log record is {@link org.apache.distributedlog.LogRecord#MAX_LOGRECORD_SIZE}.
+ */
+public class LogRecordTooLongException extends DLException {
+
+    private static final long serialVersionUID = 2788274084603111386L;
+
+    public LogRecordTooLongException(String message) {
+        super(StatusCode.TOO_LARGE_RECORD, message);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentIsTruncatedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentIsTruncatedException.java
new file mode 100644
index 0000000..ac2ebda
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentIsTruncatedException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception is thrown when reading data from a truncated log segment.
+ */
+public class LogSegmentIsTruncatedException extends DLException {
+
+    private static final long serialVersionUID = -218506870918498062L;
+
+    public LogSegmentIsTruncatedException(String logSegmentName) {
+        super(StatusCode.LOG_SEGMENT_IS_TRUNCATED, "Log Segment '"
+                + logSegmentName + "'" + " is already truncated");
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentNotFoundException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentNotFoundException.java
new file mode 100644
index 0000000..5245cdc
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/LogSegmentNotFoundException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception on log segment not found.
+ */
+public class LogSegmentNotFoundException extends DLException {
+
+    private static final long serialVersionUID = -2482324226595903864L;
+
+    public LogSegmentNotFoundException(String logSegmentPath) {
+        super(StatusCode.LOG_SEGMENT_NOT_FOUND, "Log Segment " + logSegmentPath + " not found");
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/MetadataException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/MetadataException.java
new file mode 100644
index 0000000..6dba778
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/MetadataException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception is thrown when encountering metadata errors.
+ */
+public class MetadataException extends DLException {
+
+    private static final long serialVersionUID = 6683578078046016125L;
+
+    public MetadataException(String msg) {
+        super(StatusCode.METADATA_EXCEPTION, msg);
+    }
+
+    public MetadataException(String msg, Throwable t) {
+        super(StatusCode.METADATA_EXCEPTION, msg, t);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/NotYetImplementedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/NotYetImplementedException.java
new file mode 100644
index 0000000..eb709de
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/NotYetImplementedException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception is thrown when a method is not implemented yet.
+ */
+public class NotYetImplementedException extends DLException {
+
+    private static final long serialVersionUID = -6002036746792556106L;
+
+    public NotYetImplementedException(String method) {
+        super(StatusCode.NOT_IMPLEMENTED, method + "is not supported by the current version");
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OverCapacityException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OverCapacityException.java
new file mode 100644
index 0000000..6dc4767
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OverCapacityException.java
@@ -0,0 +1,39 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception is thrown when the system is over capacity.
+ *
+ * <p>When this exception is received, the client (either writer or reader) should take actions to not
+ * overwhelm the system. For example, backoff sending any requests.
+ */
+public class OverCapacityException extends DLException {
+
+    private static final long serialVersionUID = -6398949404860680263L;
+
+    public OverCapacityException(String message) {
+        super(StatusCode.OVER_CAPACITY, message);
+    }
+
+    public OverCapacityException(StatusCode code, String message) {
+        super(code, message);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OwnershipAcquireFailedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OwnershipAcquireFailedException.java
new file mode 100644
index 0000000..af85e79
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/OwnershipAcquireFailedException.java
@@ -0,0 +1,40 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception is thrown when a log writer attempt to acquire a lock.
+ *
+ * <p>It is typically thrown when the lock is already acquired by another writer.
+ */
+public class OwnershipAcquireFailedException extends LockingException {
+    private static final long serialVersionUID = 8176056926552748001L;
+    private final String currentOwner;
+
+    public OwnershipAcquireFailedException(String lockPath, String currentOwner) {
+        super(StatusCode.FOUND, lockPath,
+              String.format("Lock acquisition failed, the current owner is %s", currentOwner));
+        this.currentOwner = currentOwner;
+    }
+
+    public String getCurrentOwner() {
+        return currentOwner;
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ReadCancelledException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ReadCancelledException.java
new file mode 100644
index 0000000..ac0508a
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ReadCancelledException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Signals that a read request has been cancelled.
+ */
+public class ReadCancelledException extends DLException {
+
+    private static final long serialVersionUID = -6273430297547510262L;
+
+    public ReadCancelledException(String stream, String reason) {
+        super(StatusCode.READ_CANCELLED_EXCEPTION,
+              "Read cancelled on stream " + stream + " : " + reason);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RegionUnavailableException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RegionUnavailableException.java
new file mode 100644
index 0000000..7a4b225
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RegionUnavailableException.java
@@ -0,0 +1,34 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception indicates that the service is not available in one region.
+ *
+ * <p>The client should retry the other regions when encountering this exception.
+ */
+public class RegionUnavailableException extends DLException {
+
+    private static final long serialVersionUID = 5727337162533143957L;
+
+    public RegionUnavailableException(String msg) {
+        super(StatusCode.REGION_UNAVAILABLE, msg);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RequestDeniedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RequestDeniedException.java
new file mode 100644
index 0000000..ab730f3
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RequestDeniedException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Signals that a request has been denied at the server.
+ */
+public class RequestDeniedException extends DLException {
+
+    private static final long serialVersionUID = 7338220414584728216L;
+
+    public RequestDeniedException(String stream, String operation) {
+        super(StatusCode.REQUEST_DENIED,
+                operation + " request to stream " + stream + " is denied");
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RetryableReadException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RetryableReadException.java
new file mode 100644
index 0000000..10a1e53
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/RetryableReadException.java
@@ -0,0 +1,36 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Signals that a read request can be retried..
+ */
+public class RetryableReadException extends DLException {
+
+    private static final long serialVersionUID = 2803207702150642330L;
+
+    public RetryableReadException (String streamName, String message) {
+        super(StatusCode.RETRYABLE_READ, String.format("Reader on %s failed with %s", streamName, message));
+    }
+
+    public RetryableReadException (String streamName, String message, Throwable cause) {
+        super(StatusCode.RETRYABLE_READ, String.format("Reader on %s failed with %s", streamName, message), cause);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ServiceUnavailableException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ServiceUnavailableException.java
new file mode 100644
index 0000000..ca266c2
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/ServiceUnavailableException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception indicates that the service is unavailable at the server side.
+ */
+public class ServiceUnavailableException extends DLException {
+
+    private static final long serialVersionUID = 6317900286881665746L;
+
+    public ServiceUnavailableException(String msg) {
+        super(StatusCode.SERVICE_UNAVAILABLE, msg);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamNotReadyException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamNotReadyException.java
new file mode 100644
index 0000000..ded9da1
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamNotReadyException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception is thrown when a log stream is not ready on server side for serving the write requests.
+ */
+public class StreamNotReadyException extends DLException {
+
+    private static final long serialVersionUID = 684211282036293028L;
+
+    public StreamNotReadyException(String msg) {
+        super(StatusCode.STREAM_NOT_READY, msg);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamUnavailableException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamUnavailableException.java
new file mode 100644
index 0000000..fd0910e
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/StreamUnavailableException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception thrown when a stream is not available for serving traffic.
+ */
+public class StreamUnavailableException extends DLException {
+
+    private static final long serialVersionUID = 684211282036993028L;
+
+    public StreamUnavailableException(String msg) {
+        super(StatusCode.STREAM_UNAVAILABLE, msg);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TooManyStreamsException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TooManyStreamsException.java
new file mode 100644
index 0000000..17e2e38
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TooManyStreamsException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Signals that a server has been serving too many streams.
+ */
+public class TooManyStreamsException extends OverCapacityException {
+
+    private static final long serialVersionUID = -6391941401860180163L;
+
+    public TooManyStreamsException(String message) {
+        super(StatusCode.TOO_MANY_STREAMS, message);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TransactionIdOutOfOrderException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TransactionIdOutOfOrderException.java
new file mode 100644
index 0000000..e16fcab
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/TransactionIdOutOfOrderException.java
@@ -0,0 +1,47 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * An exception is thrown when a log writer attempts to write a record with out-of-order transaction id.
+ */
+public class TransactionIdOutOfOrderException extends DLException {
+
+    private static final long serialVersionUID = -6239322552103630036L;
+    // TODO: copied from DistributedLogConstants (we should think about how to separated common constants)
+    public static final long INVALID_TXID = -999;
+    private final long lastTxnId;
+
+    public TransactionIdOutOfOrderException(long smallerTxnId, long lastTxnId) {
+        super(StatusCode.TRANSACTION_OUT_OF_ORDER,
+              "Received smaller txn id " + smallerTxnId + ", last txn id is " + lastTxnId);
+        this.lastTxnId = lastTxnId;
+    }
+
+    public TransactionIdOutOfOrderException(long invalidTxnId) {
+        super(StatusCode.TRANSACTION_OUT_OF_ORDER,
+            "The txn id " + invalidTxnId + " is invalid and will break the sequence");
+        lastTxnId = INVALID_TXID;
+    }
+
+    public long getLastTxnId() {
+        return lastTxnId;
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnexpectedException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnexpectedException.java
new file mode 100644
index 0000000..637886e
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnexpectedException.java
@@ -0,0 +1,40 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * An {@code UnexpectedException} is thrown when encountering unexpected conditions.
+ */
+public class UnexpectedException extends DLException {
+
+    private static final long serialVersionUID = 903763128422774055L;
+
+    public UnexpectedException() {
+        super(StatusCode.UNEXPECTED);
+    }
+
+    public UnexpectedException(String msg) {
+        super(StatusCode.UNEXPECTED, msg);
+    }
+
+    public UnexpectedException(String msg, Throwable t) {
+        super(StatusCode.UNEXPECTED, msg, t);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnsupportedMetadataVersionException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnsupportedMetadataVersionException.java
new file mode 100644
index 0000000..01fab89
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/UnsupportedMetadataVersionException.java
@@ -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.
+ */
+package org.apache.distributedlog.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Exception is thrown when found unsupported metadata version.
+ */
+public class UnsupportedMetadataVersionException extends DLException {
+
+    private static final long serialVersionUID = 4980892659955478446L;
+
+    public UnsupportedMetadataVersionException(String message) {
+        super(StatusCode.UNSUPPORTED_METADATA_VERSION, String.format(message));
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteCancelledException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteCancelledException.java
new file mode 100644
index 0000000..d9001dd
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteCancelledException.java
@@ -0,0 +1,45 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * Signals when a write request is cancelled.
+ */
+public class WriteCancelledException extends DLException {
+
+    private static final long serialVersionUID = -1836146493496072122L;
+
+    public WriteCancelledException(String stream, Throwable t) {
+        super(StatusCode.WRITE_CANCELLED_EXCEPTION,
+            "Write cancelled on stream "
+                + stream + " due to an earlier error", t);
+    }
+
+    public WriteCancelledException(String stream, String reason) {
+        super(StatusCode.WRITE_CANCELLED_EXCEPTION,
+                "Write cancelled on stream " + stream + " due to : " + reason);
+    }
+
+    public WriteCancelledException(String stream) {
+        super(StatusCode.WRITE_CANCELLED_EXCEPTION,
+            "Write cancelled on stream "
+                + stream + " due to an earlier error");
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteException.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteException.java
new file mode 100644
index 0000000..6899dbf
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/WriteException.java
@@ -0,0 +1,33 @@
+/**
+ * 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.exceptions;
+
+import org.apache.distributedlog.thrift.service.StatusCode;
+
+/**
+ * An exception on writing log records.
+ */
+public class WriteException extends DLException {
+
+    private static final long serialVersionUID = -1836146493446072122L;
+
+    public WriteException(String stream, String transmitError) {
+        super(StatusCode.WRITE_EXCEPTION,
+            "Write rejected because stream " + stream + " has encountered an error : " + transmitError);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/package-info.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/package-info.java
new file mode 100644
index 0000000..1ac2bca
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/exceptions/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Defines exceptions used across distributedlog project.
+ */
+package org.apache.distributedlog.exceptions;
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/Buffer.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/Buffer.java
new file mode 100644
index 0000000..16718c5
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/Buffer.java
@@ -0,0 +1,33 @@
+/**
+ * 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.io;
+
+import java.io.ByteArrayOutputStream;
+
+/**
+ * {@link ByteArrayOutputStream} based buffer.
+ */
+public class Buffer extends ByteArrayOutputStream {
+    public Buffer(int initialCapacity) {
+        super(initialCapacity);
+    }
+
+    public byte[] getData() {
+        return buf;
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/CompressionCodec.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/CompressionCodec.java
new file mode 100644
index 0000000..9a0e3a3
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/CompressionCodec.java
@@ -0,0 +1,80 @@
+/**
+ * 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.io;
+
+import org.apache.bookkeeper.stats.OpStatsLogger;
+
+/**
+ * Common interface for compression/decompression operations using different compression codecs.
+ */
+public interface CompressionCodec {
+    /**
+     * Enum specifying the currently supported compression types.
+     */
+    enum Type {
+        NONE, LZ4, UNKNOWN
+    }
+
+    /**
+     * Return the compressed data as a byte array.
+     * @param data
+     *          The data to be compressed
+     * @param offset
+     *          The offset in the bytes of data to compress
+     * @param length
+     *          The number of bytes of data to compress
+     * @param compressionStat
+     *          The stat to use for timing the compression operation
+     * @return
+     *          The compressed data
+     *          The returned byte array is sized to the length of the compressed data
+     */
+    byte[] compress(byte[] data, int offset, int length, OpStatsLogger compressionStat);
+
+    /**
+     * Return the decompressed data as a byte array.
+     * @param data
+     *          The data to be decompressed
+     * @param offset
+     *          The offset in the bytes of data to decompress
+     * @param length
+     *          The number of bytes of data to decompress
+     * @param decompressionStat
+     *          The stat to use for timing the decompression operation
+     * @return
+     *          The decompressed data
+     */
+    byte[] decompress(byte[] data, int offset, int length, OpStatsLogger decompressionStat);
+
+    /**
+     * Return the decompressed data as a byte array.
+     * @param data
+     *          The data to the decompressed
+     * @param offset
+     *          The offset in the bytes of data to decompress
+     * @param length
+     *          The number of bytes of data to decompress
+     * @param decompressedSize
+     *          The exact size of the decompressed data
+     * @param decompressionStat
+     *          The stat to use for timing the decompression operation
+     * @return
+     *          The decompressed data
+     */
+    byte[] decompress(byte[] data, int offset, int length, int decompressedSize, OpStatsLogger decompressionStat);
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/CompressionUtils.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/CompressionUtils.java
new file mode 100644
index 0000000..7bac92a
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/CompressionUtils.java
@@ -0,0 +1,58 @@
+/**
+ * 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.io;
+
+/**
+ * Utils for compression related operations.
+ */
+public class CompressionUtils {
+
+    public static final String LZ4 = "lz4";
+    public static final String NONE = "none";
+
+    private static final CompressionCodec IDENTITY_CODEC = new IdentityCompressionCodec();
+    private static final CompressionCodec LZ4_CODEC = new LZ4CompressionCodec();
+
+    /**
+     * Get a cached compression codec instance for the specified type.
+     * @param type compression codec type
+     * @return compression codec instance
+     */
+    public static CompressionCodec getCompressionCodec(CompressionCodec.Type type) {
+        if (type == CompressionCodec.Type.LZ4) {
+            return LZ4_CODEC;
+        }
+        // No Compression
+        return IDENTITY_CODEC;
+    }
+
+    /**
+     * Compression type value from string.
+     * @param compressionString compression codec presentation in string
+     * @return compression codec type
+     */
+    public static CompressionCodec.Type stringToType(String compressionString) {
+        if (compressionString.equals(LZ4)) {
+            return CompressionCodec.Type.LZ4;
+        } else if (compressionString.equals(NONE)) {
+            return CompressionCodec.Type.NONE;
+        } else {
+            return CompressionCodec.Type.UNKNOWN;
+        }
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/IdentityCompressionCodec.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/IdentityCompressionCodec.java
new file mode 100644
index 0000000..5034033
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/IdentityCompressionCodec.java
@@ -0,0 +1,50 @@
+/**
+ * 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.io;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import java.util.Arrays;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+
+/**
+ * An identity compression codec implementation for {@link CompressionCodec}.
+ */
+public class IdentityCompressionCodec implements CompressionCodec {
+    @Override
+    public byte[] compress(byte[] data, int offset, int length, OpStatsLogger compressionStat) {
+        checkNotNull(data);
+        checkArgument(length >= 0);
+        return Arrays.copyOfRange(data, offset, offset + length);
+    }
+
+    @Override
+    public byte[] decompress(byte[] data, int offset, int length, OpStatsLogger decompressionStat) {
+        checkNotNull(data);
+        return Arrays.copyOfRange(data, offset, offset + length);
+    }
+
+    @Override
+    // Decompressed size is the same as the length of the data because this is an
+    // Identity compressor
+    public byte[] decompress(byte[] data, int offset, int length,
+                             int decompressedSize, OpStatsLogger decompressionStat) {
+        return decompress(data, offset, length, decompressionStat);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/LZ4CompressionCodec.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/LZ4CompressionCodec.java
new file mode 100644
index 0000000..38d3ba3
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/LZ4CompressionCodec.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.io;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import com.google.common.base.Stopwatch;
+import java.util.concurrent.TimeUnit;
+import net.jpountz.lz4.LZ4Compressor;
+import net.jpountz.lz4.LZ4Exception;
+import net.jpountz.lz4.LZ4Factory;
+import net.jpountz.lz4.LZ4FastDecompressor;
+import net.jpountz.lz4.LZ4SafeDecompressor;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+
+/**
+ * An {@code lz4} based {@link CompressionCodec} implementation.
+ *
+ * <p>All functions are thread safe.
+ */
+public class LZ4CompressionCodec implements CompressionCodec {
+
+    // Used for compression
+    private final LZ4Compressor compressor;
+    // Used to decompress when the size of the output is known
+    private final LZ4FastDecompressor fastDecompressor;
+    // Used to decompress when the size of the output is not known
+    private final LZ4SafeDecompressor safeDecompressor;
+
+    public LZ4CompressionCodec() {
+        this.compressor = LZ4Factory.fastestInstance().fastCompressor();
+        this.fastDecompressor = LZ4Factory.fastestInstance().fastDecompressor();
+        this.safeDecompressor = LZ4Factory.fastestInstance().safeDecompressor();
+    }
+
+    @Override
+    public byte[] compress(byte[] data, int offset, int length, OpStatsLogger compressionStat) {
+        checkNotNull(data);
+        checkArgument(offset >= 0 && offset < data.length);
+        checkArgument(length >= 0);
+        checkNotNull(compressionStat);
+
+        Stopwatch watch = Stopwatch.createStarted();
+        byte[] compressed = compressor.compress(data, offset, length);
+        compressionStat.registerSuccessfulEvent(watch.elapsed(TimeUnit.MICROSECONDS));
+        return compressed;
+    }
+
+    @Override
+    public byte[] decompress(byte[] data, int offset, int length, OpStatsLogger decompressionStat) {
+        checkNotNull(data);
+        checkArgument(offset >= 0 && offset < data.length);
+        checkArgument(length >= 0);
+        checkNotNull(decompressionStat);
+
+        Stopwatch watch = Stopwatch.createStarted();
+        // Assume that we have a compression ratio of 1/3.
+        int outLength = length * 3;
+        while (true) {
+            try {
+                byte[] decompressed = safeDecompressor.decompress(data, offset, length, outLength);
+                decompressionStat.registerSuccessfulEvent(watch.elapsed(TimeUnit.MICROSECONDS));
+                return decompressed;
+            } catch (LZ4Exception e) {
+                outLength *= 2;
+            }
+        }
+    }
+
+    @Override
+    // length parameter is ignored here because of the way the fastDecompressor works.
+    public byte[] decompress(byte[] data, int offset, int length, int decompressedSize,
+                             OpStatsLogger decompressionStat) {
+        checkNotNull(data);
+        checkArgument(offset >= 0 && offset < data.length);
+        checkArgument(length >= 0);
+        checkArgument(decompressedSize >= 0);
+        checkNotNull(decompressionStat);
+
+        Stopwatch watch = Stopwatch.createStarted();
+        byte[] decompressed = fastDecompressor.decompress(data, offset, decompressedSize);
+        decompressionStat.registerSuccessfulEvent(watch.elapsed(TimeUnit.MICROSECONDS));
+        return decompressed;
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/TransmitListener.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/TransmitListener.java
new file mode 100644
index 0000000..a8bc112
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/TransmitListener.java
@@ -0,0 +1,56 @@
+/**
+ * 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.io;
+
+import org.apache.distributedlog.DLSN;
+
+/**
+ * Listener on transmit results.
+ */
+public interface TransmitListener {
+
+    /**
+     * Finalize the transmit result and result the last
+     * {@link DLSN} in this transmit.
+     *
+     * @param lssn
+     *          log segment sequence number
+     * @param entryId
+     *          entry id
+     * @return last dlsn in this transmit
+     */
+    DLSN finalizeTransmit(long lssn, long entryId);
+
+    /**
+     * Complete the whole transmit.
+     *
+     * @param lssn
+     *          log segment sequence number
+     * @param entryId
+     *          entry id
+     */
+    void completeTransmit(long lssn, long entryId);
+
+    /**
+     * Abort the transmit.
+     *
+     * @param reason
+     *          reason to abort transmit
+     */
+    void abortTransmit(Throwable reason);
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/package-info.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/package-info.java
new file mode 100644
index 0000000..740359b
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/io/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * I/O related utilities used across the project.
+ */
+package org.apache.distributedlog.io;
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/package-info.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/package-info.java
new file mode 100644
index 0000000..41ddd82
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Provides the core data structures used across multiple modules.
+ */
+package org.apache.distributedlog;
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/BitMaskUtils.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/BitMaskUtils.java
new file mode 100644
index 0000000..b5280c9
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/BitMaskUtils.java
@@ -0,0 +1,57 @@
+/**
+ * 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 static com.google.common.base.Preconditions.checkArgument;
+
+/**
+ * Utils for bit mask operations.
+ */
+public class BitMaskUtils {
+
+    /**
+     * 1) Unset all bits where value in mask is set.
+     * 2) Set these bits to value specified by newValue.
+     *
+     * <p>e.g.
+     * if oldValue = 1010, mask = 0011, newValue = 0001
+     * 1) 1010 -> 1000
+     * 2) 1000 -> 1001
+     *
+     * @param oldValue expected old value
+     * @param mask the mask of the value for updates
+     * @param newValue new value to set
+     * @return updated value
+     */
+    public static long set(long oldValue, long mask, long newValue) {
+        checkArgument(oldValue >= 0L && mask >= 0L && newValue >= 0L);
+        return ((oldValue & (~mask)) | (newValue & mask));
+    }
+
+    /**
+     * Get the bits where mask is 1.
+     *
+     * @param value value
+     * @param mask mask of the value
+     * @return the bit of the mask
+     */
+    public static long get(long value, long mask) {
+        checkArgument(value >= 0L && mask >= 0L);
+        return (value & mask);
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/ProtocolUtils.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/ProtocolUtils.java
new file mode 100644
index 0000000..0a37d3a
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/ProtocolUtils.java
@@ -0,0 +1,79 @@
+/**
+ * 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 static com.google.common.base.Charsets.UTF_8;
+
+import org.apache.distributedlog.DLSN;
+import java.util.zip.CRC32;
+
+/**
+ * With CRC embedded in the application, we have to keep track of per api crc. Ideally this
+ * would be done by thrift.
+ */
+public class ProtocolUtils {
+
+    // For request payload checksum
+    private static final ThreadLocal<CRC32> requestCRC = new ThreadLocal<CRC32>() {
+        @Override
+        protected CRC32 initialValue() {
+            return new CRC32();
+        }
+    };
+
+    /**
+     * Generate crc32 for WriteOp.
+     */
+    public static Long writeOpCRC32(String stream, byte[] payload) {
+        CRC32 crc = requestCRC.get();
+        try {
+            crc.update(stream.getBytes(UTF_8));
+            crc.update(payload);
+            return crc.getValue();
+        } finally {
+            crc.reset();
+        }
+    }
+
+    /**
+     * Generate crc32 for TruncateOp.
+     */
+    public static Long truncateOpCRC32(String stream, DLSN dlsn) {
+        CRC32 crc = requestCRC.get();
+        try {
+            crc.update(stream.getBytes(UTF_8));
+            crc.update(dlsn.serializeBytes());
+            return crc.getValue();
+        } finally {
+            crc.reset();
+        }
+    }
+
+    /**
+     * Generate crc32 for any op which only passes a stream name.
+     */
+    public static Long streamOpCRC32(String stream) {
+        CRC32 crc = requestCRC.get();
+        try {
+            crc.update(stream.getBytes(UTF_8));
+            return crc.getValue();
+        } finally {
+            crc.reset();
+        }
+    }
+}
diff --git a/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/package-info.java b/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/package-info.java
new file mode 100644
index 0000000..ee17950
--- /dev/null
+++ b/distributedlog-protocol/src/main/java/org/apache/distributedlog/util/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * defines the utilities used across the project.
+ */
+package org.apache.distributedlog.util;
diff --git a/distributedlog-protocol/src/main/resources/findbugsExclude.xml b/distributedlog-protocol/src/main/resources/findbugsExclude.xml
index 6b2197b..55e50f6 100644
--- a/distributedlog-protocol/src/main/resources/findbugsExclude.xml
+++ b/distributedlog-protocol/src/main/resources/findbugsExclude.xml
@@ -18,22 +18,22 @@
 <FindBugsFilter>
   <Match>
     <!-- generated code, we can't be held responsible for findbugs in it //-->
-    <Class name="~com\.twitter\.distributedlog\.thrift.*" />
+    <Class name="~org\.apache\.distributedlog\.thrift.*" />
   </Match>
   <Match>
     <!-- it is safe to store external bytes reference here. //-->
-    <Class name="com.twitter.distributedlog.LogRecord" />
+    <Class name="org.apache.distributedlog.LogRecord" />
     <Bug pattern="EI_EXPOSE_REP2" />
   </Match>
   <Match>
     <!-- it is safe to store external bytes reference here. //-->
-    <Class name="com.twitter.distributedlog.LogRecord" />
+    <Class name="org.apache.distributedlog.LogRecord" />
     <Method name="getPayload" />
     <Bug pattern="EI_EXPOSE_REP" />
   </Match>
   <Match>
     <!-- it is safe to store external bytes reference here. //-->
-    <Class name="com.twitter.distributedlog.io.Buffer" />
+    <Class name="org.apache.distributedlog.io.Buffer" />
     <Method name="getData" />
     <Bug pattern="EI_EXPOSE_REP" />
   </Match>
diff --git a/distributedlog-protocol/src/main/thrift/service.thrift b/distributedlog-protocol/src/main/thrift/service.thrift
index c2f77f3..fa318e1 100644
--- a/distributedlog-protocol/src/main/thrift/service.thrift
+++ b/distributedlog-protocol/src/main/thrift/service.thrift
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-namespace java com.twitter.distributedlog.thrift.service
+namespace java org.apache.distributedlog.thrift.service
 
 /* Response stats codes */
 enum StatusCode {
diff --git a/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestDLSN.java b/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestDLSN.java
deleted file mode 100644
index 551d0ca..0000000
--- a/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestDLSN.java
+++ /dev/null
@@ -1,64 +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 com.twitter.distributedlog;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import org.junit.Test;
-
-/**
- * Test Case for {@link DLSN}.
- */
-public class TestDLSN {
-
-    @Test(timeout = 60000)
-    public void testDLSN() {
-        DLSN dlsn = new DLSN(99L, 88L, 77L);
-        String dlsnv0 = dlsn.serialize(DLSN.VERSION0);
-        String dlsnv1 = dlsn.serialize(DLSN.VERSION1);
-        String badDLSN = "baddlsn";
-
-        assertEquals(dlsn, DLSN.deserialize(dlsnv0));
-        assertEquals(dlsn, DLSN.deserialize(dlsnv1));
-        try {
-            DLSN.deserialize(badDLSN);
-            fail("Should fail on deserializing bad dlsn");
-        } catch (IllegalArgumentException iae) {
-        }
-
-        assertEquals(dlsn, DLSN.deserialize0(dlsnv0));
-        try {
-            DLSN.deserialize0(dlsnv1);
-            fail("Should fail on deserializing version one dlsn");
-        } catch (IllegalArgumentException iae) {
-        }
-        try {
-            DLSN.deserialize0(badDLSN);
-            fail("Should fail on deserializing bad dlsn");
-        } catch (IllegalArgumentException iae) {
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testSerializeDeserializeBytes() {
-        DLSN dlsn = new DLSN(99L, 88L, 77L);
-        byte[] data = dlsn.serializeBytes();
-        assertEquals(dlsn, DLSN.deserializeBytes(data));
-    }
-}
diff --git a/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestLogRecordSet.java b/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestLogRecordSet.java
deleted file mode 100644
index 4a5fdac..0000000
--- a/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestLogRecordSet.java
+++ /dev/null
@@ -1,178 +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 com.twitter.distributedlog;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-import static com.twitter.distributedlog.LogRecordSet.HEADER_LEN;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.fail;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.LogRecordSet.Reader;
-import com.twitter.distributedlog.LogRecordSet.Writer;
-import com.twitter.distributedlog.exceptions.LogRecordTooLongException;
-import com.twitter.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.junit.Test;
-
-/**
- * Test Case for {@link LogRecordSet}.
- */
-public class TestLogRecordSet {
-
-    @Test(timeout = 60000)
-    public void testEmptyRecordSet() throws Exception {
-        Writer writer = LogRecordSet.newWriter(1024, Type.NONE);
-        assertEquals("zero user bytes", HEADER_LEN, writer.getNumBytes());
-        assertEquals("zero records", 0, writer.getNumRecords());
-
-        ByteBuffer buffer = writer.getBuffer();
-        assertEquals("zero user bytes", HEADER_LEN, buffer.remaining());
-
-        byte[] data = new byte[buffer.remaining()];
-        buffer.get(data);
-
-        LogRecordWithDLSN record = new LogRecordWithDLSN(
-                new DLSN(1L, 0L, 0L),
-                1L,
-                data,
-                1L);
-        record.setRecordSet();
-        Reader reader = LogRecordSet.of(record);
-        assertNull("Empty record set should return null",
-                reader.nextRecord());
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteTooLongRecord() throws Exception {
-        Writer writer = LogRecordSet.newWriter(1024, Type.NONE);
-        assertEquals("zero user bytes", HEADER_LEN, writer.getNumBytes());
-        assertEquals("zero records", 0, writer.getNumRecords());
-
-        ByteBuffer dataBuf = ByteBuffer.allocate(MAX_LOGRECORD_SIZE + 1);
-        try {
-            writer.writeRecord(dataBuf, new Promise<DLSN>());
-            fail("Should fail on writing large record");
-        } catch (LogRecordTooLongException lrtle) {
-            // expected
-        }
-        assertEquals("zero user bytes", HEADER_LEN, writer.getNumBytes());
-        assertEquals("zero records", 0, writer.getNumRecords());
-        ByteBuffer buffer = writer.getBuffer();
-        assertEquals("zero user bytes", HEADER_LEN, buffer.remaining());
-
-        byte[] data = new byte[buffer.remaining()];
-        buffer.get(data);
-
-        LogRecordWithDLSN record = new LogRecordWithDLSN(
-                new DLSN(1L, 0L, 0L),
-                1L,
-                data,
-                1L);
-        record.setRecordSet();
-        Reader reader = LogRecordSet.of(record);
-        assertNull("Empty record set should return null",
-                reader.nextRecord());
-    }
-
-    @Test(timeout = 20000)
-    public void testWriteRecordsNoneCompressed() throws Exception {
-        testWriteRecords(Type.NONE);
-    }
-
-    @Test(timeout = 20000)
-    public void testWriteRecordsLZ4Compressed() throws Exception {
-        testWriteRecords(Type.LZ4);
-    }
-
-    void testWriteRecords(Type codec) throws Exception {
-        Writer writer = LogRecordSet.newWriter(1024, codec);
-        assertEquals("zero user bytes", HEADER_LEN, writer.getNumBytes());
-        assertEquals("zero records", 0, writer.getNumRecords());
-
-        List<Future<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>();
-            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>());
-            fail("Should fail on writing large record");
-        } catch (LogRecordTooLongException lrtle) {
-            // expected
-        }
-        assertEquals("5 records", 5, writer.getNumRecords());
-
-        /// 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>();
-            writer.writeRecord(record, writePromise);
-            writePromiseList.add(writePromise);
-            assertEquals((i + 6) + " records", (i + 6), writer.getNumRecords());
-        }
-
-        ByteBuffer buffer = writer.getBuffer();
-        assertEquals("10 records", 10, writer.getNumRecords());
-
-        // Test transmit complete
-        writer.completeTransmit(1L, 1L, 10L);
-        List<DLSN> writeResults = Await.result(Future.collect(writePromiseList));
-        for (int i = 0; i < 10; i++) {
-            assertEquals(new DLSN(1L, 1L, 10L + i), writeResults.get(i));
-        }
-
-        // Test reading from buffer
-        byte[] data = new byte[buffer.remaining()];
-        buffer.get(data);
-
-        LogRecordWithDLSN record = new LogRecordWithDLSN(
-                new DLSN(1L, 1L, 10L),
-                99L,
-                data,
-                999L);
-        record.setPositionWithinLogSegment(888);
-        record.setRecordSet();
-        Reader reader = LogRecordSet.of(record);
-        LogRecordWithDLSN readRecord = reader.nextRecord();
-        int numReads = 0;
-        while (null != readRecord) {
-            assertEquals(new DLSN(1L, 1L, 10L + numReads), readRecord.getDlsn());
-            assertEquals(99L, readRecord.getTransactionId());
-            assertEquals(888 + numReads, readRecord.getPositionWithinLogSegment());
-            assertEquals(999L, readRecord.getStartSequenceIdOfCurrentSegment());
-            assertEquals(999L + 888 + numReads - 1, readRecord.getSequenceId());
-            // read next
-            ++numReads;
-            readRecord = reader.nextRecord();
-        }
-        assertEquals(10, numReads);
-    }
-
-}
diff --git a/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestTimedOutTestsListener.java b/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestTimedOutTestsListener.java
deleted file mode 100644
index 8d200ec..0000000
--- a/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TestTimedOutTestsListener.java
+++ /dev/null
@@ -1,183 +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 com.twitter.distributedlog;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.util.concurrent.CyclicBarrier;
-import java.util.concurrent.locks.Lock;
-import java.util.concurrent.locks.ReentrantLock;
-
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.notification.Failure;
-
-/**
- * Test Case for {@link TimedOutTestsListener}.
- */
-public class TestTimedOutTestsListener {
-
-    private static class Deadlock {
-        private CyclicBarrier barrier = new CyclicBarrier(6);
-
-        public Deadlock() {
-            DeadlockThread[] dThreads = new DeadlockThread[6];
-
-            Monitor a = new Monitor("a");
-            Monitor b = new Monitor("b");
-            Monitor c = new Monitor("c");
-            dThreads[0] = new DeadlockThread("MThread-1", a, b);
-            dThreads[1] = new DeadlockThread("MThread-2", b, c);
-            dThreads[2] = new DeadlockThread("MThread-3", c, a);
-
-            Lock d = new ReentrantLock();
-            Lock e = new ReentrantLock();
-            Lock f = new ReentrantLock();
-
-            dThreads[3] = new DeadlockThread("SThread-4", d, e);
-            dThreads[4] = new DeadlockThread("SThread-5", e, f);
-            dThreads[5] = new DeadlockThread("SThread-6", f, d);
-
-            // make them daemon threads so that the test will exit
-            for (int i = 0; i < 6; i++) {
-                dThreads[i].setDaemon(true);
-                dThreads[i].start();
-            }
-        }
-
-        class DeadlockThread extends Thread {
-            private Lock lock1 = null;
-
-            private Lock lock2 = null;
-
-            private Monitor mon1 = null;
-
-            private Monitor mon2 = null;
-
-            private boolean useSync;
-
-            DeadlockThread(String name, Lock lock1, Lock lock2) {
-                super(name);
-                this.lock1 = lock1;
-                this.lock2 = lock2;
-                this.useSync = true;
-            }
-
-            DeadlockThread(String name, Monitor mon1, Monitor mon2) {
-                super(name);
-                this.mon1 = mon1;
-                this.mon2 = mon2;
-                this.useSync = false;
-            }
-
-            public void run() {
-                if (useSync) {
-                    syncLock();
-                } else {
-                    monitorLock();
-                }
-            }
-
-            private void syncLock() {
-                lock1.lock();
-                try {
-                    try {
-                        barrier.await();
-                    } catch (Exception e) {
-                    }
-                    goSyncDeadlock();
-                } finally {
-                    lock1.unlock();
-                }
-            }
-
-            private void goSyncDeadlock() {
-                try {
-                    barrier.await();
-                } catch (Exception e) {
-                }
-                lock2.lock();
-                throw new RuntimeException("should not reach here.");
-            }
-
-            private void monitorLock() {
-                synchronized (mon1) {
-                    try {
-                        barrier.await();
-                    } catch (Exception e) {
-                    }
-                    goMonitorDeadlock();
-                }
-            }
-
-            private void goMonitorDeadlock() {
-                try {
-                    barrier.await();
-                } catch (Exception e) {
-                }
-                synchronized (mon2) {
-                    throw new RuntimeException(getName() + " should not reach here.");
-                }
-            }
-        }
-
-        class Monitor {
-            String name;
-
-            Monitor(String name) {
-                this.name = name;
-            }
-        }
-
-    }
-
-    @Test(timeout = 500)
-    public void testThreadDumpAndDeadlocks() throws Exception {
-        new Deadlock();
-        String s = null;
-        while (true) {
-            s = TimedOutTestsListener.buildDeadlockInfo();
-            if (s != null) {
-                break;
-            }
-            Thread.sleep(100);
-        }
-
-        Assert.assertEquals(3, countStringOccurrences(s, "BLOCKED"));
-
-        Failure failure = new Failure(null, new Exception(TimedOutTestsListener.TEST_TIMED_OUT_PREFIX));
-        StringWriter writer = new StringWriter();
-        new TimedOutTestsListener(new PrintWriter(writer)).testFailure(failure);
-        String out = writer.toString();
-
-        Assert.assertTrue(out.contains("THREAD DUMP"));
-        Assert.assertTrue(out.contains("DEADLOCKS DETECTED"));
-
-        System.out.println(out);
-    }
-
-    private int countStringOccurrences(String s, String substr) {
-        int n = 0;
-        int index = 0;
-        while ((index = s.indexOf(substr, index) + 1) != 0) {
-            n++;
-        }
-        return n;
-    }
-
-}
\ No newline at end of file
diff --git a/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TimedOutTestsListener.java b/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TimedOutTestsListener.java
deleted file mode 100644
index f2e80d2..0000000
--- a/distributedlog-protocol/src/test/java/com/twitter/distributedlog/TimedOutTestsListener.java
+++ /dev/null
@@ -1,168 +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 com.twitter.distributedlog;
-
-import java.io.PrintWriter;
-import java.io.StringWriter;
-import java.lang.management.LockInfo;
-import java.lang.management.ManagementFactory;
-import java.lang.management.MonitorInfo;
-import java.lang.management.ThreadInfo;
-import java.lang.management.ThreadMXBean;
-import java.text.DateFormat;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-import java.util.Map;
-
-import org.apache.commons.lang.StringUtils;
-import org.junit.runner.notification.Failure;
-import org.junit.runner.notification.RunListener;
-
-/**
- * JUnit run listener which prints full thread dump into System.err in case a test is failed due to
- * timeout.
- */
-public class TimedOutTestsListener extends RunListener {
-
-    static final String TEST_TIMED_OUT_PREFIX = "test timed out after";
-
-    private static String indent = "    ";
-
-    private final PrintWriter output;
-
-    public TimedOutTestsListener() {
-        this.output = new PrintWriter(System.err);
-    }
-
-    public TimedOutTestsListener(PrintWriter output) {
-        this.output = output;
-    }
-
-    @Override
-    public void testFailure(Failure failure) throws Exception {
-        if (failure != null && failure.getMessage() != null && failure.getMessage().startsWith(TEST_TIMED_OUT_PREFIX)) {
-            output.println("====> TEST TIMED OUT. PRINTING THREAD DUMP. <====");
-            output.println();
-            output.print(buildThreadDiagnosticString());
-        }
-    }
-
-    public static String buildThreadDiagnosticString() {
-        StringWriter sw = new StringWriter();
-        PrintWriter output = new PrintWriter(sw);
-
-        DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd hh:mm:ss,SSS");
-        output.println(String.format("Timestamp: %s", dateFormat.format(new Date())));
-        output.println();
-        output.println(buildThreadDump());
-
-        String deadlocksInfo = buildDeadlockInfo();
-        if (deadlocksInfo != null) {
-            output.println("====> DEADLOCKS DETECTED <====");
-            output.println();
-            output.println(deadlocksInfo);
-        }
-
-        return sw.toString();
-    }
-
-    static String buildThreadDump() {
-        StringBuilder dump = new StringBuilder();
-        Map<Thread, StackTraceElement[]> stackTraces = Thread.getAllStackTraces();
-        for (Map.Entry<Thread, StackTraceElement[]> e : stackTraces.entrySet()) {
-            Thread thread = e.getKey();
-            dump.append(String.format("\"%s\" %s prio=%d tid=%d %s\njava.lang.Thread.State: %s", thread.getName(),
-                (thread.isDaemon() ? "daemon" : ""), thread.getPriority(), thread.getId(),
-                Thread.State.WAITING.equals(thread.getState()) ? "in Object.wait()"
-                        : StringUtils.lowerCase(thread.getState().name()),
-                Thread.State.WAITING.equals(thread.getState()) ? "WAITING (on object monitor)" : thread.getState()));
-            for (StackTraceElement stackTraceElement : e.getValue()) {
-                dump.append("\n        at ");
-                dump.append(stackTraceElement);
-            }
-            dump.append("\n");
-        }
-        return dump.toString();
-    }
-
-    static String buildDeadlockInfo() {
-        ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
-        long[] threadIds = threadBean.findMonitorDeadlockedThreads();
-        if (threadIds != null && threadIds.length > 0) {
-            StringWriter stringWriter = new StringWriter();
-            PrintWriter out = new PrintWriter(stringWriter);
-
-            ThreadInfo[] infos = threadBean.getThreadInfo(threadIds, true, true);
-            for (ThreadInfo ti : infos) {
-                printThreadInfo(ti, out);
-                printLockInfo(ti.getLockedSynchronizers(), out);
-                out.println();
-            }
-
-            out.close();
-            return stringWriter.toString();
-        } else {
-            return null;
-        }
-    }
-
-    private static void printThreadInfo(ThreadInfo ti, PrintWriter out) {
-        // print thread information
-        printThread(ti, out);
-
-        // print stack trace with locks
-        StackTraceElement[] stacktrace = ti.getStackTrace();
-        MonitorInfo[] monitors = ti.getLockedMonitors();
-        for (int i = 0; i < stacktrace.length; i++) {
-            StackTraceElement ste = stacktrace[i];
-            out.println(indent + "at " + ste.toString());
-            for (MonitorInfo mi : monitors) {
-                if (mi.getLockedStackDepth() == i) {
-                    out.println(indent + "  - locked " + mi);
-                }
-            }
-        }
-        out.println();
-    }
-
-    private static void printThread(ThreadInfo ti, PrintWriter out) {
-        out.print("\"" + ti.getThreadName() + "\"" + " Id=" + ti.getThreadId() + " in " + ti.getThreadState());
-        if (ti.getLockName() != null) {
-            out.print(" on lock=" + ti.getLockName());
-        }
-        if (ti.isSuspended()) {
-            out.print(" (suspended)");
-        }
-        if (ti.isInNative()) {
-            out.print(" (running in native)");
-        }
-        out.println();
-        if (ti.getLockOwnerName() != null) {
-            out.println(indent + " owned by " + ti.getLockOwnerName() + " Id=" + ti.getLockOwnerId());
-        }
-    }
-
-    private static void printLockInfo(LockInfo[] locks, PrintWriter out) {
-        out.println(indent + "Locked synchronizers: count = " + locks.length);
-        for (LockInfo li : locks) {
-            out.println(indent + "  - " + li);
-        }
-        out.println();
-    }
-
-}
\ No newline at end of file
diff --git a/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestDLSN.java b/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestDLSN.java
new file mode 100644
index 0000000..f18f67c
--- /dev/null
+++ b/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestDLSN.java
@@ -0,0 +1,64 @@
+/**
+ * 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.fail;
+
+import org.junit.Test;
+
+/**
+ * Test Case for {@link DLSN}.
+ */
+public class TestDLSN {
+
+    @Test(timeout = 60000)
+    public void testDLSN() {
+        DLSN dlsn = new DLSN(99L, 88L, 77L);
+        String dlsnv0 = dlsn.serialize(DLSN.VERSION0);
+        String dlsnv1 = dlsn.serialize(DLSN.VERSION1);
+        String badDLSN = "baddlsn";
+
+        assertEquals(dlsn, DLSN.deserialize(dlsnv0));
+        assertEquals(dlsn, DLSN.deserialize(dlsnv1));
+        try {
+            DLSN.deserialize(badDLSN);
+            fail("Should fail on deserializing bad dlsn");
+        } catch (IllegalArgumentException iae) {
+        }
+
+        assertEquals(dlsn, DLSN.deserialize0(dlsnv0));
+        try {
+            DLSN.deserialize0(dlsnv1);
+            fail("Should fail on deserializing version one dlsn");
+        } catch (IllegalArgumentException iae) {
+        }
+        try {
+            DLSN.deserialize0(badDLSN);
+            fail("Should fail on deserializing bad dlsn");
+        } catch (IllegalArgumentException iae) {
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testSerializeDeserializeBytes() {
+        DLSN dlsn = new DLSN(99L, 88L, 77L);
+        byte[] data = dlsn.serializeBytes();
+        assertEquals(dlsn, DLSN.deserializeBytes(data));
+    }
+}
diff --git a/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java b/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java
new file mode 100644
index 0000000..95e03ab
--- /dev/null
+++ b/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestLogRecordSet.java
@@ -0,0 +1,178 @@
+/**
+ * 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 com.google.common.base.Charsets.UTF_8;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+import static org.apache.distributedlog.LogRecordSet.HEADER_LEN;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Lists;
+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.junit.Test;
+
+/**
+ * Test Case for {@link LogRecordSet}.
+ */
+public class TestLogRecordSet {
+
+    @Test(timeout = 60000)
+    public void testEmptyRecordSet() throws Exception {
+        Writer writer = LogRecordSet.newWriter(1024, Type.NONE);
+        assertEquals("zero user bytes", HEADER_LEN, writer.getNumBytes());
+        assertEquals("zero records", 0, writer.getNumRecords());
+
+        ByteBuffer buffer = writer.getBuffer();
+        assertEquals("zero user bytes", HEADER_LEN, buffer.remaining());
+
+        byte[] data = new byte[buffer.remaining()];
+        buffer.get(data);
+
+        LogRecordWithDLSN record = new LogRecordWithDLSN(
+                new DLSN(1L, 0L, 0L),
+                1L,
+                data,
+                1L);
+        record.setRecordSet();
+        Reader reader = LogRecordSet.of(record);
+        assertNull("Empty record set should return null",
+                reader.nextRecord());
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteTooLongRecord() throws Exception {
+        Writer writer = LogRecordSet.newWriter(1024, Type.NONE);
+        assertEquals("zero user bytes", HEADER_LEN, writer.getNumBytes());
+        assertEquals("zero records", 0, writer.getNumRecords());
+
+        ByteBuffer dataBuf = ByteBuffer.allocate(MAX_LOGRECORD_SIZE + 1);
+        try {
+            writer.writeRecord(dataBuf, new Promise<DLSN>());
+            fail("Should fail on writing large record");
+        } catch (LogRecordTooLongException lrtle) {
+            // expected
+        }
+        assertEquals("zero user bytes", HEADER_LEN, writer.getNumBytes());
+        assertEquals("zero records", 0, writer.getNumRecords());
+        ByteBuffer buffer = writer.getBuffer();
+        assertEquals("zero user bytes", HEADER_LEN, buffer.remaining());
+
+        byte[] data = new byte[buffer.remaining()];
+        buffer.get(data);
+
+        LogRecordWithDLSN record = new LogRecordWithDLSN(
+                new DLSN(1L, 0L, 0L),
+                1L,
+                data,
+                1L);
+        record.setRecordSet();
+        Reader reader = LogRecordSet.of(record);
+        assertNull("Empty record set should return null",
+                reader.nextRecord());
+    }
+
+    @Test(timeout = 20000)
+    public void testWriteRecordsNoneCompressed() throws Exception {
+        testWriteRecords(Type.NONE);
+    }
+
+    @Test(timeout = 20000)
+    public void testWriteRecordsLZ4Compressed() throws Exception {
+        testWriteRecords(Type.LZ4);
+    }
+
+    void testWriteRecords(Type codec) throws Exception {
+        Writer writer = LogRecordSet.newWriter(1024, codec);
+        assertEquals("zero user bytes", HEADER_LEN, writer.getNumBytes());
+        assertEquals("zero records", 0, writer.getNumRecords());
+
+        List<Future<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>();
+            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>());
+            fail("Should fail on writing large record");
+        } catch (LogRecordTooLongException lrtle) {
+            // expected
+        }
+        assertEquals("5 records", 5, writer.getNumRecords());
+
+        /// 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>();
+            writer.writeRecord(record, writePromise);
+            writePromiseList.add(writePromise);
+            assertEquals((i + 6) + " records", (i + 6), writer.getNumRecords());
+        }
+
+        ByteBuffer buffer = writer.getBuffer();
+        assertEquals("10 records", 10, writer.getNumRecords());
+
+        // Test transmit complete
+        writer.completeTransmit(1L, 1L, 10L);
+        List<DLSN> writeResults = Await.result(Future.collect(writePromiseList));
+        for (int i = 0; i < 10; i++) {
+            assertEquals(new DLSN(1L, 1L, 10L + i), writeResults.get(i));
+        }
+
+        // Test reading from buffer
+        byte[] data = new byte[buffer.remaining()];
+        buffer.get(data);
+
+        LogRecordWithDLSN record = new LogRecordWithDLSN(
+                new DLSN(1L, 1L, 10L),
+                99L,
+                data,
+                999L);
+        record.setPositionWithinLogSegment(888);
+        record.setRecordSet();
+        Reader reader = LogRecordSet.of(record);
+        LogRecordWithDLSN readRecord = reader.nextRecord();
+        int numReads = 0;
+        while (null != readRecord) {
+            assertEquals(new DLSN(1L, 1L, 10L + numReads), readRecord.getDlsn());
+            assertEquals(99L, readRecord.getTransactionId());
+            assertEquals(888 + numReads, readRecord.getPositionWithinLogSegment());
+            assertEquals(999L, readRecord.getStartSequenceIdOfCurrentSegment());
+            assertEquals(999L + 888 + numReads - 1, readRecord.getSequenceId());
+            // read next
+            ++numReads;
+            readRecord = reader.nextRecord();
+        }
+        assertEquals(10, numReads);
+    }
+
+}
diff --git a/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestTimedOutTestsListener.java b/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestTimedOutTestsListener.java
new file mode 100644
index 0000000..20cf53c
--- /dev/null
+++ b/distributedlog-protocol/src/test/java/org/apache/distributedlog/TestTimedOutTestsListener.java
@@ -0,0 +1,183 @@
+/**
+ * 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 java.io.PrintWriter;
+import java.io.StringWriter;
+import java.util.concurrent.CyclicBarrier;
+import java.util.concurrent.locks.Lock;
+import java.util.concurrent.locks.ReentrantLock;
+
+import org.junit.Assert;
+import org.junit.Test;
+import org.junit.runner.notification.Failure;
+
+/**
+ * Test Case for {@link TimedOutTestsListener}.
+ */
+public class TestTimedOutTestsListener {
+
+    private static class Deadlock {
+        private CyclicBarrier barrier = new CyclicBarrier(6);
+
+        public Deadlock() {
+            DeadlockThread[] dThreads = new DeadlockThread[6];
+
+            Monitor a = new Monitor("a");
+            Monitor b = new Monitor("b");
+            Monitor c = new Monitor("c");
+            dThreads[0] = new DeadlockThread("MThread-1", a, b);
+            dThreads[1] = new DeadlockThread("MThread-2", b, c);
+            dThreads[2] = new DeadlockThread("MThread-3", c, a);
+
+            Lock d = new ReentrantLock();
+            Lock e = new ReentrantLock();
+            Lock f = new ReentrantLock();
+
+            dThreads[3] = new DeadlockThread("SThread-4", d, e);
+            dThreads[4] = new DeadlockThread("SThread-5", e, f);
+            dThreads[5] = new DeadlockThread("SThread-6", f, d);
+
+            // make them daemon threads so that the test will exit
+            for (int i = 0; i < 6; i++) {
+                dThreads[i].setDaemon(true);
+                dThreads[i].start();
+            }
+        }
+
+        class DeadlockThread extends Thread {
+            private Lock lock1 = null;
+
+            private Lock lock2 = null;
+
+            private Monitor mon1 = null;
+
+            private Monitor mon2 = null;
+
+            private boolean useSync;
+
+            DeadlockThread(String name, Lock lock1, Lock lock2) {
+                super(name);
+                this.lock1 = lock1;
+                this.lock2 = lock2;
+                this.useSync = true;
+            }
+
+            DeadlockThread(String name, Monitor mon1, Monitor mon2) {
+                super(name);
+                this.mon1 = mon1;
+                this.mon2 = mon2;
+                this.useSync = false;
+            }
+
+            public void run() {
+                if (useSync) {
+                    syncLock();
+                } else {
+                    monitorLock();
+                }
+            }
+
+            private void syncLock() {
+                lock1.lock();
+                try {
+                    try {
+                        barrier.await();
+                    } catch (Exception e) {
+                    }
+                    goSyncDeadlock();
+                } finally {
+                    lock1.unlock();
+                }
+            }
+
+            private void goSyncDeadlock() {
+                try {
+                    barrier.await();
+                } catch (Exception e) {
+                }
+                lock2.lock();
+                throw new RuntimeException("should not reach here.");
+            }
+
+            private void monitorLock() {
+                synchronized (mon1) {
+                    try {
+                        barrier.await();
+                    } catch (Exception e) {
+                    }
+                    goMonitorDeadlock();
+                }
+            }
+
+            private void goMonitorDeadlock() {
+                try {
+                    barrier.await();
+                } catch (Exception e) {
+                }
+                synchronized (mon2) {
+                    throw new RuntimeException(getName() + " should not reach here.");
+                }
+            }
+        }
+
+        class Monitor {
+            String name;
+
+            Monitor(String name) {
+                this.name = name;
+            }
+        }
+
+    }
+
+    @Test(timeout = 500)
+    public void testThreadDumpAndDeadlocks() throws Exception {
+        new Deadlock();
+        String s = null;
+        while (true) {
+            s = TimedOutTestsListener.buildDeadlockInfo();
+            if (s != null) {
+                break;
+            }
+            Thread.sleep(100);
+        }
+
+        Assert.assertEquals(3, countStringOccurrences(s, "BLOCKED"));
+
+        Failure failure = new Failure(null, new Exception(TimedOutTestsListener.TEST_TIMED_OUT_PREFIX));
+        StringWriter writer = new StringWriter();
+        new TimedOutTestsListener(new PrintWriter(writer)).testFailure(failure);
+        String out = writer.toString();
+
+        Assert.assertTrue(out.contains("THREAD DUMP"));
+        Assert.assertTrue(out.contains("DEADLOCKS DETECTED"));
+
+        System.out.println(out);
+    }
+
+    private int countStringOccurrences(String s, String substr) {
+        int n = 0;
+        int index = 0;
+        while ((index = s.indexOf(substr, index) + 1) != 0) {
+            n++;
+        }
+        return n;
+    }
+
+}
diff --git a/distributedlog-protocol/src/test/java/org/apache/distributedlog/TimedOutTestsListener.java b/distributedlog-protocol/src/test/java/org/apache/distributedlog/TimedOutTestsListener.java
new file mode 100644
index 0000000..db0ee4e
--- /dev/null
+++ b/distributedlog-protocol/src/test/java/org/apache/distributedlog/TimedOutTestsListener.java
@@ -0,0 +1,168 @@
+/**
+ * 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 java.io.PrintWriter;
+import java.io.StringWriter;
+import java.lang.management.LockInfo;
+import java.lang.management.ManagementFactory;
+import java.lang.management.MonitorInfo;
+import java.lang.management.ThreadInfo;
+import java.lang.management.ThreadMXBean;
+import java.text.DateFormat;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+import java.util.Map;
+
+import org.apache.commons.lang.StringUtils;
+import org.junit.runner.notification.Failure;
+import org.junit.runner.notification.RunListener;
+
+/**
+ * JUnit run listener which prints full thread dump into System.err in case a test is failed due to
+ * timeout.
+ */
+public class TimedOutTestsListener extends RunListener {
+
+    static final String TEST_TIMED_OUT_PREFIX = "test timed out after";
+
+    private static String indent = "    ";
+
+    private final PrintWriter output;
+
+    public TimedOutTestsListener() {
+        this.output = new PrintWriter(System.err);
+    }
+
+    public TimedOutTestsListener(PrintWriter output) {
+        this.output = output;
+    }
+
+    @Override
+    public void testFailure(Failure failure) throws Exception {
+        if (failure != null && failure.getMessage() != null && failure.getMessage().startsWith(TEST_TIMED_OUT_PREFIX)) {
+            output.println("====> TEST TIMED OUT. PRINTING THREAD DUMP. <====");
+            output.println();
+            output.print(buildThreadDiagnosticString());
+        }
+    }
+
+    public static String buildThreadDiagnosticString() {
+        StringWriter sw = new StringWriter();
+        PrintWriter output = new PrintWriter(sw);
+
+        DateFormat dateFormat = new SimpleDateFormat("yyyy-MM-dd hh:mm:ss,SSS");
+        output.println(String.format("Timestamp: %s", dateFormat.format(new Date())));
+        output.println();
+        output.println(buildThreadDump());
+
+        String deadlocksInfo = buildDeadlockInfo();
+        if (deadlocksInfo != null) {
+            output.println("====> DEADLOCKS DETECTED <====");
+            output.println();
+            output.println(deadlocksInfo);
+        }
+
+        return sw.toString();
+    }
+
+    static String buildThreadDump() {
+        StringBuilder dump = new StringBuilder();
+        Map<Thread, StackTraceElement[]> stackTraces = Thread.getAllStackTraces();
+        for (Map.Entry<Thread, StackTraceElement[]> e : stackTraces.entrySet()) {
+            Thread thread = e.getKey();
+            dump.append(String.format("\"%s\" %s prio=%d tid=%d %s\njava.lang.Thread.State: %s", thread.getName(),
+                (thread.isDaemon() ? "daemon" : ""), thread.getPriority(), thread.getId(),
+                Thread.State.WAITING.equals(thread.getState()) ? "in Object.wait()"
+                        : StringUtils.lowerCase(thread.getState().name()),
+                Thread.State.WAITING.equals(thread.getState()) ? "WAITING (on object monitor)" : thread.getState()));
+            for (StackTraceElement stackTraceElement : e.getValue()) {
+                dump.append("\n        at ");
+                dump.append(stackTraceElement);
+            }
+            dump.append("\n");
+        }
+        return dump.toString();
+    }
+
+    static String buildDeadlockInfo() {
+        ThreadMXBean threadBean = ManagementFactory.getThreadMXBean();
+        long[] threadIds = threadBean.findMonitorDeadlockedThreads();
+        if (threadIds != null && threadIds.length > 0) {
+            StringWriter stringWriter = new StringWriter();
+            PrintWriter out = new PrintWriter(stringWriter);
+
+            ThreadInfo[] infos = threadBean.getThreadInfo(threadIds, true, true);
+            for (ThreadInfo ti : infos) {
+                printThreadInfo(ti, out);
+                printLockInfo(ti.getLockedSynchronizers(), out);
+                out.println();
+            }
+
+            out.close();
+            return stringWriter.toString();
+        } else {
+            return null;
+        }
+    }
+
+    private static void printThreadInfo(ThreadInfo ti, PrintWriter out) {
+        // print thread information
+        printThread(ti, out);
+
+        // print stack trace with locks
+        StackTraceElement[] stacktrace = ti.getStackTrace();
+        MonitorInfo[] monitors = ti.getLockedMonitors();
+        for (int i = 0; i < stacktrace.length; i++) {
+            StackTraceElement ste = stacktrace[i];
+            out.println(indent + "at " + ste.toString());
+            for (MonitorInfo mi : monitors) {
+                if (mi.getLockedStackDepth() == i) {
+                    out.println(indent + "  - locked " + mi);
+                }
+            }
+        }
+        out.println();
+    }
+
+    private static void printThread(ThreadInfo ti, PrintWriter out) {
+        out.print("\"" + ti.getThreadName() + "\"" + " Id=" + ti.getThreadId() + " in " + ti.getThreadState());
+        if (ti.getLockName() != null) {
+            out.print(" on lock=" + ti.getLockName());
+        }
+        if (ti.isSuspended()) {
+            out.print(" (suspended)");
+        }
+        if (ti.isInNative()) {
+            out.print(" (running in native)");
+        }
+        out.println();
+        if (ti.getLockOwnerName() != null) {
+            out.println(indent + " owned by " + ti.getLockOwnerName() + " Id=" + ti.getLockOwnerId());
+        }
+    }
+
+    private static void printLockInfo(LockInfo[] locks, PrintWriter out) {
+        out.println(indent + "Locked synchronizers: count = " + locks.length);
+        for (LockInfo li : locks) {
+            out.println(indent + "  - " + li);
+        }
+        out.println();
+    }
+
+}
diff --git a/distributedlog-service/bin/dlog-daemon.sh b/distributedlog-service/bin/dlog-daemon.sh
index 976d4f1..fdd87df 100755
--- a/distributedlog-service/bin/dlog-daemon.sh
+++ b/distributedlog-service/bin/dlog-daemon.sh
@@ -78,7 +78,7 @@
     DLOG_ROOT_LOGGER=${BK_ROOT_LOGGER:-'INFO,R'}
     ;;
   (writeproxy)
-    service_class="com.twitter.distributedlog.service.DistributedLogServerApp"
+    service_class="org.apache.distributedlog.service.DistributedLogServerApp"
     DLOG_ROOT_LOGGER=${WP_ROOT_LOGGER:-'INFO,R'}
     WP_CONF_FILE=${WP_CONF_FILE:-"$DL_HOME/conf/write_proxy.conf"}
     WP_SERVICE_PORT=${WP_SERVICE_PORT:-'4181'}
diff --git a/distributedlog-service/bin/dlog-start.sh b/distributedlog-service/bin/dlog-start.sh
index 03c1f6e..a4a539b 100755
--- a/distributedlog-service/bin/dlog-start.sh
+++ b/distributedlog-service/bin/dlog-start.sh
@@ -30,7 +30,7 @@
      -Dlog4j.configuration=conf/log4j.properties \
      -DstatsHttpPort=9000 -DstatsExport=true \
      -Dserver_shard=0 \
-     com.twitter.distributedlog.service.DistributedLogServerApp \
+     org.apache.distributedlog.service.DistributedLogServerApp \
      --port 8000 \
      --uri "${DISTRIBUTEDLOG_URI}" \
      --conf conf/distributedlog.conf
diff --git a/distributedlog-service/pom.xml b/distributedlog-service/pom.xml
index 8399675..4c155e3 100644
--- a/distributedlog-service/pom.xml
+++ b/distributedlog-service/pom.xml
@@ -195,7 +195,7 @@
           <properties>
             <property>
               <name>listener</name>
-              <value>com.twitter.distributedlog.TimedOutTestsListener</value>
+              <value>org.apache.distributedlog.TimedOutTestsListener</value>
             </property>
           </properties>
         </configuration>
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/ClientUtils.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/ClientUtils.java
deleted file mode 100644
index da36014..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/ClientUtils.java
+++ /dev/null
@@ -1,33 +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 com.twitter.distributedlog.service;
-
-import com.twitter.distributedlog.client.DistributedLogClientImpl;
-import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
-import org.apache.commons.lang3.tuple.Pair;
-
-/**
- * DistributedLog Client Related Utils.
- */
-public class ClientUtils {
-
-    public static Pair<DistributedLogClient, MonitorServiceClient> buildClient(DistributedLogClientBuilder builder) {
-        DistributedLogClientImpl clientImpl = builder.buildClient();
-        return Pair.of((DistributedLogClient) clientImpl, (MonitorServiceClient) clientImpl);
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogCluster.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogCluster.java
deleted file mode 100644
index 029c822..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogCluster.java
+++ /dev/null
@@ -1,352 +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 com.twitter.distributedlog.service;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.LocalDLMEmulator;
-import com.twitter.distributedlog.client.routing.SingleHostRoutingService;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.metadata.DLMetadata;
-import com.twitter.distributedlog.service.placement.EqualLoadAppraiser;
-import com.twitter.distributedlog.service.streamset.IdentityStreamPartitionConverter;
-import com.twitter.finagle.builder.Server;
-import java.io.File;
-import java.net.BindException;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.conf.ServerConfiguration;
-import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
-import org.apache.bookkeeper.stats.NullStatsProvider;
-import org.apache.bookkeeper.util.IOUtils;
-import org.apache.bookkeeper.util.LocalBookKeeper;
-import org.apache.commons.io.FileUtils;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * DistributedLog Cluster is an emulator to run distributedlog components.
- */
-public class DistributedLogCluster {
-
-    private static final Logger LOG = LoggerFactory.getLogger(DistributedLogCluster.class);
-
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    /**
-     * Builder to build distributedlog cluster.
-     */
-    public static class Builder {
-
-        int numBookies = 3;
-        boolean shouldStartZK = true;
-        String zkHost = "127.0.0.1";
-        int zkPort = 0;
-        boolean shouldStartProxy = true;
-        int proxyPort = 7000;
-        boolean thriftmux = false;
-        DistributedLogConfiguration dlConf = new DistributedLogConfiguration()
-                .setLockTimeout(10)
-                .setOutputBufferSize(0)
-                .setImmediateFlushEnabled(true);
-        ServerConfiguration bkConf = new ServerConfiguration();
-
-        private Builder() {}
-
-        /**
-         * How many bookies to run. By default is 3.
-         *
-         * @return builder
-         */
-        public Builder numBookies(int numBookies) {
-            this.numBookies = numBookies;
-            return this;
-        }
-
-        /**
-         * Whether to start zookeeper? By default is true.
-         *
-         * @param startZK
-         *          flag to start zookeeper?
-         * @return builder
-         */
-        public Builder shouldStartZK(boolean startZK) {
-            this.shouldStartZK = startZK;
-            return this;
-        }
-
-        /**
-         * ZooKeeper server to run. By default it runs locally on '127.0.0.1'.
-         *
-         * @param zkServers
-         *          zk servers
-         * @return builder
-         */
-        public Builder zkServers(String zkServers) {
-            this.zkHost = zkServers;
-            return this;
-        }
-
-        /**
-         * ZooKeeper server port to listen on. By default it listens on 2181.
-         *
-         * @param zkPort
-         *          zookeeper server port.
-         * @return builder.
-         */
-        public Builder zkPort(int zkPort) {
-            this.zkPort = zkPort;
-            return this;
-        }
-
-        /**
-         * Whether to start proxy or not. By default is true.
-         *
-         * @param startProxy
-         *          whether to start proxy or not.
-         * @return builder
-         */
-        public Builder shouldStartProxy(boolean startProxy) {
-            this.shouldStartProxy = startProxy;
-            return this;
-        }
-
-        /**
-         * Port that proxy server to listen on. By default is 7000.
-         *
-         * @param proxyPort
-         *          port that proxy server to listen on.
-         * @return builder
-         */
-        public Builder proxyPort(int proxyPort) {
-            this.proxyPort = proxyPort;
-            return this;
-        }
-
-        /**
-         * Set the distributedlog configuration.
-         *
-         * @param dlConf
-         *          distributedlog configuration
-         * @return builder
-         */
-        public Builder dlConf(DistributedLogConfiguration dlConf) {
-            this.dlConf = dlConf;
-            return this;
-        }
-
-        /**
-         * Set the Bookkeeper server configuration.
-         *
-         * @param bkConf
-         *          bookkeeper server configuration
-         * @return builder
-         */
-        public Builder bkConf(ServerConfiguration bkConf) {
-            this.bkConf = bkConf;
-            return this;
-        }
-
-        /**
-         * Enable thriftmux for the dl server.
-         *
-         * @param enabled flag to enable thriftmux
-         * @return builder
-         */
-        public Builder thriftmux(boolean enabled) {
-            this.thriftmux = enabled;
-            return this;
-        }
-
-        public DistributedLogCluster build() throws Exception {
-            // build the cluster
-            return new DistributedLogCluster(
-                dlConf,
-                bkConf,
-                numBookies,
-                shouldStartZK,
-                zkHost,
-                zkPort,
-                shouldStartProxy,
-                proxyPort,
-                thriftmux);
-        }
-    }
-
-    /**
-     * Run a distributedlog proxy server.
-     */
-    public static class DLServer {
-
-        static final int MAX_RETRIES = 20;
-        static final int MIN_PORT = 1025;
-        static final int MAX_PORT = 65535;
-
-        int proxyPort;
-
-        public final InetSocketAddress address;
-        public final Pair<DistributedLogServiceImpl, Server> dlServer;
-        private final SingleHostRoutingService routingService = SingleHostRoutingService.of(null);
-
-        protected DLServer(DistributedLogConfiguration dlConf,
-                           URI uri,
-                           int basePort,
-                           boolean thriftmux) throws Exception {
-            proxyPort = basePort;
-
-            boolean success = false;
-            int retries = 0;
-            Pair<DistributedLogServiceImpl, Server> serverPair = null;
-            while (!success) {
-                try {
-                    com.twitter.distributedlog.service.config.ServerConfiguration serverConf =
-                            new com.twitter.distributedlog.service.config.ServerConfiguration();
-                    serverConf.loadConf(dlConf);
-                    serverConf.setServerShardId(proxyPort);
-                    serverPair = DistributedLogServer.runServer(
-                            serverConf,
-                            dlConf,
-                            uri,
-                            new IdentityStreamPartitionConverter(),
-                            routingService,
-                            new NullStatsProvider(),
-                            proxyPort,
-                            thriftmux,
-                            new EqualLoadAppraiser());
-                    routingService.setAddress(DLSocketAddress.getSocketAddress(proxyPort));
-                    routingService.startService();
-                    serverPair.getLeft().startPlacementPolicy();
-                    success = true;
-                } catch (BindException be) {
-                    retries++;
-                    if (retries > MAX_RETRIES) {
-                        throw be;
-                    }
-                    proxyPort++;
-                    if (proxyPort > MAX_PORT) {
-                        proxyPort = MIN_PORT;
-                    }
-                }
-            }
-
-            LOG.info("Running DL on port {}", proxyPort);
-
-            dlServer = serverPair;
-            address = DLSocketAddress.getSocketAddress(proxyPort);
-        }
-
-        public InetSocketAddress getAddress() {
-            return address;
-        }
-
-        public void shutdown() {
-            DistributedLogServer.closeServer(dlServer, 0, TimeUnit.MILLISECONDS);
-            routingService.stopService();
-        }
-    }
-
-    private final DistributedLogConfiguration dlConf;
-    private final ZooKeeperServerShim zks;
-    private final LocalDLMEmulator dlmEmulator;
-    private DLServer dlServer;
-    private final boolean shouldStartProxy;
-    private final int proxyPort;
-    private final boolean thriftmux;
-    private final List<File> tmpDirs = new ArrayList<File>();
-
-    private DistributedLogCluster(DistributedLogConfiguration dlConf,
-                                  ServerConfiguration bkConf,
-                                  int numBookies,
-                                  boolean shouldStartZK,
-                                  String zkServers,
-                                  int zkPort,
-                                  boolean shouldStartProxy,
-                                  int proxyPort,
-                                  boolean thriftmux) throws Exception {
-        this.dlConf = dlConf;
-        if (shouldStartZK) {
-            File zkTmpDir = IOUtils.createTempDir("zookeeper", "distrlog");
-            tmpDirs.add(zkTmpDir);
-            if (0 == zkPort) {
-                Pair<ZooKeeperServerShim, Integer> serverAndPort = LocalDLMEmulator.runZookeeperOnAnyPort(zkTmpDir);
-                this.zks = serverAndPort.getLeft();
-                zkPort = serverAndPort.getRight();
-            } else {
-                this.zks = LocalBookKeeper.runZookeeper(1000, zkPort, zkTmpDir);
-            }
-        } else {
-            this.zks = null;
-        }
-        this.dlmEmulator = LocalDLMEmulator.newBuilder()
-                .numBookies(numBookies)
-                .zkHost(zkServers)
-                .zkPort(zkPort)
-                .serverConf(bkConf)
-                .shouldStartZK(false)
-                .build();
-        this.shouldStartProxy = shouldStartProxy;
-        this.proxyPort = proxyPort;
-        this.thriftmux = thriftmux;
-    }
-
-    public void start() throws Exception {
-        this.dlmEmulator.start();
-        BKDLConfig bkdlConfig = new BKDLConfig(this.dlmEmulator.getZkServers(), "/ledgers").setACLRootPath(".acl");
-        DLMetadata.create(bkdlConfig).update(this.dlmEmulator.getUri());
-        if (shouldStartProxy) {
-            this.dlServer = new DLServer(
-                    dlConf,
-                    this.dlmEmulator.getUri(),
-                    proxyPort,
-                    thriftmux);
-        } else {
-            this.dlServer = null;
-        }
-    }
-
-    public void stop() throws Exception {
-        if (null != dlServer) {
-            this.dlServer.shutdown();
-        }
-        this.dlmEmulator.teardown();
-        if (null != this.zks) {
-            this.zks.stop();
-        }
-        for (File dir : tmpDirs) {
-            FileUtils.deleteDirectory(dir);
-        }
-    }
-
-    public URI getUri() {
-        return this.dlmEmulator.getUri();
-    }
-
-    public String getZkServers() {
-        return this.dlmEmulator.getZkServers();
-    }
-
-    public String getProxyFinagleStr() {
-        return "inet!" + (dlServer == null ? "127.0.0.1:" + proxyPort : dlServer.getAddress().toString());
-    }
-
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogServer.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogServer.java
deleted file mode 100644
index 248bcf7..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogServer.java
+++ /dev/null
@@ -1,460 +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 com.twitter.distributedlog.service;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.client.routing.RoutingService;
-import com.twitter.distributedlog.config.DynamicConfigurationFactory;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.service.announcer.Announcer;
-import com.twitter.distributedlog.service.announcer.NOPAnnouncer;
-import com.twitter.distributedlog.service.announcer.ServerSetAnnouncer;
-import com.twitter.distributedlog.service.config.DefaultStreamConfigProvider;
-import com.twitter.distributedlog.service.config.NullStreamConfigProvider;
-import com.twitter.distributedlog.service.config.ServerConfiguration;
-import com.twitter.distributedlog.service.config.ServiceStreamConfigProvider;
-import com.twitter.distributedlog.service.config.StreamConfigProvider;
-import com.twitter.distributedlog.service.placement.EqualLoadAppraiser;
-import com.twitter.distributedlog.service.placement.LoadAppraiser;
-import com.twitter.distributedlog.service.streamset.IdentityStreamPartitionConverter;
-import com.twitter.distributedlog.service.streamset.StreamPartitionConverter;
-import com.twitter.distributedlog.thrift.service.DistributedLogService;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.finagle.Stack;
-import com.twitter.finagle.ThriftMuxServer$;
-import com.twitter.finagle.builder.Server;
-import com.twitter.finagle.builder.ServerBuilder;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.finagle.thrift.ClientIdRequiredFilter;
-import com.twitter.finagle.thrift.ThriftServerFramedCodec;
-import com.twitter.finagle.transport.Transport;
-import com.twitter.util.Duration;
-import java.io.File;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.bookkeeper.stats.StatsProvider;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.apache.thrift.protocol.TBinaryProtocol;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.Option;
-import scala.Tuple2;
-
-/**
- * Running the distributedlog proxy server.
- */
-public class DistributedLogServer {
-
-    private static final Logger logger = LoggerFactory.getLogger(DistributedLogServer.class);
-    private static final String DEFAULT_LOAD_APPRIASER = EqualLoadAppraiser.class.getCanonicalName();
-
-    private DistributedLogServiceImpl dlService = null;
-    private Server server = null;
-    private RoutingService routingService;
-    private StatsProvider statsProvider;
-    private Announcer announcer = null;
-    private ScheduledExecutorService configExecutorService;
-    private long gracefulShutdownMs = 0L;
-
-    private final StatsReceiver statsReceiver;
-    private final CountDownLatch keepAliveLatch = new CountDownLatch(1);
-    private final Optional<String> uri;
-    private final Optional<String> conf;
-    private final Optional<String> streamConf;
-    private final Optional<Integer> port;
-    private final Optional<Integer> statsPort;
-    private final Optional<Integer> shardId;
-    private final Optional<Boolean> announceServerSet;
-    private final Optional<String> loadAppraiserClassStr;
-    private final Optional<Boolean> thriftmux;
-
-    DistributedLogServer(Optional<String> uri,
-                         Optional<String> conf,
-                         Optional<String> streamConf,
-                         Optional<Integer> port,
-                         Optional<Integer> statsPort,
-                         Optional<Integer> shardId,
-                         Optional<Boolean> announceServerSet,
-                         Optional<String> loadAppraiserClass,
-                         Optional<Boolean> thriftmux,
-                         RoutingService routingService,
-                         StatsReceiver statsReceiver,
-                         StatsProvider statsProvider) {
-        this.uri = uri;
-        this.conf = conf;
-        this.streamConf = streamConf;
-        this.port = port;
-        this.statsPort = statsPort;
-        this.shardId = shardId;
-        this.announceServerSet = announceServerSet;
-        this.thriftmux = thriftmux;
-        this.routingService = routingService;
-        this.statsReceiver = statsReceiver;
-        this.statsProvider = statsProvider;
-        this.loadAppraiserClassStr = loadAppraiserClass;
-    }
-
-    public void runServer()
-        throws ConfigurationException, IllegalArgumentException, IOException, ClassNotFoundException {
-        if (!uri.isPresent()) {
-            throw new IllegalArgumentException("No distributedlog uri provided.");
-        }
-        URI dlUri = URI.create(uri.get());
-        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
-        if (conf.isPresent()) {
-            String configFile = conf.get();
-            try {
-                dlConf.loadConf(new File(configFile).toURI().toURL());
-            } catch (ConfigurationException e) {
-                throw new IllegalArgumentException("Failed to load distributedlog configuration from "
-                    + configFile + ".");
-            } catch (MalformedURLException e) {
-                throw new IllegalArgumentException("Failed to load distributedlog configuration from malformed "
-                        + configFile + ".");
-            }
-        }
-
-        this.configExecutorService = Executors.newScheduledThreadPool(1,
-                new ThreadFactoryBuilder()
-                        .setNameFormat("DistributedLogService-Dyncfg-%d")
-                        .setDaemon(true)
-                        .build());
-
-        // server configuration and dynamic configuration
-        ServerConfiguration serverConf = new ServerConfiguration();
-        serverConf.loadConf(dlConf);
-
-        // overwrite the shard id if it is provided in the args
-        if (shardId.isPresent()) {
-            serverConf.setServerShardId(shardId.get());
-        }
-
-        serverConf.validate();
-
-        DynamicDistributedLogConfiguration dynDlConf = getServiceDynConf(dlConf);
-
-        logger.info("Starting stats provider : {}", statsProvider.getClass());
-        statsProvider.start(dlConf);
-
-        if (announceServerSet.isPresent() && announceServerSet.get()) {
-            announcer = new ServerSetAnnouncer(
-                    dlUri,
-                    port.or(0),
-                    statsPort.or(0),
-                    shardId.or(0));
-        } else {
-            announcer = new NOPAnnouncer();
-        }
-
-        // Build the stream partition converter
-        StreamPartitionConverter converter;
-        try {
-            converter = ReflectionUtils.newInstance(serverConf.getStreamPartitionConverterClass());
-        } catch (ConfigurationException e) {
-            logger.warn("Failed to load configured stream-to-partition converter. Fallback to use {}",
-                    IdentityStreamPartitionConverter.class.getName());
-            converter = new IdentityStreamPartitionConverter();
-        }
-        Class loadAppraiserClass = Class.forName(loadAppraiserClassStr.or(DEFAULT_LOAD_APPRIASER));
-        LoadAppraiser loadAppraiser = (LoadAppraiser) ReflectionUtils.newInstance(loadAppraiserClass);
-        logger.info("Supplied load appraiser class is " + loadAppraiserClassStr.get()
-            + " Instantiated " + loadAppraiser.getClass().getCanonicalName());
-
-        StreamConfigProvider streamConfProvider =
-                getStreamConfigProvider(dlConf, converter);
-
-        // pre-run
-        preRun(dlConf, serverConf);
-
-        Pair<DistributedLogServiceImpl, Server> serverPair = runServer(
-                serverConf,
-                dlConf,
-                dynDlConf,
-                dlUri,
-                converter,
-                routingService,
-                statsProvider,
-                port.or(0),
-                keepAliveLatch,
-                statsReceiver,
-                thriftmux.isPresent(),
-                streamConfProvider,
-                loadAppraiser);
-
-        this.dlService = serverPair.getLeft();
-        this.server = serverPair.getRight();
-
-        // announce the service
-        announcer.announce();
-        // start the routing service after announced
-        routingService.startService();
-        logger.info("Started the routing service.");
-        dlService.startPlacementPolicy();
-        logger.info("Started the placement policy.");
-    }
-
-    protected void preRun(DistributedLogConfiguration conf, ServerConfiguration serverConf) {
-        this.gracefulShutdownMs = serverConf.getGracefulShutdownPeriodMs();
-        if (!serverConf.isDurableWriteEnabled()) {
-            conf.setDurableWriteEnabled(false);
-        }
-    }
-
-    private DynamicDistributedLogConfiguration getServiceDynConf(DistributedLogConfiguration dlConf)
-        throws ConfigurationException {
-        Optional<DynamicDistributedLogConfiguration> dynConf = Optional.absent();
-        if (conf.isPresent()) {
-            DynamicConfigurationFactory configFactory = new DynamicConfigurationFactory(
-                    configExecutorService, dlConf.getDynamicConfigReloadIntervalSec(), TimeUnit.SECONDS);
-            dynConf = configFactory.getDynamicConfiguration(conf.get());
-        }
-        if (dynConf.isPresent()) {
-            return dynConf.get();
-        } else {
-            return ConfUtils.getConstDynConf(dlConf);
-        }
-    }
-
-    private StreamConfigProvider getStreamConfigProvider(DistributedLogConfiguration dlConf,
-                                                         StreamPartitionConverter partitionConverter)
-            throws ConfigurationException {
-        StreamConfigProvider streamConfProvider = new NullStreamConfigProvider();
-        if (streamConf.isPresent() && conf.isPresent()) {
-            String dynConfigPath = streamConf.get();
-            String defaultConfigFile = conf.get();
-            streamConfProvider = new ServiceStreamConfigProvider(
-                    dynConfigPath,
-                    defaultConfigFile,
-                    partitionConverter,
-                    configExecutorService,
-                    dlConf.getDynamicConfigReloadIntervalSec(),
-                    TimeUnit.SECONDS);
-        } else if (conf.isPresent()) {
-            String configFile = conf.get();
-            streamConfProvider = new DefaultStreamConfigProvider(configFile, configExecutorService,
-                    dlConf.getDynamicConfigReloadIntervalSec(), TimeUnit.SECONDS);
-        }
-        return streamConfProvider;
-    }
-
-    static Pair<DistributedLogServiceImpl, Server> runServer(
-            ServerConfiguration serverConf,
-            DistributedLogConfiguration dlConf,
-            URI dlUri,
-            StreamPartitionConverter converter,
-            RoutingService routingService,
-            StatsProvider provider,
-            int port,
-            boolean thriftmux,
-            LoadAppraiser loadAppraiser) throws IOException {
-
-        return runServer(serverConf,
-                dlConf,
-                ConfUtils.getConstDynConf(dlConf),
-                dlUri,
-                converter,
-                routingService,
-                provider,
-                port,
-                new CountDownLatch(0),
-                new NullStatsReceiver(),
-                thriftmux,
-                new NullStreamConfigProvider(),
-                loadAppraiser);
-    }
-
-    static Pair<DistributedLogServiceImpl, Server> runServer(
-            ServerConfiguration serverConf,
-            DistributedLogConfiguration dlConf,
-            DynamicDistributedLogConfiguration dynDlConf,
-            URI dlUri,
-            StreamPartitionConverter partitionConverter,
-            RoutingService routingService,
-            StatsProvider provider,
-            int port,
-            CountDownLatch keepAliveLatch,
-            StatsReceiver statsReceiver,
-            boolean thriftmux,
-            StreamConfigProvider streamConfProvider,
-            LoadAppraiser loadAppraiser) throws IOException {
-        logger.info("Running server @ uri {}.", dlUri);
-
-        boolean perStreamStatsEnabled = serverConf.isPerStreamStatEnabled();
-        StatsLogger perStreamStatsLogger;
-        if (perStreamStatsEnabled) {
-            perStreamStatsLogger = provider.getStatsLogger("stream");
-        } else {
-            perStreamStatsLogger = NullStatsLogger.INSTANCE;
-        }
-
-        // dl service
-        DistributedLogServiceImpl dlService = new DistributedLogServiceImpl(
-            serverConf,
-            dlConf,
-            dynDlConf,
-            streamConfProvider,
-            dlUri,
-            partitionConverter,
-            routingService,
-            provider.getStatsLogger(""),
-            perStreamStatsLogger,
-            keepAliveLatch,
-            loadAppraiser);
-
-        StatsReceiver serviceStatsReceiver = statsReceiver.scope("service");
-        StatsLogger serviceStatsLogger = provider.getStatsLogger("service");
-
-        ServerBuilder serverBuilder = ServerBuilder.get()
-                .name("DistributedLogServer")
-                .codec(ThriftServerFramedCodec.get())
-                .reportTo(statsReceiver)
-                .keepAlive(true)
-                .bindTo(new InetSocketAddress(port));
-
-        if (thriftmux) {
-            logger.info("Using thriftmux.");
-            Tuple2<Transport.Liveness, Stack.Param<Transport.Liveness>> livenessParam = new Transport.Liveness(
-                    Duration.Top(), Duration.Top(), Option.apply((Object) Boolean.valueOf(true))).mk();
-            serverBuilder = serverBuilder.stack(
-                ThriftMuxServer$.MODULE$.configured(livenessParam._1(), livenessParam._2()));
-        }
-
-        logger.info("DistributedLogServer running with the following configuration : \n{}", dlConf.getPropsAsString());
-
-        // starts dl server
-        Server server = ServerBuilder.safeBuild(
-                new ClientIdRequiredFilter<byte[], byte[]>(serviceStatsReceiver).andThen(
-                    new StatsFilter<byte[], byte[]>(serviceStatsLogger).andThen(
-                        new DistributedLogService.Service(dlService, new TBinaryProtocol.Factory()))),
-                serverBuilder);
-
-        logger.info("Started DistributedLog Server.");
-        return Pair.of(dlService, server);
-    }
-
-    static void closeServer(Pair<DistributedLogServiceImpl, Server> pair,
-                            long gracefulShutdownPeriod,
-                            TimeUnit timeUnit) {
-        if (null != pair.getLeft()) {
-            pair.getLeft().shutdown();
-            if (gracefulShutdownPeriod > 0) {
-                try {
-                    timeUnit.sleep(gracefulShutdownPeriod);
-                } catch (InterruptedException e) {
-                    logger.info("Interrupted on waiting service shutting down state propagated to all clients : ", e);
-                }
-            }
-        }
-        if (null != pair.getRight()) {
-            logger.info("Closing dl thrift server.");
-            pair.getRight().close();
-            logger.info("Closed dl thrift server.");
-        }
-    }
-
-    /**
-     * Close the server.
-     */
-    public void close() {
-        if (null != announcer) {
-            try {
-                announcer.unannounce();
-            } catch (IOException e) {
-                logger.warn("Error on unannouncing service : ", e);
-            }
-            announcer.close();
-        }
-        closeServer(Pair.of(dlService, server), gracefulShutdownMs, TimeUnit.MILLISECONDS);
-        routingService.stopService();
-        if (null != statsProvider) {
-            statsProvider.stop();
-        }
-        SchedulerUtils.shutdownScheduler(configExecutorService, 60, TimeUnit.SECONDS);
-        keepAliveLatch.countDown();
-    }
-
-    public void join() throws InterruptedException {
-        keepAliveLatch.await();
-    }
-
-    /**
-     * Running distributedlog server.
-     *
-     * @param uri distributedlog namespace
-     * @param conf distributedlog configuration file location
-     * @param streamConf per stream configuration dir location
-     * @param port listen port
-     * @param statsPort stats port
-     * @param shardId shard id
-     * @param announceServerSet whether to announce itself to server set
-     * @param thriftmux flag to enable thrift mux
-     * @param statsReceiver receiver to receive finagle stats
-     * @param statsProvider provider to receive dl stats
-     * @return distributedlog server
-     * @throws ConfigurationException
-     * @throws IllegalArgumentException
-     * @throws IOException
-     * @throws ClassNotFoundException
-     */
-    public static DistributedLogServer runServer(
-               Optional<String> uri,
-               Optional<String> conf,
-               Optional<String> streamConf,
-               Optional<Integer> port,
-               Optional<Integer> statsPort,
-               Optional<Integer> shardId,
-               Optional<Boolean> announceServerSet,
-               Optional<String> loadAppraiserClass,
-               Optional<Boolean> thriftmux,
-               RoutingService routingService,
-               StatsReceiver statsReceiver,
-               StatsProvider statsProvider)
-        throws ConfigurationException, IllegalArgumentException, IOException, ClassNotFoundException {
-
-        final DistributedLogServer server = new DistributedLogServer(
-                uri,
-                conf,
-                streamConf,
-                port,
-                statsPort,
-                shardId,
-                announceServerSet,
-                loadAppraiserClass,
-                thriftmux,
-                routingService,
-                statsReceiver,
-                statsProvider);
-
-        server.runServer();
-        return server;
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogServerApp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogServerApp.java
deleted file mode 100644
index 55ed84f..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogServerApp.java
+++ /dev/null
@@ -1,187 +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 com.twitter.distributedlog.service;
-
-import static com.google.common.base.Preconditions.checkArgument;
-import static com.twitter.distributedlog.util.CommandLineUtils.getOptionalBooleanArg;
-import static com.twitter.distributedlog.util.CommandLineUtils.getOptionalIntegerArg;
-import static com.twitter.distributedlog.util.CommandLineUtils.getOptionalStringArg;
-
-import com.google.common.base.Function;
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.client.routing.RoutingService;
-import com.twitter.distributedlog.client.routing.RoutingUtils;
-import com.twitter.distributedlog.client.serverset.DLZkServerSet;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.io.File;
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.util.Arrays;
-import java.util.concurrent.TimeUnit;
-import javax.annotation.Nullable;
-import org.apache.bookkeeper.stats.NullStatsProvider;
-import org.apache.bookkeeper.stats.StatsProvider;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.cli.BasicParser;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.configuration.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * The launcher of the distributedlog proxy server.
- */
-public class DistributedLogServerApp {
-
-    private static final Logger logger = LoggerFactory.getLogger(DistributedLogServerApp.class);
-
-    private static final String USAGE = "DistributedLogServerApp [-u <uri>] [-c <conf>]";
-    private final String[] args;
-    private final Options options = new Options();
-
-    private DistributedLogServerApp(String[] args) {
-        this.args = args;
-
-        // prepare options
-        options.addOption("u", "uri", true, "DistributedLog URI");
-        options.addOption("c", "conf", true, "DistributedLog Configuration File");
-        options.addOption("sc", "stream-conf", true, "Per Stream Configuration Directory");
-        options.addOption("p", "port", true, "DistributedLog Server Port");
-        options.addOption("sp", "stats-port", true, "DistributedLog Stats Port");
-        options.addOption("pd", "stats-provider", true, "DistributedLog Stats Provider");
-        options.addOption("si", "shard-id", true, "DistributedLog Shard ID");
-        options.addOption("a", "announce", false, "ServerSet Path to Announce");
-        options.addOption("la", "load-appraiser", true, "LoadAppraiser Implementation to Use");
-        options.addOption("mx", "thriftmux", false, "Is thriftmux enabled");
-    }
-
-    private void printUsage() {
-        HelpFormatter helpFormatter = new HelpFormatter();
-        helpFormatter.printHelp(USAGE, options);
-    }
-
-    private void run() {
-        try {
-            logger.info("Running distributedlog server : args = {}", Arrays.toString(args));
-            BasicParser parser = new BasicParser();
-            CommandLine cmdline = parser.parse(options, args);
-            runCmd(cmdline);
-        } catch (ParseException pe) {
-            logger.error("Argument error : {}", pe.getMessage());
-            printUsage();
-            Runtime.getRuntime().exit(-1);
-        } catch (IllegalArgumentException iae) {
-            logger.error("Argument error : {}", iae.getMessage());
-            printUsage();
-            Runtime.getRuntime().exit(-1);
-        } catch (ConfigurationException ce) {
-            logger.error("Configuration error : {}", ce.getMessage());
-            printUsage();
-            Runtime.getRuntime().exit(-1);
-        } catch (IOException ie) {
-            logger.error("Failed to start distributedlog server : ", ie);
-            Runtime.getRuntime().exit(-1);
-        } catch (ClassNotFoundException cnf) {
-          logger.error("Failed to start distributedlog server : ", cnf);
-          Runtime.getRuntime().exit(-1);
-        }
-    }
-
-    private void runCmd(CommandLine cmdline)
-        throws IllegalArgumentException, IOException, ConfigurationException, ClassNotFoundException {
-        final StatsReceiver statsReceiver = NullStatsReceiver.get();
-        Optional<String> confOptional = getOptionalStringArg(cmdline, "c");
-        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
-        if (confOptional.isPresent()) {
-            String configFile = confOptional.get();
-            try {
-                dlConf.loadConf(new File(configFile).toURI().toURL());
-            } catch (ConfigurationException e) {
-                throw new IllegalArgumentException("Failed to load distributedlog configuration from "
-                    + configFile + ".");
-            } catch (MalformedURLException e) {
-                throw new IllegalArgumentException("Failed to load distributedlog configuration from malformed "
-                        + configFile + ".");
-            }
-        }
-        // load the stats provider
-        final StatsProvider statsProvider = getOptionalStringArg(cmdline, "pd")
-                .transform(new Function<String, StatsProvider>() {
-                    @Nullable
-                    @Override
-                    public StatsProvider apply(@Nullable String name) {
-                        return ReflectionUtils.newInstance(name, StatsProvider.class);
-                    }
-                }).or(new NullStatsProvider());
-
-        final Optional<String> uriOption = getOptionalStringArg(cmdline, "u");
-        checkArgument(uriOption.isPresent(), "No distributedlog uri provided.");
-        URI dlUri = URI.create(uriOption.get());
-
-        DLZkServerSet serverSet = DLZkServerSet.of(dlUri, (int) TimeUnit.SECONDS.toMillis(60));
-        RoutingService routingService = RoutingUtils.buildRoutingService(serverSet.getServerSet())
-                .statsReceiver(statsReceiver.scope("routing"))
-                .build();
-
-        final DistributedLogServer server = DistributedLogServer.runServer(
-                uriOption,
-                confOptional,
-                getOptionalStringArg(cmdline, "sc"),
-                getOptionalIntegerArg(cmdline, "p"),
-                getOptionalIntegerArg(cmdline, "sp"),
-                getOptionalIntegerArg(cmdline, "si"),
-                getOptionalBooleanArg(cmdline, "a"),
-                getOptionalStringArg(cmdline, "la"),
-                getOptionalBooleanArg(cmdline, "mx"),
-                routingService,
-                statsReceiver,
-                statsProvider);
-
-        Runtime.getRuntime().addShutdownHook(new Thread() {
-            @Override
-            public void run() {
-                logger.info("Closing DistributedLog Server.");
-                server.close();
-                logger.info("Closed DistributedLog Server.");
-                statsProvider.stop();
-            }
-        });
-
-        try {
-            server.join();
-        } catch (InterruptedException e) {
-            logger.warn("Interrupted when waiting distributedlog server to be finished : ", e);
-        }
-
-        logger.info("DistributedLog Service Interrupted.");
-        server.close();
-        logger.info("Closed DistributedLog Server.");
-        statsProvider.stop();
-    }
-
-    public static void main(String[] args) {
-        final DistributedLogServerApp launcher = new DistributedLogServerApp(args);
-        launcher.run();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogServiceImpl.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogServiceImpl.java
deleted file mode 100644
index db1346e..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/DistributedLogServiceImpl.java
+++ /dev/null
@@ -1,794 +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 com.twitter.distributedlog.service;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Stopwatch;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.common.net.InetSocketAddressHelper;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.client.resolver.DefaultRegionResolver;
-import com.twitter.distributedlog.client.resolver.RegionResolver;
-import com.twitter.distributedlog.client.routing.RoutingService;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.RegionUnavailableException;
-import com.twitter.distributedlog.exceptions.ServiceUnavailableException;
-import com.twitter.distributedlog.exceptions.StreamUnavailableException;
-import com.twitter.distributedlog.exceptions.TooManyStreamsException;
-import com.twitter.distributedlog.feature.AbstractFeatureProvider;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.rate.MovingAverageRate;
-import com.twitter.distributedlog.rate.MovingAverageRateFactory;
-import com.twitter.distributedlog.service.config.ServerConfiguration;
-import com.twitter.distributedlog.service.config.StreamConfigProvider;
-import com.twitter.distributedlog.service.placement.LeastLoadPlacementPolicy;
-import com.twitter.distributedlog.service.placement.LoadAppraiser;
-import com.twitter.distributedlog.service.placement.PlacementPolicy;
-import com.twitter.distributedlog.service.placement.ZKPlacementStateManager;
-import com.twitter.distributedlog.service.stream.BulkWriteOp;
-import com.twitter.distributedlog.service.stream.DeleteOp;
-import com.twitter.distributedlog.service.stream.HeartbeatOp;
-import com.twitter.distributedlog.service.stream.ReleaseOp;
-import com.twitter.distributedlog.service.stream.Stream;
-import com.twitter.distributedlog.service.stream.StreamFactory;
-import com.twitter.distributedlog.service.stream.StreamFactoryImpl;
-import com.twitter.distributedlog.service.stream.StreamManager;
-import com.twitter.distributedlog.service.stream.StreamManagerImpl;
-import com.twitter.distributedlog.service.stream.StreamOp;
-import com.twitter.distributedlog.service.stream.StreamOpStats;
-import com.twitter.distributedlog.service.stream.TruncateOp;
-import com.twitter.distributedlog.service.stream.WriteOp;
-import com.twitter.distributedlog.service.stream.WriteOpWithPayload;
-import com.twitter.distributedlog.service.stream.admin.CreateOp;
-import com.twitter.distributedlog.service.stream.admin.StreamAdminOp;
-import com.twitter.distributedlog.service.stream.limiter.ServiceRequestLimiter;
-import com.twitter.distributedlog.service.streamset.StreamPartitionConverter;
-import com.twitter.distributedlog.service.utils.ServerUtils;
-import com.twitter.distributedlog.thrift.service.BulkWriteResponse;
-import com.twitter.distributedlog.thrift.service.ClientInfo;
-import com.twitter.distributedlog.thrift.service.DistributedLogService;
-import com.twitter.distributedlog.thrift.service.HeartbeatOptions;
-import com.twitter.distributedlog.thrift.service.ResponseHeader;
-import com.twitter.distributedlog.thrift.service.ServerInfo;
-import com.twitter.distributedlog.thrift.service.ServerStatus;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.distributedlog.thrift.service.WriteContext;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.SchedulerUtils;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Function;
-import com.twitter.util.Function0;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.ScheduledThreadPoolTimer;
-import com.twitter.util.Timer;
-import java.io.IOException;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.BoxedUnit;
-
-/**
- * Implementation of distributedlog thrift service.
- */
-public class DistributedLogServiceImpl implements DistributedLogService.ServiceIface,
-                                                  FatalErrorHandler {
-
-    private static final Logger logger = LoggerFactory.getLogger(DistributedLogServiceImpl.class);
-
-    private static final int MOVING_AVERAGE_WINDOW_SECS = 60;
-
-    private final ServerConfiguration serverConfig;
-    private final DistributedLogConfiguration dlConfig;
-    private final DistributedLogNamespace dlNamespace;
-    private final int serverRegionId;
-    private final PlacementPolicy placementPolicy;
-    private ServerStatus serverStatus = ServerStatus.WRITE_AND_ACCEPT;
-    private final ReentrantReadWriteLock closeLock =
-            new ReentrantReadWriteLock();
-    private final CountDownLatch keepAliveLatch;
-    private final byte dlsnVersion;
-    private final String clientId;
-    private final OrderedScheduler scheduler;
-    private final AccessControlManager accessControlManager;
-    private final StreamConfigProvider streamConfigProvider;
-    private final StreamManager streamManager;
-    private final StreamFactory streamFactory;
-    private final RoutingService routingService;
-    private final RegionResolver regionResolver;
-    private final MovingAverageRateFactory movingAvgFactory;
-    private final MovingAverageRate windowedRps;
-    private final MovingAverageRate windowedBps;
-    private final ServiceRequestLimiter limiter;
-    private final Timer timer;
-    private final HashedWheelTimer requestTimer;
-
-    // Features
-    private final FeatureProvider featureProvider;
-    private final Feature featureRegionStopAcceptNewStream;
-    private final Feature featureChecksumDisabled;
-    private final Feature limiterDisabledFeature;
-
-    // Stats
-    private final StatsLogger statsLogger;
-    private final StatsLogger perStreamStatsLogger;
-    private final StreamPartitionConverter streamPartitionConverter;
-    private final StreamOpStats streamOpStats;
-    private final Counter bulkWritePendingStat;
-    private final Counter writePendingStat;
-    private final Counter redirects;
-    private final Counter receivedRecordCounter;
-    private final StatsLogger statusCodeStatLogger;
-    private final ConcurrentHashMap<StatusCode, Counter> statusCodeCounters =
-            new ConcurrentHashMap<StatusCode, Counter>();
-    private final Counter statusCodeTotal;
-    private final Gauge<Number> proxyStatusGauge;
-    private final Gauge<Number> movingAvgRpsGauge;
-    private final Gauge<Number> movingAvgBpsGauge;
-    private final Gauge<Number> streamAcquiredGauge;
-    private final Gauge<Number> streamCachedGauge;
-    private final int shard;
-
-    DistributedLogServiceImpl(ServerConfiguration serverConf,
-                              DistributedLogConfiguration dlConf,
-                              DynamicDistributedLogConfiguration dynDlConf,
-                              StreamConfigProvider streamConfigProvider,
-                              URI uri,
-                              StreamPartitionConverter converter,
-                              RoutingService routingService,
-                              StatsLogger statsLogger,
-                              StatsLogger perStreamStatsLogger,
-                              CountDownLatch keepAliveLatch,
-                              LoadAppraiser loadAppraiser)
-            throws IOException {
-        // Configuration.
-        this.serverConfig = serverConf;
-        this.dlConfig = dlConf;
-        this.perStreamStatsLogger = perStreamStatsLogger;
-        this.dlsnVersion = serverConf.getDlsnVersion();
-        this.serverRegionId = serverConf.getRegionId();
-        this.streamPartitionConverter = converter;
-        int serverPort = serverConf.getServerPort();
-        this.shard = serverConf.getServerShardId();
-        int numThreads = serverConf.getServerThreads();
-        this.clientId = DLSocketAddress.toLockId(DLSocketAddress.getSocketAddress(serverPort), shard);
-        String allocatorPoolName = ServerUtils.getLedgerAllocatorPoolName(
-            serverRegionId,
-            shard,
-            serverConf.isUseHostnameAsAllocatorPoolName());
-        dlConf.setLedgerAllocatorPoolName(allocatorPoolName);
-        this.featureProvider = AbstractFeatureProvider.getFeatureProvider("", dlConf, statsLogger.scope("features"));
-        if (this.featureProvider instanceof AbstractFeatureProvider) {
-            ((AbstractFeatureProvider) featureProvider).start();
-        }
-
-        // Build the namespace
-        this.dlNamespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(dlConf)
-                .uri(uri)
-                .statsLogger(statsLogger)
-                .featureProvider(this.featureProvider)
-                .clientId(clientId)
-                .regionId(serverRegionId)
-                .build();
-        this.accessControlManager = this.dlNamespace.createAccessControlManager();
-        this.keepAliveLatch = keepAliveLatch;
-        this.streamConfigProvider = streamConfigProvider;
-
-        // Stats pertaining to stream op execution
-        this.streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-
-        // Executor Service.
-        this.scheduler = OrderedScheduler.newBuilder()
-                .corePoolSize(numThreads)
-                .name("DistributedLogService-Executor")
-                .traceTaskExecution(true)
-                .statsLogger(statsLogger.scope("scheduler"))
-                .build();
-
-        // Timer, kept separate to ensure reliability of timeouts.
-        this.requestTimer = new HashedWheelTimer(
-            new ThreadFactoryBuilder().setNameFormat("DLServiceTimer-%d").build(),
-            dlConf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS,
-            dlConf.getTimeoutTimerNumTicks());
-
-        // Creating and managing Streams
-        this.streamFactory = new StreamFactoryImpl(clientId,
-                streamOpStats,
-                serverConf,
-                dlConf,
-                featureProvider,
-                streamConfigProvider,
-                converter,
-                dlNamespace,
-                scheduler,
-                this,
-                requestTimer);
-        this.streamManager = new StreamManagerImpl(
-                clientId,
-                dlConf,
-                scheduler,
-                streamFactory,
-                converter,
-                streamConfigProvider,
-                dlNamespace);
-        this.routingService = routingService;
-        this.regionResolver = new DefaultRegionResolver();
-
-        // Service features
-        this.featureRegionStopAcceptNewStream = this.featureProvider.getFeature(
-                ServerFeatureKeys.REGION_STOP_ACCEPT_NEW_STREAM.name().toLowerCase());
-        this.featureChecksumDisabled = this.featureProvider.getFeature(
-                ServerFeatureKeys.SERVICE_CHECKSUM_DISABLED.name().toLowerCase());
-        this.limiterDisabledFeature = this.featureProvider.getFeature(
-                ServerFeatureKeys.SERVICE_GLOBAL_LIMITER_DISABLED.name().toLowerCase());
-
-        // Resource limiting
-        this.timer = new ScheduledThreadPoolTimer(1, "timer", true);
-        this.movingAvgFactory = new MovingAverageRateFactory(timer);
-        this.windowedRps = movingAvgFactory.create(MOVING_AVERAGE_WINDOW_SECS);
-        this.windowedBps = movingAvgFactory.create(MOVING_AVERAGE_WINDOW_SECS);
-        this.limiter = new ServiceRequestLimiter(
-                dynDlConf,
-                streamOpStats.baseScope("service_limiter"),
-                windowedRps,
-                windowedBps,
-                streamManager,
-                limiterDisabledFeature);
-
-        this.placementPolicy = new LeastLoadPlacementPolicy(
-            loadAppraiser,
-            routingService,
-            dlNamespace,
-            new ZKPlacementStateManager(uri, dlConf, statsLogger),
-            Duration.fromSeconds(serverConf.getResourcePlacementRefreshInterval()),
-            statsLogger);
-        logger.info("placement started");
-
-        // Stats
-        this.statsLogger = statsLogger;
-
-        // Gauges for server status/health
-        this.proxyStatusGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return ServerStatus.DOWN == serverStatus ? -1 : (featureRegionStopAcceptNewStream.isAvailable()
-                    ? 3 : (ServerStatus.WRITE_AND_ACCEPT == serverStatus ? 1 : 2));
-            }
-        };
-        this.movingAvgRpsGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return windowedRps.get();
-            }
-        };
-        this.movingAvgBpsGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return windowedBps.get();
-            }
-        };
-        // Gauges for streams
-        this.streamAcquiredGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return streamManager.numAcquired();
-            }
-        };
-        this.streamCachedGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return streamManager.numCached();
-            }
-        };
-
-        // Stats on server
-        statsLogger.registerGauge("proxy_status", proxyStatusGauge);
-        // Global moving average rps
-        statsLogger.registerGauge("moving_avg_rps", movingAvgRpsGauge);
-        // Global moving average bps
-        statsLogger.registerGauge("moving_avg_bps", movingAvgBpsGauge);
-        // Stats on requests
-        this.bulkWritePendingStat = streamOpStats.requestPendingCounter("bulkWritePending");
-        this.writePendingStat = streamOpStats.requestPendingCounter("writePending");
-        this.redirects = streamOpStats.requestCounter("redirect");
-        this.statusCodeStatLogger = streamOpStats.requestScope("statuscode");
-        this.statusCodeTotal = streamOpStats.requestCounter("statuscode_count");
-        this.receivedRecordCounter = streamOpStats.recordsCounter("received");
-
-        // Stats for streams
-        StatsLogger streamsStatsLogger = statsLogger.scope("streams");
-        streamsStatsLogger.registerGauge("acquired", this.streamAcquiredGauge);
-        streamsStatsLogger.registerGauge("cached", this.streamCachedGauge);
-
-        // Setup complete
-        logger.info("Running distributedlog server : client id {}, allocator pool {}, perstream stat {},"
-            + " dlsn version {}.",
-            new Object[] { clientId, allocatorPoolName, serverConf.isPerStreamStatEnabled(), dlsnVersion });
-    }
-
-    private void countStatusCode(StatusCode code) {
-        Counter counter = statusCodeCounters.get(code);
-        if (null == counter) {
-            counter = statusCodeStatLogger.getCounter(code.name());
-            Counter oldCounter = statusCodeCounters.putIfAbsent(code, counter);
-            if (null != oldCounter) {
-                counter = oldCounter;
-            }
-        }
-        counter.inc();
-        statusCodeTotal.inc();
-    }
-
-    @Override
-    public Future<ServerInfo> handshake() {
-        return handshakeWithClientInfo(new ClientInfo());
-    }
-
-    @Override
-    public Future<ServerInfo> handshakeWithClientInfo(ClientInfo clientInfo) {
-        ServerInfo serverInfo = new ServerInfo();
-        closeLock.readLock().lock();
-        try {
-            serverInfo.setServerStatus(serverStatus);
-        } finally {
-            closeLock.readLock().unlock();
-        }
-
-        if (clientInfo.isSetGetOwnerships() && !clientInfo.isGetOwnerships()) {
-            return Future.value(serverInfo);
-        }
-
-        Optional<String> regex = Optional.absent();
-        if (clientInfo.isSetStreamNameRegex()) {
-            regex = Optional.of(clientInfo.getStreamNameRegex());
-        }
-
-        Map<String, String> ownershipMap = streamManager.getStreamOwnershipMap(regex);
-        serverInfo.setOwnerships(ownershipMap);
-        return Future.value(serverInfo);
-    }
-
-    @VisibleForTesting
-    Stream getLogWriter(String stream) throws IOException {
-        Stream writer = streamManager.getStream(stream);
-        if (null == writer) {
-            closeLock.readLock().lock();
-            try {
-                if (featureRegionStopAcceptNewStream.isAvailable()) {
-                    // accept new stream is disabled in current dc
-                    throw new RegionUnavailableException("Region is unavailable right now.");
-                } else if (!(ServerStatus.WRITE_AND_ACCEPT == serverStatus)) {
-                    // if it is closed, we would not acquire stream again.
-                    return null;
-                }
-                writer = streamManager.getOrCreateStream(stream, true);
-            } finally {
-                closeLock.readLock().unlock();
-            }
-        }
-        return writer;
-    }
-
-    // Service interface methods
-
-    @Override
-    public Future<WriteResponse> write(final String stream, ByteBuffer data) {
-        receivedRecordCounter.inc();
-        return doWrite(stream, data, null /* checksum */, false);
-    }
-
-    @Override
-    public Future<BulkWriteResponse> writeBulkWithContext(final String stream,
-                                                          List<ByteBuffer> data,
-                                                          WriteContext ctx) {
-        bulkWritePendingStat.inc();
-        receivedRecordCounter.add(data.size());
-        BulkWriteOp op = new BulkWriteOp(stream, data, statsLogger, perStreamStatsLogger, streamPartitionConverter,
-            getChecksum(ctx), featureChecksumDisabled, accessControlManager);
-        executeStreamOp(op);
-        return op.result().ensure(new Function0<BoxedUnit>() {
-            public BoxedUnit apply() {
-                bulkWritePendingStat.dec();
-                return null;
-            }
-        });
-    }
-
-    @Override
-    public Future<WriteResponse> writeWithContext(final String stream, ByteBuffer data, WriteContext ctx) {
-        return doWrite(stream, data, getChecksum(ctx), ctx.isIsRecordSet());
-    }
-
-    @Override
-    public Future<WriteResponse> heartbeat(String stream, WriteContext ctx) {
-        HeartbeatOp op = new HeartbeatOp(stream, statsLogger, perStreamStatsLogger, dlsnVersion, getChecksum(ctx),
-            featureChecksumDisabled, accessControlManager);
-        executeStreamOp(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<WriteResponse> heartbeatWithOptions(String stream, WriteContext ctx, HeartbeatOptions options) {
-        HeartbeatOp op = new HeartbeatOp(stream, statsLogger, perStreamStatsLogger, dlsnVersion, getChecksum(ctx),
-            featureChecksumDisabled, accessControlManager);
-        if (options.isSendHeartBeatToReader()) {
-            op.setWriteControlRecord(true);
-        }
-        executeStreamOp(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<WriteResponse> truncate(String stream, String dlsn, WriteContext ctx) {
-        TruncateOp op = new TruncateOp(
-            stream,
-            DLSN.deserialize(dlsn),
-            statsLogger,
-            perStreamStatsLogger,
-            getChecksum(ctx),
-            featureChecksumDisabled,
-            accessControlManager);
-        executeStreamOp(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<WriteResponse> delete(String stream, WriteContext ctx) {
-        DeleteOp op = new DeleteOp(stream, statsLogger, perStreamStatsLogger, streamManager, getChecksum(ctx),
-            featureChecksumDisabled, accessControlManager);
-        executeStreamOp(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<WriteResponse> release(String stream, WriteContext ctx) {
-        ReleaseOp op = new ReleaseOp(stream, statsLogger, perStreamStatsLogger, streamManager, getChecksum(ctx),
-            featureChecksumDisabled, accessControlManager);
-        executeStreamOp(op);
-        return op.result();
-    }
-
-    @Override
-    public Future<WriteResponse> create(String stream, WriteContext ctx) {
-        CreateOp op = new CreateOp(stream, statsLogger, streamManager, getChecksum(ctx), featureChecksumDisabled);
-        return executeStreamAdminOp(op);
-    }
-
-    //
-    // Ownership RPC
-    //
-
-    @Override
-    public Future<WriteResponse> getOwner(String streamName, WriteContext ctx) {
-        if (streamManager.isAcquired(streamName)) {
-            // the stream is already acquired
-            return Future.value(new WriteResponse(ResponseUtils.ownerToHeader(clientId)));
-        }
-
-        return placementPolicy.placeStream(streamName).map(new Function<String, WriteResponse>() {
-            @Override
-            public WriteResponse apply(String server) {
-                String host = DLSocketAddress.toLockId(InetSocketAddressHelper.parse(server), -1);
-                return new WriteResponse(ResponseUtils.ownerToHeader(host));
-            }
-        });
-    }
-
-
-    //
-    // Admin RPCs
-    //
-
-    @Override
-    public Future<Void> setAcceptNewStream(boolean enabled) {
-        closeLock.writeLock().lock();
-        try {
-            logger.info("Set AcceptNewStream = {}", enabled);
-            if (ServerStatus.DOWN != serverStatus) {
-                if (enabled) {
-                    serverStatus = ServerStatus.WRITE_AND_ACCEPT;
-                } else {
-                    serverStatus = ServerStatus.WRITE_ONLY;
-                }
-            }
-        } finally {
-            closeLock.writeLock().unlock();
-        }
-        return Future.Void();
-    }
-
-    private Future<WriteResponse> doWrite(final String name,
-                                          ByteBuffer data,
-                                          Long checksum,
-                                          boolean isRecordSet) {
-        writePendingStat.inc();
-        receivedRecordCounter.inc();
-        WriteOp op = newWriteOp(name, data, checksum, isRecordSet);
-        executeStreamOp(op);
-        return op.result().ensure(new Function0<BoxedUnit>() {
-            public BoxedUnit apply() {
-                writePendingStat.dec();
-                return null;
-            }
-        });
-    }
-
-    private Long getChecksum(WriteContext ctx) {
-        return ctx.isSetCrc32() ? ctx.getCrc32() : null;
-    }
-
-    private Future<WriteResponse> executeStreamAdminOp(final StreamAdminOp op) {
-        try {
-            op.preExecute();
-        } catch (DLException dle) {
-            return Future.exception(dle);
-        }
-        return op.execute();
-    }
-
-    private void executeStreamOp(final StreamOp op) {
-
-        // Must attach this as early as possible--returning before this point will cause us to
-        // lose the status code.
-        op.responseHeader().addEventListener(new FutureEventListener<ResponseHeader>() {
-            @Override
-            public void onSuccess(ResponseHeader header) {
-                if (header.getLocation() != null || header.getCode() == StatusCode.FOUND) {
-                    redirects.inc();
-                }
-                countStatusCode(header.getCode());
-            }
-            @Override
-            public void onFailure(Throwable cause) {
-            }
-        });
-
-        try {
-            // Apply the request limiter
-            limiter.apply(op);
-
-            // Execute per-op pre-exec code
-            op.preExecute();
-
-        } catch (TooManyStreamsException e) {
-            // Translate to StreamUnavailableException to ensure that the client will redirect
-            // to a different host. Ideally we would be able to return TooManyStreamsException,
-            // but the way exception handling works right now we can't control the handling in
-            // the client because client changes deploy very slowly.
-            op.fail(new StreamUnavailableException(e.getMessage()));
-            return;
-        } catch (Exception e) {
-            op.fail(e);
-            return;
-        }
-
-        Stream stream;
-        try {
-            stream = getLogWriter(op.streamName());
-        } catch (RegionUnavailableException rue) {
-            // redirect the requests to other region
-            op.fail(new RegionUnavailableException("Region " + serverRegionId + " is unavailable."));
-            return;
-        } catch (IOException e) {
-            op.fail(e);
-            return;
-        }
-        if (null == stream) {
-            // redirect the requests when stream is unavailable.
-            op.fail(new ServiceUnavailableException("Server " + clientId + " is closed."));
-            return;
-        }
-
-        if (op instanceof WriteOpWithPayload) {
-            WriteOpWithPayload writeOp = (WriteOpWithPayload) op;
-            windowedBps.add(writeOp.getPayloadSize());
-            windowedRps.inc();
-        }
-
-        stream.submit(op);
-    }
-
-    void shutdown() {
-        try {
-            closeLock.writeLock().lock();
-            try {
-                if (ServerStatus.DOWN == serverStatus) {
-                    return;
-                }
-                serverStatus = ServerStatus.DOWN;
-            } finally {
-                closeLock.writeLock().unlock();
-            }
-
-            streamManager.close();
-            movingAvgFactory.close();
-            limiter.close();
-
-            Stopwatch closeStreamsStopwatch = Stopwatch.createStarted();
-
-            Future<List<Void>> closeResult = streamManager.closeStreams();
-            logger.info("Waiting for closing all streams ...");
-            try {
-                Await.result(closeResult, Duration.fromTimeUnit(5, TimeUnit.MINUTES));
-                logger.info("Closed all streams in {} millis.",
-                        closeStreamsStopwatch.elapsed(TimeUnit.MILLISECONDS));
-            } catch (InterruptedException e) {
-                logger.warn("Interrupted on waiting for closing all streams : ", e);
-                Thread.currentThread().interrupt();
-            } catch (Exception e) {
-                logger.warn("Sorry, we didn't close all streams gracefully in 5 minutes : ", e);
-            }
-
-            // shutdown the dl namespace
-            logger.info("Closing distributedlog namespace ...");
-            dlNamespace.close();
-            logger.info("Closed distributedlog namespace .");
-
-            // Stop the feature provider
-            if (this.featureProvider instanceof AbstractFeatureProvider) {
-                ((AbstractFeatureProvider) featureProvider).stop();
-            }
-
-            // Stop the timer.
-            timer.stop();
-            placementPolicy.close();
-
-            // clean up gauge
-            unregisterGauge();
-
-            // shutdown the executor after requesting closing streams.
-            SchedulerUtils.shutdownScheduler(scheduler, 60, TimeUnit.SECONDS);
-        } catch (Exception ex) {
-            logger.info("Exception while shutting down distributedlog service.");
-        } finally {
-            // release the keepAliveLatch in case shutdown is called from a shutdown hook.
-            keepAliveLatch.countDown();
-            logger.info("Finished shutting down distributedlog service.");
-        }
-    }
-
-    protected void startPlacementPolicy() {
-        this.placementPolicy.start(shard == 0);
-    }
-
-    @Override
-    public void notifyFatalError() {
-        triggerShutdown();
-    }
-
-    private void triggerShutdown() {
-        // release the keepAliveLatch to let the main thread shutdown the whole service.
-        logger.info("Releasing KeepAlive Latch to trigger shutdown ...");
-        keepAliveLatch.countDown();
-        logger.info("Released KeepAlive Latch. Main thread will shut the service down.");
-    }
-
-    // Test methods.
-
-    private DynamicDistributedLogConfiguration getDynConf(String streamName) {
-        Optional<DynamicDistributedLogConfiguration> dynDlConf =
-                streamConfigProvider.getDynamicStreamConfig(streamName);
-        if (dynDlConf.isPresent()) {
-            return dynDlConf.get();
-        } else {
-            return ConfUtils.getConstDynConf(dlConfig);
-        }
-    }
-
-    /**
-     * clean up the gauge before we close to help GC.
-     */
-    private void unregisterGauge(){
-        this.statsLogger.unregisterGauge("proxy_status", this.proxyStatusGauge);
-        this.statsLogger.unregisterGauge("moving_avg_rps", this.movingAvgRpsGauge);
-        this.statsLogger.unregisterGauge("moving_avg_bps", this.movingAvgBpsGauge);
-        this.statsLogger.unregisterGauge("acquired", this.streamAcquiredGauge);
-        this.statsLogger.unregisterGauge("cached", this.streamCachedGauge);
-    }
-
-    @VisibleForTesting
-    Stream newStream(String name) throws IOException {
-        return streamManager.getOrCreateStream(name, false);
-    }
-
-    @VisibleForTesting
-    WriteOp newWriteOp(String stream, ByteBuffer data, Long checksum) {
-        return newWriteOp(stream, data, checksum, false);
-    }
-
-    @VisibleForTesting
-    RoutingService getRoutingService() {
-        return this.routingService;
-    }
-
-    @VisibleForTesting
-    DLSocketAddress getServiceAddress() throws IOException {
-        return DLSocketAddress.deserialize(clientId);
-    }
-
-    WriteOp newWriteOp(String stream,
-                       ByteBuffer data,
-                       Long checksum,
-                       boolean isRecordSet) {
-        return new WriteOp(stream, data, statsLogger, perStreamStatsLogger, streamPartitionConverter,
-            serverConfig, dlsnVersion, checksum, isRecordSet, featureChecksumDisabled,
-            accessControlManager);
-    }
-
-    @VisibleForTesting
-    Future<List<Void>> closeStreams() {
-        return streamManager.closeStreams();
-    }
-
-    @VisibleForTesting
-    public DistributedLogNamespace getDistributedLogNamespace() {
-        return dlNamespace;
-    }
-
-    @VisibleForTesting
-    StreamManager getStreamManager() {
-        return streamManager;
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/FatalErrorHandler.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/FatalErrorHandler.java
deleted file mode 100644
index d6922b9..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/FatalErrorHandler.java
+++ /dev/null
@@ -1,30 +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 com.twitter.distributedlog.service;
-
-/**
- * Implement handling for an unrecoverable error.
- */
-public interface FatalErrorHandler {
-
-    /**
-     * This method is invoked when an unrecoverable error has occurred
-     * and no progress can be made. It should implement a shutdown routine.
-     */
-    void notifyFatalError();
-}
\ No newline at end of file
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/MonitorService.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/MonitorService.java
deleted file mode 100644
index 4ff5b87..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/MonitorService.java
+++ /dev/null
@@ -1,469 +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 com.twitter.distributedlog.service;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.common.base.Optional;
-import com.google.common.base.Stopwatch;
-import com.google.common.collect.Sets;
-import com.google.common.hash.HashFunction;
-import com.google.common.hash.Hashing;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.callback.LogSegmentListener;
-import com.twitter.distributedlog.callback.NamespaceListener;
-import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
-import com.twitter.distributedlog.client.serverset.DLZkServerSet;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.stats.Stat;
-import com.twitter.finagle.stats.StatsReceiver;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
-import java.io.File;
-import java.io.IOException;
-import java.net.MalformedURLException;
-import java.net.URI;
-import java.util.ArrayList;
-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.CountDownLatch;
-import java.util.concurrent.Executors;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.StatsProvider;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.lang.StringUtils;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Monitor Service.
- */
-public class MonitorService implements NamespaceListener {
-
-    private static final Logger logger = LoggerFactory.getLogger(MonitorService.class);
-
-    private DistributedLogNamespace dlNamespace = null;
-    private MonitorServiceClient dlClient = null;
-    private DLZkServerSet[] zkServerSets = null;
-    private final ScheduledExecutorService executorService =
-            Executors.newScheduledThreadPool(Runtime.getRuntime().availableProcessors());
-    private final CountDownLatch keepAliveLatch = new CountDownLatch(1);
-    private final Map<String, StreamChecker> knownStreams = new HashMap<String, StreamChecker>();
-
-    // Settings
-    private int regionId = DistributedLogConstants.LOCAL_REGION_ID;
-    private int interval = 100;
-    private String streamRegex = null;
-    private boolean watchNamespaceChanges = false;
-    private boolean handshakeWithClientInfo = false;
-    private int heartbeatEveryChecks = 0;
-    private int instanceId = -1;
-    private int totalInstances = -1;
-    private boolean isThriftMux = false;
-
-    // Options
-    private final Optional<String> uriArg;
-    private final Optional<String> confFileArg;
-    private final Optional<String> serverSetArg;
-    private final Optional<Integer> intervalArg;
-    private final Optional<Integer> regionIdArg;
-    private final Optional<String> streamRegexArg;
-    private final Optional<Integer> instanceIdArg;
-    private final Optional<Integer> totalInstancesArg;
-    private final Optional<Integer> heartbeatEveryChecksArg;
-    private final Optional<Boolean> handshakeWithClientInfoArg;
-    private final Optional<Boolean> watchNamespaceChangesArg;
-    private final Optional<Boolean> isThriftMuxArg;
-
-    // Stats
-    private final StatsProvider statsProvider;
-    private final StatsReceiver statsReceiver;
-    private final StatsReceiver monitorReceiver;
-    private final Stat successStat;
-    private final Stat failureStat;
-    private final Gauge<Number> numOfStreamsGauge;
-    // Hash Function
-    private final HashFunction hashFunction = Hashing.md5();
-
-    class StreamChecker implements Runnable, FutureEventListener<Void>, LogSegmentListener {
-        private final String name;
-        private volatile boolean closed = false;
-        private volatile boolean checking = false;
-        private final Stopwatch stopwatch = Stopwatch.createUnstarted();
-        private DistributedLogManager dlm = null;
-        private int numChecks = 0;
-
-        StreamChecker(String name) {
-            this.name = name;
-        }
-
-        @Override
-        public void run() {
-            if (null == dlm) {
-                try {
-                    dlm = dlNamespace.openLog(name);
-                    dlm.registerListener(this);
-                } catch (IOException e) {
-                    if (null != dlm) {
-                        try {
-                            dlm.close();
-                        } catch (IOException e1) {
-                            logger.error("Failed to close dlm for {} : ", name, e1);
-                        }
-                        dlm = null;
-                    }
-                    executorService.schedule(this, interval, TimeUnit.MILLISECONDS);
-                }
-            } else {
-                stopwatch.reset().start();
-                boolean sendHeartBeat;
-                if (heartbeatEveryChecks > 0) {
-                    synchronized (this) {
-                        ++numChecks;
-                        if (numChecks >= Integer.MAX_VALUE) {
-                            numChecks = 0;
-                        }
-                        sendHeartBeat = (numChecks % heartbeatEveryChecks) == 0;
-                    }
-                } else {
-                    sendHeartBeat = false;
-                }
-                if (sendHeartBeat) {
-                    dlClient.heartbeat(name).addEventListener(this);
-                } else {
-                    dlClient.check(name).addEventListener(this);
-                }
-            }
-        }
-
-        @Override
-        public void onSegmentsUpdated(List<LogSegmentMetadata> segments) {
-            if (segments.size() > 0 && segments.get(0).getRegionId() == regionId) {
-                if (!checking) {
-                    logger.info("Start checking stream {}.", name);
-                    checking = true;
-                    run();
-                }
-            } else {
-                if (checking) {
-                    logger.info("Stop checking stream {}.", name);
-                }
-            }
-        }
-
-        @Override
-        public void onLogStreamDeleted() {
-            logger.info("Stream {} is deleted", name);
-        }
-
-        @Override
-        public void onSuccess(Void value) {
-            successStat.add(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            scheduleCheck();
-        }
-
-        @Override
-        public void onFailure(Throwable cause) {
-            failureStat.add(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            scheduleCheck();
-        }
-
-        private void scheduleCheck() {
-            if (closed) {
-                return;
-            }
-            if (!checking) {
-                return;
-            }
-            try {
-                executorService.schedule(this, interval, TimeUnit.MILLISECONDS);
-            } catch (RejectedExecutionException ree) {
-                logger.error("Failed to schedule checking stream {} in {} ms : ",
-                        new Object[] { name, interval, ree });
-            }
-        }
-
-        private void close() {
-            closed = true;
-            if (null != dlm) {
-                try {
-                    dlm.close();
-                } catch (IOException e) {
-                    logger.error("Failed to close dlm for {} : ", name, e);
-                }
-            }
-        }
-    }
-
-    MonitorService(Optional<String> uriArg,
-                   Optional<String> confFileArg,
-                   Optional<String> serverSetArg,
-                   Optional<Integer> intervalArg,
-                   Optional<Integer> regionIdArg,
-                   Optional<String> streamRegexArg,
-                   Optional<Integer> instanceIdArg,
-                   Optional<Integer> totalInstancesArg,
-                   Optional<Integer> heartbeatEveryChecksArg,
-                   Optional<Boolean> handshakeWithClientInfoArg,
-                   Optional<Boolean> watchNamespaceChangesArg,
-                   Optional<Boolean> isThriftMuxArg,
-                   StatsReceiver statsReceiver,
-                   StatsProvider statsProvider) {
-        // options
-        this.uriArg = uriArg;
-        this.confFileArg = confFileArg;
-        this.serverSetArg = serverSetArg;
-        this.intervalArg = intervalArg;
-        this.regionIdArg = regionIdArg;
-        this.streamRegexArg = streamRegexArg;
-        this.instanceIdArg = instanceIdArg;
-        this.totalInstancesArg = totalInstancesArg;
-        this.heartbeatEveryChecksArg = heartbeatEveryChecksArg;
-        this.handshakeWithClientInfoArg = handshakeWithClientInfoArg;
-        this.watchNamespaceChangesArg = watchNamespaceChangesArg;
-        this.isThriftMuxArg = isThriftMuxArg;
-
-        // Stats
-        this.statsReceiver = statsReceiver;
-        this.monitorReceiver = statsReceiver.scope("monitor");
-        this.successStat = monitorReceiver.stat0("success");
-        this.failureStat = monitorReceiver.stat0("failure");
-        this.statsProvider = statsProvider;
-        this.numOfStreamsGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                return knownStreams.size();
-            }
-        };
-    }
-
-    public void runServer() throws IllegalArgumentException, IOException {
-        checkArgument(uriArg.isPresent(),
-                "No distributedlog uri provided.");
-        checkArgument(serverSetArg.isPresent(),
-                "No proxy server set provided.");
-        if (intervalArg.isPresent()) {
-            interval = intervalArg.get();
-        }
-        if (regionIdArg.isPresent()) {
-            regionId = regionIdArg.get();
-        }
-        if (streamRegexArg.isPresent()) {
-            streamRegex = streamRegexArg.get();
-        }
-        if (instanceIdArg.isPresent()) {
-            instanceId = instanceIdArg.get();
-        }
-        if (totalInstancesArg.isPresent()) {
-            totalInstances = totalInstancesArg.get();
-        }
-        if (heartbeatEveryChecksArg.isPresent()) {
-            heartbeatEveryChecks = heartbeatEveryChecksArg.get();
-        }
-        if (instanceId < 0 || totalInstances <= 0 || instanceId >= totalInstances) {
-            throw new IllegalArgumentException("Invalid instance id or total instances number.");
-        }
-        handshakeWithClientInfo = handshakeWithClientInfoArg.isPresent();
-        watchNamespaceChanges = watchNamespaceChangesArg.isPresent();
-        isThriftMux = isThriftMuxArg.isPresent();
-        URI uri = URI.create(uriArg.get());
-        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
-        if (confFileArg.isPresent()) {
-            String configFile = confFileArg.get();
-            try {
-                dlConf.loadConf(new File(configFile).toURI().toURL());
-            } catch (ConfigurationException e) {
-                throw new IOException("Failed to load distributedlog configuration from " + configFile + ".");
-            } catch (MalformedURLException e) {
-                throw new IOException("Failed to load distributedlog configuration from malformed "
-                        + configFile + ".");
-            }
-        }
-        logger.info("Starting stats provider : {}.", statsProvider.getClass());
-        statsProvider.start(dlConf);
-        String[] serverSetPaths = StringUtils.split(serverSetArg.get(), ",");
-        if (serverSetPaths.length == 0) {
-            throw new IllegalArgumentException("Invalid serverset paths provided : " + serverSetArg.get());
-        }
-
-        ServerSet[] serverSets = createServerSets(serverSetPaths);
-        ServerSet local = serverSets[0];
-        ServerSet[] remotes  = new ServerSet[serverSets.length - 1];
-        System.arraycopy(serverSets, 1, remotes, 0, remotes.length);
-
-        ClientBuilder finagleClientBuilder = ClientBuilder.get()
-            .connectTimeout(Duration.fromSeconds(1))
-            .tcpConnectTimeout(Duration.fromSeconds(1))
-            .requestTimeout(Duration.fromSeconds(2))
-            .keepAlive(true)
-            .failFast(false);
-
-        if (!isThriftMux) {
-            finagleClientBuilder = finagleClientBuilder
-                .hostConnectionLimit(2)
-                .hostConnectionCoresize(2);
-        }
-
-        dlClient = DistributedLogClientBuilder.newBuilder()
-                .name("monitor")
-                .thriftmux(isThriftMux)
-                .clientId(ClientId$.MODULE$.apply("monitor"))
-                .redirectBackoffMaxMs(50)
-                .redirectBackoffStartMs(100)
-                .requestTimeoutMs(2000)
-                .maxRedirects(2)
-                .serverSets(local, remotes)
-                .streamNameRegex(streamRegex)
-                .handshakeWithClientInfo(handshakeWithClientInfo)
-                .clientBuilder(finagleClientBuilder)
-                .statsReceiver(monitorReceiver.scope("client"))
-                .buildMonitorClient();
-        runMonitor(dlConf, uri);
-    }
-
-    ServerSet[] createServerSets(String[] serverSetPaths) {
-        ServerSet[] serverSets = new ServerSet[serverSetPaths.length];
-        zkServerSets = new DLZkServerSet[serverSetPaths.length];
-        for (int i = 0; i < serverSetPaths.length; i++) {
-            String serverSetPath = serverSetPaths[i];
-            zkServerSets[i] = parseServerSet(serverSetPath);
-            serverSets[i] = zkServerSets[i].getServerSet();
-        }
-        return serverSets;
-    }
-
-    protected DLZkServerSet parseServerSet(String serverSetPath) {
-        return DLZkServerSet.of(URI.create(serverSetPath), 60000);
-    }
-
-    @Override
-    public void onStreamsChanged(Iterator<String> streams) {
-        Set<String> newSet = new HashSet<String>();
-        while (streams.hasNext()) {
-            String s = streams.next();
-            if (null == streamRegex || s.matches(streamRegex)) {
-                if (Math.abs(hashFunction.hashUnencodedChars(s).asInt()) % totalInstances == instanceId) {
-                    newSet.add(s);
-                }
-            }
-        }
-        List<StreamChecker> tasksToCancel = new ArrayList<StreamChecker>();
-        synchronized (knownStreams) {
-            Set<String> knownStreamSet = new HashSet<String>(knownStreams.keySet());
-            Set<String> removedStreams = Sets.difference(knownStreamSet, newSet).immutableCopy();
-            Set<String> addedStreams = Sets.difference(newSet, knownStreamSet).immutableCopy();
-            for (String s : removedStreams) {
-                StreamChecker task = knownStreams.remove(s);
-                if (null != task) {
-                    logger.info("Removed stream {}", s);
-                    tasksToCancel.add(task);
-                }
-            }
-            for (String s : addedStreams) {
-                if (!knownStreams.containsKey(s)) {
-                    logger.info("Added stream {}", s);
-                    StreamChecker sc = new StreamChecker(s);
-                    knownStreams.put(s, sc);
-                    sc.run();
-                }
-            }
-        }
-        for (StreamChecker sc : tasksToCancel) {
-            sc.close();
-        }
-    }
-
-    void runMonitor(DistributedLogConfiguration conf, URI dlUri) throws IOException {
-        // stats
-        statsProvider.getStatsLogger("monitor").registerGauge("num_streams", numOfStreamsGauge);
-        logger.info("Construct dl namespace @ {}", dlUri);
-        dlNamespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(dlUri)
-                .build();
-        if (watchNamespaceChanges) {
-            dlNamespace.registerNamespaceListener(this);
-        } else {
-            onStreamsChanged(dlNamespace.getLogs());
-        }
-    }
-
-    /**
-     * Close the server.
-     */
-    public void close() {
-        logger.info("Closing monitor service.");
-        if (null != dlClient) {
-            dlClient.close();
-        }
-        if (null != zkServerSets) {
-            for (DLZkServerSet zkServerSet : zkServerSets) {
-                zkServerSet.close();
-            }
-        }
-        if (null != dlNamespace) {
-            dlNamespace.close();
-        }
-        executorService.shutdown();
-        try {
-            if (!executorService.awaitTermination(1, TimeUnit.MINUTES)) {
-                executorService.shutdownNow();
-            }
-        } catch (InterruptedException e) {
-            logger.error("Interrupted on waiting shutting down monitor executor service : ", e);
-        }
-        if (null != statsProvider) {
-            // clean up the gauges
-            unregisterGauge();
-            statsProvider.stop();
-        }
-        keepAliveLatch.countDown();
-        logger.info("Closed monitor service.");
-    }
-
-    public void join() throws InterruptedException {
-        keepAliveLatch.await();
-    }
-
-    /**
-     * clean up the gauge before we close to help GC.
-     */
-    private void unregisterGauge(){
-        statsProvider.getStatsLogger("monitor").unregisterGauge("num_streams", numOfStreamsGauge);
-    }
-
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/MonitorServiceApp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/MonitorServiceApp.java
deleted file mode 100644
index b5b4ca8..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/MonitorServiceApp.java
+++ /dev/null
@@ -1,133 +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 com.twitter.distributedlog.service;
-
-import static com.twitter.distributedlog.util.CommandLineUtils.getOptionalBooleanArg;
-import static com.twitter.distributedlog.util.CommandLineUtils.getOptionalIntegerArg;
-import static com.twitter.distributedlog.util.CommandLineUtils.getOptionalStringArg;
-
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.io.IOException;
-import org.apache.bookkeeper.stats.NullStatsProvider;
-import org.apache.bookkeeper.stats.StatsProvider;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.cli.BasicParser;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.HelpFormatter;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-
-/**
- * The launcher to run monitor service.
- */
-public class MonitorServiceApp {
-
-    private static final Logger logger = LoggerFactory.getLogger(MonitorServiceApp.class);
-
-    static final String USAGE = "MonitorService [-u <uri>] [-c <conf>] [-s serverset]";
-
-    final String[] args;
-    final Options options = new Options();
-
-    private MonitorServiceApp(String[] args) {
-        this.args = args;
-        // prepare options
-        options.addOption("u", "uri", true, "DistributedLog URI");
-        options.addOption("c", "conf", true, "DistributedLog Configuration File");
-        options.addOption("s", "serverset", true, "Proxy Server Set");
-        options.addOption("i", "interval", true, "Check interval");
-        options.addOption("d", "region", true, "Region ID");
-        options.addOption("p", "provider", true, "DistributedLog Stats Provider");
-        options.addOption("f", "filter", true, "Filter streams by regex");
-        options.addOption("w", "watch", false, "Watch stream changes under a given namespace");
-        options.addOption("n", "instance_id", true, "Instance ID");
-        options.addOption("t", "total_instances", true, "Total instances");
-        options.addOption("hck", "heartbeat-num-checks", true, "Send a heartbeat after num checks");
-        options.addOption("hsci", "handshake-with-client-info", false, "Enable handshaking with client info");
-    }
-
-    void printUsage() {
-        HelpFormatter helpFormatter = new HelpFormatter();
-        helpFormatter.printHelp(USAGE, options);
-    }
-
-    private void run() {
-        try {
-            logger.info("Running monitor service.");
-            BasicParser parser = new BasicParser();
-            CommandLine cmdline = parser.parse(options, args);
-            runCmd(cmdline);
-        } catch (ParseException pe) {
-            printUsage();
-            Runtime.getRuntime().exit(-1);
-        } catch (IOException ie) {
-            logger.error("Failed to start monitor service : ", ie);
-            Runtime.getRuntime().exit(-1);
-        }
-    }
-
-    void runCmd(CommandLine cmdline) throws IOException {
-        StatsProvider statsProvider = new NullStatsProvider();
-        if (cmdline.hasOption("p")) {
-            String providerClass = cmdline.getOptionValue("p");
-            statsProvider = ReflectionUtils.newInstance(providerClass, StatsProvider.class);
-        }
-        StatsReceiver statsReceiver = NullStatsReceiver.get();
-
-        final MonitorService monitorService = new MonitorService(
-                getOptionalStringArg(cmdline, "u"),
-                getOptionalStringArg(cmdline, "c"),
-                getOptionalStringArg(cmdline, "s"),
-                getOptionalIntegerArg(cmdline, "i"),
-                getOptionalIntegerArg(cmdline, "d"),
-                getOptionalStringArg(cmdline, "f"),
-                getOptionalIntegerArg(cmdline, "n"),
-                getOptionalIntegerArg(cmdline, "t"),
-                getOptionalIntegerArg(cmdline, "hck"),
-                getOptionalBooleanArg(cmdline, "hsci"),
-                getOptionalBooleanArg(cmdline, "w"),
-                getOptionalBooleanArg(cmdline, "mx"),
-                statsReceiver,
-                statsProvider);
-
-        monitorService.runServer();
-
-        Runtime.getRuntime().addShutdownHook(new Thread() {
-            @Override
-            public void run() {
-                logger.info("Closing monitor service.");
-                monitorService.close();
-                logger.info("Closed monitor service.");
-            }
-        });
-        try {
-            monitorService.join();
-        } catch (InterruptedException ie) {
-            logger.warn("Interrupted when waiting monitor service to be finished : ", ie);
-        }
-    }
-
-    public static void main(String[] args) {
-        final MonitorServiceApp launcher = new MonitorServiceApp(args);
-        launcher.run();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/ResponseUtils.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/ResponseUtils.java
deleted file mode 100644
index cee9dba..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/ResponseUtils.java
+++ /dev/null
@@ -1,86 +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 com.twitter.distributedlog.service;
-
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.distributedlog.thrift.service.BulkWriteResponse;
-import com.twitter.distributedlog.thrift.service.ResponseHeader;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-
-/**
- * Utility methods for building write proxy service responses.
- */
-public class ResponseUtils {
-    public static ResponseHeader deniedHeader() {
-        return new ResponseHeader(StatusCode.REQUEST_DENIED);
-    }
-
-    public static ResponseHeader streamUnavailableHeader() {
-        return new ResponseHeader(StatusCode.STREAM_UNAVAILABLE);
-    }
-
-    public static ResponseHeader successHeader() {
-        return new ResponseHeader(StatusCode.SUCCESS);
-    }
-
-    public static ResponseHeader ownerToHeader(String owner) {
-        return new ResponseHeader(StatusCode.FOUND).setLocation(owner);
-    }
-
-    public static ResponseHeader exceptionToHeader(Throwable t) {
-        ResponseHeader response = new ResponseHeader();
-        if (t instanceof DLException) {
-            DLException dle = (DLException) t;
-            if (dle instanceof OwnershipAcquireFailedException) {
-                response.setLocation(((OwnershipAcquireFailedException) dle).getCurrentOwner());
-            }
-            response.setCode(dle.getCode());
-            response.setErrMsg(dle.getMessage());
-        } else {
-            response.setCode(StatusCode.INTERNAL_SERVER_ERROR);
-            response.setErrMsg("Internal server error : " + t.getMessage());
-        }
-        return response;
-    }
-
-    public static WriteResponse write(ResponseHeader responseHeader) {
-        return new WriteResponse(responseHeader);
-    }
-
-    public static WriteResponse writeSuccess() {
-        return new WriteResponse(successHeader());
-    }
-
-    public static WriteResponse writeDenied() {
-        return new WriteResponse(deniedHeader());
-    }
-
-    public static BulkWriteResponse bulkWrite(ResponseHeader responseHeader) {
-        return new BulkWriteResponse(responseHeader);
-    }
-
-    public static BulkWriteResponse bulkWriteSuccess() {
-        return new BulkWriteResponse(successHeader());
-    }
-
-    public static BulkWriteResponse bulkWriteDenied() {
-        return new BulkWriteResponse(deniedHeader());
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/ServerFeatureKeys.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/ServerFeatureKeys.java
deleted file mode 100644
index d779cd0..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/ServerFeatureKeys.java
+++ /dev/null
@@ -1,29 +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 com.twitter.distributedlog.service;
-
-/**
- * List of feature keys used by distributedlog server.
- */
-public enum ServerFeatureKeys {
-
-    REGION_STOP_ACCEPT_NEW_STREAM,
-    SERVICE_RATE_LIMIT_DISABLED,
-    SERVICE_CHECKSUM_DISABLED,
-    SERVICE_GLOBAL_LIMITER_DISABLED
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/StatsFilter.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/StatsFilter.java
deleted file mode 100644
index bd0a992..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/StatsFilter.java
+++ /dev/null
@@ -1,60 +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 com.twitter.distributedlog.service;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.finagle.Service;
-import com.twitter.finagle.SimpleFilter;
-import com.twitter.util.Future;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Track distributedlog server finagle-service stats.
- */
-class StatsFilter<Req, Rep> extends SimpleFilter<Req, Rep> {
-
-    private final StatsLogger stats;
-    private final Counter outstandingAsync;
-    private final OpStatsLogger serviceExec;
-
-    @Override
-    public Future<Rep> apply(Req req, Service<Req, Rep> service) {
-        Future<Rep> result = null;
-        outstandingAsync.inc();
-        final Stopwatch stopwatch = Stopwatch.createStarted();
-        try {
-            result = service.apply(req);
-            serviceExec.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-        } finally {
-            outstandingAsync.dec();
-            if (null == result) {
-                serviceExec.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
-            }
-        }
-        return result;
-    }
-
-    public StatsFilter(StatsLogger stats) {
-        this.stats = stats;
-        this.outstandingAsync = stats.getCounter("outstandingAsync");
-        this.serviceExec = stats.getOpStatsLogger("serviceExec");
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/announcer/Announcer.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/announcer/Announcer.java
deleted file mode 100644
index cb37088..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/announcer/Announcer.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 com.twitter.distributedlog.service.announcer;
-
-import java.io.IOException;
-
-/**
- * Announce service information.
- */
-public interface Announcer {
-
-    /**
-     * Announce service info.
-     */
-    void announce() throws IOException;
-
-    /**
-     * Unannounce the service info.
-     */
-    void unannounce() throws IOException;
-
-    /**
-     * Close the announcer.
-     */
-    void close();
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/announcer/NOPAnnouncer.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/announcer/NOPAnnouncer.java
deleted file mode 100644
index 471f954..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/announcer/NOPAnnouncer.java
+++ /dev/null
@@ -1,40 +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 com.twitter.distributedlog.service.announcer;
-
-import java.io.IOException;
-
-/**
- * A no-op implementation of {@link Announcer}.
- */
-public class NOPAnnouncer implements Announcer {
-    @Override
-    public void announce() throws IOException {
-        // nop
-    }
-
-    @Override
-    public void unannounce() throws IOException {
-        // nop
-    }
-
-    @Override
-    public void close() {
-        // nop
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/announcer/ServerSetAnnouncer.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/announcer/ServerSetAnnouncer.java
deleted file mode 100644
index eaf4c26..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/announcer/ServerSetAnnouncer.java
+++ /dev/null
@@ -1,111 +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 com.twitter.distributedlog.service.announcer;
-
-import com.twitter.common.zookeeper.Group;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.client.serverset.DLZkServerSet;
-import java.io.IOException;
-import java.net.InetAddress;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.net.UnknownHostException;
-import java.util.HashMap;
-import java.util.Map;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * ServerSet based announcer.
- */
-public class ServerSetAnnouncer implements Announcer {
-
-    private static final Logger logger = LoggerFactory.getLogger(ServerSetAnnouncer.class);
-
-    final String localAddr;
-    final InetSocketAddress serviceEndpoint;
-    final Map<String, InetSocketAddress> additionalEndpoints;
-    final int shardId;
-
-    // ServerSet
-    DLZkServerSet zkServerSet;
-
-    // Service Status
-    ServerSet.EndpointStatus serviceStatus = null;
-
-    /**
-     * Announce server infos.
-     *
-     * @param servicePort
-     *          service port
-     * @param statsPort
-     *          stats port
-     * @param shardId
-     *          shard id
-     */
-    public ServerSetAnnouncer(URI uri,
-                              int servicePort,
-                              int statsPort,
-                              int shardId) throws UnknownHostException {
-        this.shardId = shardId;
-        this.localAddr = InetAddress.getLocalHost().getHostAddress();
-        // service endpoint
-        this.serviceEndpoint = new InetSocketAddress(localAddr, servicePort);
-        // stats endpoint
-        InetSocketAddress statsEndpoint = new InetSocketAddress(localAddr, statsPort);
-        this.additionalEndpoints = new HashMap<String, InetSocketAddress>();
-        this.additionalEndpoints.put("aurora", statsEndpoint);
-        this.additionalEndpoints.put("stats", statsEndpoint);
-        this.additionalEndpoints.put("service", serviceEndpoint);
-        this.additionalEndpoints.put("thrift", serviceEndpoint);
-
-        // Create zookeeper and server set
-        this.zkServerSet = DLZkServerSet.of(uri, 60000);
-    }
-
-    @Override
-    public synchronized void announce() throws IOException {
-        try {
-            serviceStatus =
-                    zkServerSet.getServerSet().join(serviceEndpoint, additionalEndpoints, shardId);
-        } catch (Group.JoinException e) {
-            throw new IOException("Failed to announce service : ", e);
-        } catch (InterruptedException e) {
-            logger.warn("Interrupted on announcing service : ", e);
-            Thread.currentThread().interrupt();
-        }
-    }
-
-    @Override
-    public synchronized void unannounce() throws IOException {
-        if (null == serviceStatus) {
-            logger.warn("No service to unannounce.");
-            return;
-        }
-        try {
-            serviceStatus.leave();
-        } catch (ServerSet.UpdateException e) {
-            throw new IOException("Failed to unannounce service : ", e);
-        }
-    }
-
-    @Override
-    public void close() {
-        zkServerSet.close();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/announcer/package-info.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/announcer/package-info.java
deleted file mode 100644
index bca36df..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/announcer/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Announcers to announce servers to server set.
- */
-package com.twitter.distributedlog.service.announcer;
\ No newline at end of file
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/Balancer.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/Balancer.java
deleted file mode 100644
index 3ffe54b..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/Balancer.java
+++ /dev/null
@@ -1,68 +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 com.twitter.distributedlog.service.balancer;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-
-/**
- * Balancer Interface.
- *
- * <p>A balancer is used for balance the streams across the proxy cluster.
- */
-public interface Balancer {
-
-    /**
-     * Rebalance all the streams from <i>source</i> to others.
-     *
-     * @param source
-     *          source target name.
-     * @param rebalanceConcurrency
-     *          the concurrency to move streams for re-balance.
-     * @param rebalanceRateLimiter
-     *          the rate limiting to move streams for re-balance.
-     */
-    void balanceAll(String source,
-                    int rebalanceConcurrency,
-                    Optional<RateLimiter> rebalanceRateLimiter);
-
-    /**
-     * Balance the streams across all targets.
-     *
-     * @param rebalanceWaterMark
-     *          rebalance water mark. if number of streams of a given target is less than
-     *          the water mark, no streams will be re-balanced from this target.
-     * @param rebalanceTolerancePercentage
-     *          tolerance percentage for the balancer. if number of streams of a given target is
-     *          less than average + average * <i>tolerancePercentage</i> / 100.0, no streams will
-     *          be re-balanced from that target.
-     * @param rebalanceConcurrency
-     *          the concurrency to move streams for re-balance.
-     * @param rebalanceRateLimiter
-     *          the rate limiting to move streams for re-balance.
-     */
-    void balance(int rebalanceWaterMark,
-                 double rebalanceTolerancePercentage,
-                 int rebalanceConcurrency,
-                 Optional<RateLimiter> rebalanceRateLimiter);
-
-    /**
-     * Close the balancer.
-     */
-    void close();
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerTool.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerTool.java
deleted file mode 100644
index 48430df..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerTool.java
+++ /dev/null
@@ -1,327 +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 com.twitter.distributedlog.service.balancer;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-import com.twitter.common.zookeeper.ServerSet;
-import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
-import com.twitter.distributedlog.client.serverset.DLZkServerSet;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.service.ClientUtils;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.distributedlog.tools.Tool;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Tool to rebalance cluster.
- */
-public class BalancerTool extends Tool {
-
-    private static final Logger logger = LoggerFactory.getLogger(BalancerTool.class);
-
-    static DistributedLogClientBuilder createDistributedLogClientBuilder(ServerSet serverSet) {
-        return DistributedLogClientBuilder.newBuilder()
-                        .name("rebalancer_tool")
-                        .clientId(ClientId$.MODULE$.apply("rebalancer_tool"))
-                        .maxRedirects(2)
-                        .serverSet(serverSet)
-                        .clientBuilder(ClientBuilder.get()
-                                .connectionTimeout(Duration.fromSeconds(2))
-                                .tcpConnectTimeout(Duration.fromSeconds(2))
-                                .requestTimeout(Duration.fromSeconds(10))
-                                .hostConnectionLimit(1)
-                                .hostConnectionCoresize(1)
-                                .keepAlive(true)
-                                .failFast(false));
-    }
-
-    /**
-     * Base Command to run balancer.
-     */
-    protected abstract static class BalancerCommand extends OptsCommand {
-
-        protected Options options = new Options();
-        protected int rebalanceWaterMark = 0;
-        protected double rebalanceTolerancePercentage = 0.0f;
-        protected int rebalanceConcurrency = 1;
-        protected Double rate = null;
-        protected Optional<RateLimiter> rateLimiter;
-
-        BalancerCommand(String name, String description) {
-            super(name, description);
-            options.addOption("rwm", "rebalance-water-mark", true, "Rebalance water mark per proxy");
-            options.addOption("rtp", "rebalance-tolerance-percentage", true,
-                "Rebalance tolerance percentage per proxy");
-            options.addOption("rc", "rebalance-concurrency", true, "Concurrency to rebalance stream distribution");
-            options.addOption("r", "rate", true, "Rebalance rate");
-        }
-
-        Optional<RateLimiter> getRateLimiter() {
-            return rateLimiter;
-        }
-
-        @Override
-        protected Options getOptions() {
-            return options;
-        }
-
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            if (cmdline.hasOption("rwm")) {
-                this.rebalanceWaterMark = Integer.parseInt(cmdline.getOptionValue("rwm"));
-            }
-            if (cmdline.hasOption("rtp")) {
-                this.rebalanceTolerancePercentage = Double.parseDouble(cmdline.getOptionValue("rtp"));
-            }
-            if (cmdline.hasOption("rc")) {
-                this.rebalanceConcurrency = Integer.parseInt(cmdline.getOptionValue("rc"));
-            }
-            if (cmdline.hasOption("r")) {
-                this.rate = Double.parseDouble(cmdline.getOptionValue("r"));
-            }
-            checkArgument(rebalanceWaterMark >= 0,
-                    "Rebalance Water Mark should be a non-negative number");
-            checkArgument(rebalanceTolerancePercentage >= 0.0f,
-                    "Rebalance Tolerance Percentage should be a non-negative number");
-            checkArgument(rebalanceConcurrency > 0,
-                    "Rebalance Concurrency should be a positive number");
-            if (null == rate || rate <= 0.0f) {
-                rateLimiter = Optional.absent();
-            } else {
-                rateLimiter = Optional.of(RateLimiter.create(rate));
-            }
-        }
-
-        @Override
-        protected int runCmd(CommandLine cmdline) throws Exception {
-            try {
-                parseCommandLine(cmdline);
-            } catch (ParseException pe) {
-                println("ERROR: fail to parse commandline : '" + pe.getMessage() + "'");
-                printUsage();
-                return -1;
-            }
-            return executeCommand(cmdline);
-        }
-
-        protected abstract int executeCommand(CommandLine cmdline) throws Exception;
-    }
-
-    /**
-     * Command to balance streams within a cluster.
-     */
-    protected static class ClusterBalancerCommand extends BalancerCommand {
-
-        protected URI uri;
-        protected String source = null;
-
-        protected ClusterBalancerCommand() {
-            super("clusterbalancer", "Balance streams inside a cluster");
-            options.addOption("u", "uri", true, "DistributedLog URI");
-            options.addOption("sp", "source-proxy", true, "Source proxy to balance");
-        }
-
-        @Override
-        protected String getUsage() {
-            return "clusterbalancer [options]";
-        }
-
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("u")) {
-                throw new ParseException("No proxy serverset provided.");
-            }
-            uri = URI.create(cmdline.getOptionValue("u"));
-            if (cmdline.hasOption("sp")) {
-                String sourceProxyStr = cmdline.getOptionValue("sp");
-                try {
-                    DLSocketAddress.parseSocketAddress(sourceProxyStr);
-                } catch (IllegalArgumentException iae) {
-                    throw new ParseException("Invalid source proxy " + sourceProxyStr + " : " + iae.getMessage());
-                }
-                this.source = sourceProxyStr;
-            }
-        }
-
-        @Override
-        protected int executeCommand(CommandLine cmdline) throws Exception {
-            DLZkServerSet serverSet = DLZkServerSet.of(uri, 60000);
-            logger.info("Created serverset for {}", uri);
-            try {
-                DistributedLogClientBuilder clientBuilder =
-                        createDistributedLogClientBuilder(serverSet.getServerSet());
-                ClusterBalancer balancer = new ClusterBalancer(clientBuilder);
-                try {
-                    return runBalancer(clientBuilder, balancer);
-                } finally {
-                    balancer.close();
-                }
-            } finally {
-                serverSet.close();
-            }
-        }
-
-        protected int runBalancer(DistributedLogClientBuilder clientBuilder,
-                                  ClusterBalancer balancer)
-                throws Exception {
-            if (null == source) {
-                balancer.balance(
-                    rebalanceWaterMark,
-                    rebalanceTolerancePercentage,
-                    rebalanceConcurrency,
-                    getRateLimiter());
-            } else {
-                balanceFromSource(clientBuilder, balancer, source, getRateLimiter());
-            }
-            return 0;
-        }
-
-        protected void balanceFromSource(DistributedLogClientBuilder clientBuilder,
-                                         ClusterBalancer balancer,
-                                         String source,
-                                         Optional<RateLimiter> rateLimiter)
-                throws Exception {
-            InetSocketAddress sourceAddr = DLSocketAddress.parseSocketAddress(source);
-            DistributedLogClientBuilder sourceClientBuilder =
-                    DistributedLogClientBuilder.newBuilder(clientBuilder)
-                            .host(sourceAddr);
-
-            Pair<DistributedLogClient, MonitorServiceClient> clientPair =
-                    ClientUtils.buildClient(sourceClientBuilder);
-            try {
-                Await.result(clientPair.getRight().setAcceptNewStream(false));
-                logger.info("Disable accepting new stream on proxy {}.", source);
-                balancer.balanceAll(source, rebalanceConcurrency, rateLimiter);
-            } finally {
-                clientPair.getLeft().close();
-            }
-        }
-    }
-
-    /**
-     * Command to balance streams between regions.
-     */
-    protected static class RegionBalancerCommand extends BalancerCommand {
-
-        protected URI region1;
-        protected URI region2;
-        protected String source = null;
-
-        protected RegionBalancerCommand() {
-            super("regionbalancer", "Balance streams between regions");
-            options.addOption("rs", "regions", true, "DistributedLog Region URI: uri1[,uri2]");
-            options.addOption("s", "source", true, "DistributedLog Source Region to balance");
-        }
-
-        @Override
-        protected String getUsage() {
-            return "regionbalancer [options]";
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("rs")) {
-                throw new ParseException("No regions provided.");
-            }
-            String regionsStr = cmdline.getOptionValue("rs");
-            String[] regions = regionsStr.split(",");
-            if (regions.length != 2) {
-                throw new ParseException("Invalid regions provided. Expected : serverset1[,serverset2]");
-            }
-            region1 = URI.create(regions[0]);
-            region2 = URI.create(regions[1]);
-            if (cmdline.hasOption("s")) {
-                source = cmdline.getOptionValue("s");
-            }
-        }
-
-        @Override
-        protected int executeCommand(CommandLine cmdline) throws Exception {
-            DLZkServerSet serverSet1 = DLZkServerSet.of(region1, 60000);
-            logger.info("Created serverset for {}", region1);
-            DLZkServerSet serverSet2 = DLZkServerSet.of(region2, 60000);
-            logger.info("Created serverset for {}", region2);
-            try {
-                DistributedLogClientBuilder builder1 =
-                        createDistributedLogClientBuilder(serverSet1.getServerSet());
-                Pair<DistributedLogClient, MonitorServiceClient> pair1 =
-                        ClientUtils.buildClient(builder1);
-                DistributedLogClientBuilder builder2 =
-                        createDistributedLogClientBuilder(serverSet2.getServerSet());
-                Pair<DistributedLogClient, MonitorServiceClient> pair2 =
-                        ClientUtils.buildClient(builder2);
-                try {
-                    SimpleBalancer balancer = new SimpleBalancer(
-                            BKNamespaceDriver.getZKServersFromDLUri(region1), pair1.getLeft(), pair1.getRight(),
-                            BKNamespaceDriver.getZKServersFromDLUri(region2), pair2.getLeft(), pair2.getRight());
-                    try {
-                        return runBalancer(balancer);
-                    } finally {
-                        balancer.close();
-                    }
-                } finally {
-                    pair1.getLeft().close();
-                    pair2.getLeft().close();
-                }
-            } finally {
-                serverSet1.close();
-                serverSet2.close();
-            }
-        }
-
-        protected int runBalancer(SimpleBalancer balancer) throws Exception {
-            if (null == source) {
-                balancer.balance(
-                    rebalanceWaterMark,
-                    rebalanceTolerancePercentage,
-                    rebalanceConcurrency,
-                    getRateLimiter());
-            } else {
-                balancer.balanceAll(source, rebalanceConcurrency, getRateLimiter());
-            }
-            return 0;
-        }
-    }
-
-    public BalancerTool() {
-        super();
-        addCommand(new ClusterBalancerCommand());
-        addCommand(new RegionBalancerCommand());
-    }
-
-    @Override
-    protected String getName() {
-        return "balancer";
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerUtils.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerUtils.java
deleted file mode 100644
index f1877f3..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/BalancerUtils.java
+++ /dev/null
@@ -1,74 +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 com.twitter.distributedlog.service.balancer;
-
-import java.util.Map;
-
-/**
- * Utils for balancer.
- */
-public class BalancerUtils {
-
-    /**
-     * Util function to calculate how many streams to balance for <i>nodeToRebalance</i>,
-     * based on the load distribution <i>loadDistribution</i>.
-     *
-     * @param nodeToRebalance
-     *          node to rebalance
-     * @param loadDistribution
-     *          load distribution map
-     * @param rebalanceWaterMark
-     *          if number of streams of <i>nodeToRebalance</i>
-     *          is less than <i>rebalanceWaterMark</i>, no streams will be re-balanced.
-     * @param tolerancePercentage
-     *          tolerance percentage for the balancer. if number of streams of <i>nodeToRebalance</i>
-     *          is less than average + average * <i>tolerancePercentage</i> / 100.0, no streams will
-     *          be re-balanced.
-     * @param <K>
-     * @return number of streams to rebalance
-     */
-    public static <K> int calculateNumStreamsToRebalance(K nodeToRebalance,
-                                                         Map<K, Integer> loadDistribution,
-                                                         int rebalanceWaterMark,
-                                                         double tolerancePercentage) {
-        Integer myLoad = loadDistribution.get(nodeToRebalance);
-        if (null == myLoad || myLoad <= rebalanceWaterMark) {
-            return 0;
-        }
-
-        long totalLoad = 0L;
-        int numNodes = loadDistribution.size();
-
-        for (Map.Entry<K, Integer> entry : loadDistribution.entrySet()) {
-            if (null == entry.getKey() || null == entry.getValue()) {
-                continue;
-            }
-            totalLoad += entry.getValue();
-        }
-
-        double averageLoad = ((double) totalLoad) / numNodes;
-        long permissibleLoad =
-                Math.max(1L, (long) Math.ceil(averageLoad + averageLoad * tolerancePercentage / 100.0f));
-
-        if (myLoad <= permissibleLoad) {
-            return 0;
-        }
-
-        return Math.max(0, myLoad - (int) Math.ceil(averageLoad));
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/ClusterBalancer.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/ClusterBalancer.java
deleted file mode 100644
index 3a3dc1f..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/ClusterBalancer.java
+++ /dev/null
@@ -1,378 +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 com.twitter.distributedlog.service.balancer;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
-import com.twitter.distributedlog.service.ClientUtils;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.util.Await;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import java.io.Serializable;
-import java.net.SocketAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A balancer balances ownerships with a cluster of targets.
- */
-public class ClusterBalancer implements Balancer {
-
-    private static final Logger logger = LoggerFactory.getLogger(ClusterBalancer.class);
-
-    /**
-     * Represent a single host. Ordered by number of streams in desc order.
-     */
-    static class Host {
-
-        final SocketAddress address;
-        final Set<String> streams;
-        final DistributedLogClientBuilder clientBuilder;
-        DistributedLogClient client = null;
-        MonitorServiceClient monitor = null;
-
-        Host(SocketAddress address, Set<String> streams,
-             DistributedLogClientBuilder clientBuilder) {
-            this.address = address;
-            this.streams = streams;
-            this.clientBuilder = clientBuilder;
-        }
-
-        private void initializeClientsIfNeeded() {
-            if (null == client) {
-                Pair<DistributedLogClient, MonitorServiceClient> clientPair =
-                        createDistributedLogClient(address, clientBuilder);
-                client = clientPair.getLeft();
-                monitor = clientPair.getRight();
-            }
-        }
-
-        synchronized DistributedLogClient getClient() {
-            initializeClientsIfNeeded();
-            return client;
-        }
-
-        synchronized MonitorServiceClient getMonitor() {
-            initializeClientsIfNeeded();
-            return monitor;
-        }
-
-        synchronized void close() {
-            if (null != client) {
-                client.close();
-            }
-        }
-
-        @Override
-        public String toString() {
-            StringBuilder sb = new StringBuilder();
-            sb.append("Host(").append(address).append(")");
-            return sb.toString();
-        }
-    }
-
-    static class HostComparator implements Comparator<Host>, Serializable {
-        private static final long serialVersionUID = 7984973796525102538L;
-
-        @Override
-        public int compare(Host h1, Host h2) {
-            return h2.streams.size() - h1.streams.size();
-        }
-    }
-
-    protected final DistributedLogClientBuilder clientBuilder;
-    protected final DistributedLogClient client;
-    protected final MonitorServiceClient monitor;
-
-    public ClusterBalancer(DistributedLogClientBuilder clientBuilder) {
-        this(clientBuilder, ClientUtils.buildClient(clientBuilder));
-    }
-
-    ClusterBalancer(DistributedLogClientBuilder clientBuilder,
-                    Pair<DistributedLogClient, MonitorServiceClient> clientPair) {
-        this.clientBuilder = clientBuilder;
-        this.client = clientPair.getLeft();
-        this.monitor = clientPair.getRight();
-    }
-
-    /**
-     * Build a new distributedlog client to a single host <i>host</i>.
-     *
-     * @param host
-     *          host to access
-     * @return distributedlog clients
-     */
-    static Pair<DistributedLogClient, MonitorServiceClient> createDistributedLogClient(
-            SocketAddress host, DistributedLogClientBuilder clientBuilder) {
-        DistributedLogClientBuilder newBuilder =
-                DistributedLogClientBuilder.newBuilder(clientBuilder).host(host);
-        return ClientUtils.buildClient(newBuilder);
-    }
-
-    @Override
-    public void balanceAll(String source,
-                           int rebalanceConcurrency, /* unused */
-                           Optional<RateLimiter> rebalanceRateLimiter) {
-        balance(0, 0.0f, rebalanceConcurrency, Optional.of(source), rebalanceRateLimiter);
-    }
-
-    @Override
-    public void balance(int rebalanceWaterMark,
-                        double rebalanceTolerancePercentage,
-                        int rebalanceConcurrency, /* unused */
-                        Optional<RateLimiter> rebalanceRateLimiter) {
-        Optional<String> source = Optional.absent();
-        balance(rebalanceWaterMark, rebalanceTolerancePercentage, rebalanceConcurrency, source, rebalanceRateLimiter);
-    }
-
-    public void balance(int rebalanceWaterMark,
-                        double rebalanceTolerancePercentage,
-                        int rebalanceConcurrency,
-                        Optional<String> source,
-                        Optional<RateLimiter> rebalanceRateLimiter) {
-        Map<SocketAddress, Set<String>> distribution = monitor.getStreamOwnershipDistribution();
-        if (distribution.size() <= 1) {
-            return;
-        }
-        SocketAddress sourceAddr = null;
-        if (source.isPresent()) {
-            sourceAddr = DLSocketAddress.parseSocketAddress(source.get());
-            logger.info("Balancer source is {}", sourceAddr);
-            if (!distribution.containsKey(sourceAddr)) {
-                return;
-            }
-        }
-        // Get the list of hosts ordered by number of streams in DESC order
-        List<Host> hosts = new ArrayList<Host>(distribution.size());
-        for (Map.Entry<SocketAddress, Set<String>> entry : distribution.entrySet()) {
-            Host host = new Host(entry.getKey(), entry.getValue(), clientBuilder);
-            hosts.add(host);
-        }
-        Collections.sort(hosts, new HostComparator());
-        try {
-
-            // find the host to move streams from.
-            int hostIdxMoveFrom = -1;
-            if (null != sourceAddr) {
-                for (Host host : hosts) {
-                    ++hostIdxMoveFrom;
-                    if (sourceAddr.equals(host.address)) {
-                        break;
-                    }
-                }
-            }
-
-            // compute the average load.
-            int totalStream = 0;
-            for (Host host : hosts) {
-                totalStream += host.streams.size();
-            }
-            double averageLoad;
-            if (hostIdxMoveFrom >= 0) {
-                averageLoad = ((double) totalStream / (hosts.size() - 1));
-            } else {
-                averageLoad = ((double) totalStream / hosts.size());
-            }
-
-            int moveFromLowWaterMark;
-            int moveToHighWaterMark =
-                Math.max(1, (int) (averageLoad + averageLoad * rebalanceTolerancePercentage / 100.0f));
-
-            if (hostIdxMoveFrom >= 0) {
-                moveFromLowWaterMark = Math.max(0, rebalanceWaterMark);
-                moveStreams(
-                        hosts,
-                        new AtomicInteger(hostIdxMoveFrom), moveFromLowWaterMark,
-                        new AtomicInteger(hosts.size() - 1), moveToHighWaterMark,
-                        rebalanceRateLimiter);
-                moveRemainingStreamsFromSource(hosts.get(hostIdxMoveFrom), hosts, rebalanceRateLimiter);
-            } else {
-                moveFromLowWaterMark = Math.max((int) Math.ceil(averageLoad), rebalanceWaterMark);
-                AtomicInteger moveFrom = new AtomicInteger(0);
-                AtomicInteger moveTo = new AtomicInteger(hosts.size() - 1);
-                while (moveFrom.get() < moveTo.get()) {
-                    moveStreams(hosts, moveFrom, moveFromLowWaterMark,
-                        moveTo, moveToHighWaterMark, rebalanceRateLimiter);
-                    moveFrom.incrementAndGet();
-                }
-            }
-        } finally {
-            for (Host host : hosts) {
-                host.close();
-            }
-        }
-    }
-
-    void moveStreams(List<Host> hosts,
-                     AtomicInteger hostIdxMoveFrom,
-                     int moveFromLowWaterMark,
-                     AtomicInteger hostIdxMoveTo,
-                     int moveToHighWaterMark,
-                     Optional<RateLimiter> rateLimiter) {
-        if (hostIdxMoveFrom.get() < 0 || hostIdxMoveFrom.get() >= hosts.size()
-                || hostIdxMoveTo.get() < 0 || hostIdxMoveTo.get() >= hosts.size()
-                || hostIdxMoveFrom.get() >= hostIdxMoveTo.get()) {
-            return;
-        }
-
-        if (logger.isDebugEnabled()) {
-            logger.debug("Moving streams : hosts = {}, from = {}, to = {} :"
-                + " from_low_water_mark = {}, to_high_water_mark = {}",
-                new Object[] {
-                    hosts,
-                    hostIdxMoveFrom.get(),
-                    hostIdxMoveTo.get(),
-                    moveFromLowWaterMark,
-                    moveToHighWaterMark });
-        }
-
-        Host hostMoveFrom = hosts.get(hostIdxMoveFrom.get());
-        int numStreamsOnFromHost = hostMoveFrom.streams.size();
-        if (numStreamsOnFromHost <= moveFromLowWaterMark) {
-            // do nothing
-            return;
-        }
-
-        int numStreamsToMove = numStreamsOnFromHost - moveFromLowWaterMark;
-        LinkedList<String> streamsToMove = new LinkedList<String>(hostMoveFrom.streams);
-        Collections.shuffle(streamsToMove);
-
-        if (logger.isDebugEnabled()) {
-            logger.debug("Try to move {} streams from host {} : streams = {}",
-                         new Object[] { numStreamsToMove, hostMoveFrom.address, streamsToMove });
-        }
-
-        while (numStreamsToMove-- > 0 && !streamsToMove.isEmpty()) {
-            if (rateLimiter.isPresent()) {
-                rateLimiter.get().acquire();
-            }
-
-            // pick a host to move
-            Host hostMoveTo = hosts.get(hostIdxMoveTo.get());
-            while (hostMoveTo.streams.size() >= moveToHighWaterMark) {
-                int hostIdx = hostIdxMoveTo.decrementAndGet();
-                logger.info("move to host : {}, from {}", hostIdx, hostIdxMoveFrom.get());
-                if (hostIdx <= hostIdxMoveFrom.get()) {
-                    return;
-                } else {
-                    hostMoveTo = hosts.get(hostIdx);
-                    if (logger.isDebugEnabled()) {
-                        logger.debug("Target host to move moved to host {} @ {}",
-                                hostIdx, hostMoveTo);
-                    }
-                }
-            }
-
-            // pick a stream
-            String stream = streamsToMove.remove();
-
-            // move the stream
-            if (moveStream(stream, hostMoveFrom, hostMoveTo)) {
-                hostMoveFrom.streams.remove(stream);
-                hostMoveTo.streams.add(stream);
-            }
-        }
-
-    }
-
-    void moveRemainingStreamsFromSource(Host source,
-                                        List<Host> hosts,
-                                        Optional<RateLimiter> rateLimiter) {
-        LinkedList<String> streamsToMove = new LinkedList<String>(source.streams);
-        Collections.shuffle(streamsToMove);
-
-        if (logger.isDebugEnabled()) {
-            logger.debug("Try to move remaining streams from {} : {}", source, streamsToMove);
-        }
-
-        int hostIdx = hosts.size() - 1;
-
-        while (!streamsToMove.isEmpty()) {
-            if (rateLimiter.isPresent()) {
-                rateLimiter.get().acquire();
-            }
-
-            Host target = hosts.get(hostIdx);
-            if (!target.address.equals(source.address)) {
-                String stream = streamsToMove.remove();
-                // move the stream
-                if (moveStream(stream, source, target)) {
-                    source.streams.remove(stream);
-                    target.streams.add(stream);
-                }
-            }
-            --hostIdx;
-            if (hostIdx < 0) {
-                hostIdx = hosts.size() - 1;
-            }
-        }
-    }
-
-    private boolean moveStream(String stream, Host from, Host to) {
-        try {
-            doMoveStream(stream, from, to);
-            return true;
-        } catch (Exception e) {
-            return false;
-        }
-    }
-
-    private void doMoveStream(final String stream, final Host from, final Host to) throws Exception {
-        logger.info("Moving stream {} from {} to {}.",
-                    new Object[] { stream, from.address, to.address });
-        Await.result(from.getClient().release(stream).flatMap(new Function<Void, Future<Void>>() {
-            @Override
-            public Future<Void> apply(Void result) {
-                logger.info("Released stream {} from {}.", stream, from.address);
-                return to.getMonitor().check(stream).addEventListener(new FutureEventListener<Void>() {
-
-                    @Override
-                    public void onSuccess(Void value) {
-                        logger.info("Moved stream {} from {} to {}.",
-                                    new Object[] { stream, from.address, to.address });
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        logger.info("Failed to move stream {} from {} to {} : ",
-                                    new Object[] { stream, from.address, to.address, cause });
-                    }
-                });
-            }
-        }));
-    }
-
-    @Override
-    public void close() {
-        client.close();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/CountBasedStreamChooser.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/CountBasedStreamChooser.java
deleted file mode 100644
index fab37b3..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/CountBasedStreamChooser.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 com.twitter.distributedlog.service.balancer;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import java.io.Serializable;
-import java.net.SocketAddress;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.LinkedList;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.apache.commons.lang3.tuple.Pair;
-
-/**
- * A stream chooser based on number of streams.
- */
-class CountBasedStreamChooser implements StreamChooser, Serializable,
-        Comparator<Pair<SocketAddress, LinkedList<String>>> {
-
-    private static final long serialVersionUID = 4664153397369979203L;
-
-    final List<Pair<SocketAddress, LinkedList<String>>> streamsDistribution;
-
-    // pivot index in the list of hosts. the chooser will remove streams from the hosts before
-    // pivot, which will reduce their stream counts to make them equal to the stream count of the pivot.
-    int pivot;
-    int pivotCount;
-
-    // next index in the list of hosts to choose stream from.
-    int next;
-
-    CountBasedStreamChooser(Map<SocketAddress, Set<String>> streams) {
-        checkArgument(streams.size() > 0, "Only support no-empty streams distribution");
-        streamsDistribution = new ArrayList<Pair<SocketAddress, LinkedList<String>>>(streams.size());
-        for (Map.Entry<SocketAddress, Set<String>> entry : streams.entrySet()) {
-            LinkedList<String> randomizedStreams = new LinkedList<String>(entry.getValue());
-            Collections.shuffle(randomizedStreams);
-            streamsDistribution.add(Pair.of(entry.getKey(), randomizedStreams));
-        }
-        // sort the hosts by the number of streams in descending order
-        Collections.sort(streamsDistribution, this);
-        pivot = 0;
-        pivotCount = streamsDistribution.get(0).getValue().size();
-        findNextPivot();
-        next = 0;
-    }
-
-    private void findNextPivot() {
-        int prevPivotCount = pivotCount;
-        while (++pivot < streamsDistribution.size()) {
-            pivotCount = streamsDistribution.get(pivot).getValue().size();
-            if (pivotCount < prevPivotCount) {
-                return;
-            }
-        }
-        pivot = streamsDistribution.size();
-        pivotCount = 0;
-    }
-
-    @Override
-    public synchronized String choose() {
-        // reach the pivot
-        if (next == pivot) {
-            if (streamsDistribution.get(next - 1).getRight().size() > pivotCount) {
-                next = 0;
-            } else if (pivotCount == 0) { // the streams are empty now
-                return null;
-            } else {
-                findNextPivot();
-                next = 0;
-            }
-        }
-
-        // get stream count that next host to choose from
-        LinkedList<String> nextStreams = streamsDistribution.get(next).getRight();
-        if (nextStreams.size() == 0) {
-            return null;
-        }
-
-        String chosenStream = nextStreams.remove();
-        ++next;
-        return chosenStream;
-    }
-
-    @Override
-    public int compare(Pair<SocketAddress, LinkedList<String>> o1,
-                       Pair<SocketAddress, LinkedList<String>> o2) {
-        return o2.getValue().size() - o1.getValue().size();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/LimitedStreamChooser.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/LimitedStreamChooser.java
deleted file mode 100644
index 069e596..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/LimitedStreamChooser.java
+++ /dev/null
@@ -1,57 +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 com.twitter.distributedlog.service.balancer;
-
-/**
- * A stream chooser that can only choose limited number of streams.
- */
-public class LimitedStreamChooser implements StreamChooser {
-
-  /**
-   * Create a limited stream chooser by {@code limit}.
-   *
-   * @param underlying the underlying stream chooser.
-   * @param limit the limit of number of streams to choose.
-   * @return the limited stream chooser.
-   */
-    public static LimitedStreamChooser of(StreamChooser underlying, int limit) {
-        return new LimitedStreamChooser(underlying, limit);
-    }
-
-    final StreamChooser underlying;
-    int limit;
-
-    LimitedStreamChooser(StreamChooser underlying, int limit) {
-        this.underlying = underlying;
-        this.limit = limit;
-    }
-
-    @Override
-    public synchronized String choose() {
-        if (limit <= 0) {
-            return null;
-        }
-        String s = underlying.choose();
-        if (s == null) {
-            limit = 0;
-            return null;
-        }
-        --limit;
-        return s;
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/SimpleBalancer.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/SimpleBalancer.java
deleted file mode 100644
index b205d5f..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/SimpleBalancer.java
+++ /dev/null
@@ -1,246 +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 com.twitter.distributedlog.service.balancer;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * A balancer balances ownerships between two targets.
- */
-public class SimpleBalancer implements Balancer {
-
-    private static final Logger logger = LoggerFactory.getLogger(SimpleBalancer.class);
-
-    protected final String target1;
-    protected final String target2;
-    protected final DistributedLogClient targetClient1;
-    protected final DistributedLogClient targetClient2;
-    protected final MonitorServiceClient targetMonitor1;
-    protected final MonitorServiceClient targetMonitor2;
-
-    public SimpleBalancer(String name1,
-                          DistributedLogClient client1,
-                          MonitorServiceClient monitor1,
-                          String name2,
-                          DistributedLogClient client2,
-                          MonitorServiceClient monitor2) {
-        this.target1 = name1;
-        this.targetClient1 = client1;
-        this.targetMonitor1 = monitor1;
-        this.target2 = name2;
-        this.targetClient2 = client2;
-        this.targetMonitor2 = monitor2;
-    }
-
-    protected static int countNumberStreams(Map<SocketAddress, Set<String>> distribution) {
-        int count = 0;
-        for (Set<String> streams : distribution.values()) {
-            count += streams.size();
-        }
-        return count;
-    }
-
-    @Override
-    public void balance(int rebalanceWaterMark,
-                        double rebalanceTolerancePercentage,
-                        int rebalanceConcurrency,
-                        Optional<RateLimiter> rebalanceRateLimiter) {
-        // get the ownership distributions from individual targets
-        Map<SocketAddress, Set<String>> distribution1 = targetMonitor1.getStreamOwnershipDistribution();
-        Map<SocketAddress, Set<String>> distribution2 = targetMonitor2.getStreamOwnershipDistribution();
-
-        // get stream counts
-        int proxyCount1 = distribution1.size();
-        int streamCount1 = countNumberStreams(distribution1);
-        int proxyCount2 = distribution2.size();
-        int streamCount2 = countNumberStreams(distribution2);
-
-        logger.info("'{}' has {} streams by {} proxies; while '{}' has {} streams by {} proxies.",
-                    new Object[] {target1, streamCount1, proxyCount1, target2, streamCount2, proxyCount2 });
-
-        String source, target;
-        Map<SocketAddress, Set<String>> srcDistribution;
-        DistributedLogClient srcClient, targetClient;
-        MonitorServiceClient srcMonitor, targetMonitor;
-        int srcStreamCount, targetStreamCount;
-        if (streamCount1 > streamCount2) {
-            source = target1;
-            srcStreamCount = streamCount1;
-            srcClient = targetClient1;
-            srcMonitor = targetMonitor1;
-            srcDistribution = distribution1;
-
-            target = target2;
-            targetStreamCount = streamCount2;
-            targetClient = targetClient2;
-            targetMonitor = targetMonitor2;
-        } else {
-            source = target2;
-            srcStreamCount = streamCount2;
-            srcClient = targetClient2;
-            srcMonitor = targetMonitor2;
-            srcDistribution = distribution2;
-
-            target = target1;
-            targetStreamCount = streamCount1;
-            targetClient = targetClient1;
-            targetMonitor = targetMonitor1;
-        }
-
-        Map<String, Integer> loadDistribution = new HashMap<String, Integer>();
-        loadDistribution.put(source, srcStreamCount);
-        loadDistribution.put(target, targetStreamCount);
-
-        // Calculate how many streams to be rebalanced from src region to target region
-        int numStreamsToRebalance = BalancerUtils.calculateNumStreamsToRebalance(
-            source, loadDistribution, rebalanceWaterMark, rebalanceTolerancePercentage);
-
-        if (numStreamsToRebalance <= 0) {
-            logger.info("No streams need to be rebalanced from '{}' to '{}'.", source, target);
-            return;
-        }
-
-        StreamChooser streamChooser =
-                LimitedStreamChooser.of(new CountBasedStreamChooser(srcDistribution), numStreamsToRebalance);
-        StreamMover streamMover =
-            new StreamMoverImpl(source, srcClient, srcMonitor, target, targetClient, targetMonitor);
-
-        moveStreams(streamChooser, streamMover, rebalanceConcurrency, rebalanceRateLimiter);
-    }
-
-    @Override
-    public void balanceAll(String source,
-                           int rebalanceConcurrency,
-                           Optional<RateLimiter> rebalanceRateLimiter) {
-        String target;
-        DistributedLogClient sourceClient, targetClient;
-        MonitorServiceClient sourceMonitor, targetMonitor;
-        if (target1.equals(source)) {
-            sourceClient = targetClient1;
-            sourceMonitor = targetMonitor1;
-            target = target2;
-            targetClient = targetClient2;
-            targetMonitor = targetMonitor2;
-        } else if (target2.equals(source)) {
-            sourceClient = targetClient2;
-            sourceMonitor = targetMonitor2;
-            target = target1;
-            targetClient = targetClient1;
-            targetMonitor = targetMonitor1;
-        } else {
-            throw new IllegalArgumentException("Unknown target " + source);
-        }
-
-        // get the ownership distributions from individual targets
-        Map<SocketAddress, Set<String>> distribution = sourceMonitor.getStreamOwnershipDistribution();
-
-        if (distribution.isEmpty()) {
-            return;
-        }
-
-        StreamChooser streamChooser = new CountBasedStreamChooser(distribution);
-        StreamMover streamMover =
-            new StreamMoverImpl(source, sourceClient, sourceMonitor, target, targetClient, targetMonitor);
-
-        moveStreams(streamChooser, streamMover, rebalanceConcurrency, rebalanceRateLimiter);
-    }
-
-    private void moveStreams(StreamChooser streamChooser,
-                             StreamMover streamMover,
-                             int concurrency,
-                             Optional<RateLimiter> rateLimiter) {
-        CountDownLatch doneLatch = new CountDownLatch(concurrency);
-        RegionMover regionMover = new RegionMover(streamChooser, streamMover, rateLimiter, doneLatch);
-        ExecutorService executorService = Executors.newFixedThreadPool(concurrency);
-        try {
-            for (int i = 0; i < concurrency; i++) {
-                executorService.submit(regionMover);
-            }
-
-            try {
-                doneLatch.await();
-            } catch (InterruptedException e) {
-                logger.info("{} is interrupted. Stopping it ...", streamMover);
-                regionMover.shutdown();
-            }
-        } finally {
-            executorService.shutdown();
-        }
-
-    }
-
-    /**
-     * Move streams from <i>src</i> region to <i>target</i> region.
-     */
-    static class RegionMover implements Runnable {
-
-        final StreamChooser streamChooser;
-        final StreamMover streamMover;
-        final Optional<RateLimiter> rateLimiter;
-        final CountDownLatch doneLatch;
-        volatile boolean running = true;
-
-        RegionMover(StreamChooser streamChooser,
-                    StreamMover streamMover,
-                    Optional<RateLimiter> rateLimiter,
-                    CountDownLatch doneLatch) {
-            this.streamChooser = streamChooser;
-            this.streamMover = streamMover;
-            this.rateLimiter = rateLimiter;
-            this.doneLatch = doneLatch;
-        }
-
-        @Override
-        public void run() {
-            while (running) {
-                if (rateLimiter.isPresent()) {
-                    rateLimiter.get().acquire();
-                }
-
-                String stream = streamChooser.choose();
-                if (null == stream) {
-                    break;
-                }
-
-                streamMover.moveStream(stream);
-            }
-            doneLatch.countDown();
-        }
-
-        void shutdown() {
-            running = false;
-        }
-    }
-
-    @Override
-    public void close() {
-        // no-op
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/StreamChooser.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/StreamChooser.java
deleted file mode 100644
index d92aef0..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/StreamChooser.java
+++ /dev/null
@@ -1,30 +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 com.twitter.distributedlog.service.balancer;
-
-/**
- * Choose a stream to rebalance.
- */
-public interface StreamChooser {
-    /**
-     * Choose a stream to rebalance.
-     *
-     * @return stream chose
-     */
-    String choose();
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/StreamMover.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/StreamMover.java
deleted file mode 100644
index 6e4205b..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/StreamMover.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 com.twitter.distributedlog.service.balancer;
-
-/**
- * A stream mover to move streams between proxies.
- */
-public interface StreamMover {
-
-    /**
-     * Move given stream <i>streamName</i>.
-     *
-     * @param streamName
-     *          stream name to move
-     * @return <i>true</i> if successfully moved the stream, <i>false</i> when failure happens.
-     * @throws Exception
-     */
-    boolean moveStream(final String streamName);
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/StreamMoverImpl.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/StreamMoverImpl.java
deleted file mode 100644
index fc67fb2..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/StreamMoverImpl.java
+++ /dev/null
@@ -1,94 +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 com.twitter.distributedlog.service.balancer;
-
-import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.util.Await;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Move Streams from <i>src</i> to <i>target</i>.
- */
-public class StreamMoverImpl implements StreamMover {
-
-    private static final Logger logger = LoggerFactory.getLogger(StreamMoverImpl.class);
-
-    final String source, target;
-    final DistributedLogClient srcClient, targetClient;
-    final MonitorServiceClient srcMonitor, targetMonitor;
-
-    public StreamMoverImpl(String source, DistributedLogClient srcClient, MonitorServiceClient srcMonitor,
-                           String target, DistributedLogClient targetClient, MonitorServiceClient targetMonitor) {
-        this.source = source;
-        this.srcClient = srcClient;
-        this.srcMonitor = srcMonitor;
-        this.target = target;
-        this.targetClient = targetClient;
-        this.targetMonitor = targetMonitor;
-    }
-
-    /**
-     * Move given stream <i>streamName</i>.
-     *
-     * @param streamName
-     *          stream name to move
-     * @return <i>true</i> if successfully moved the stream, <i>false</i> when failure happens.
-     * @throws Exception
-     */
-    public boolean moveStream(final String streamName) {
-        try {
-            doMoveStream(streamName);
-            return true;
-        } catch (Exception e) {
-            return false;
-        }
-    }
-
-    private void doMoveStream(final String streamName) throws Exception {
-        Await.result(srcClient.release(streamName).flatMap(new Function<Void, Future<Void>>() {
-            @Override
-            public Future<Void> apply(Void result) {
-                return targetMonitor.check(streamName).addEventListener(new FutureEventListener<Void>() {
-                    @Override
-                    public void onSuccess(Void value) {
-                        logger.info("Moved stream {} from {} to {}.",
-                                new Object[]{streamName, source, target});
-                    }
-
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        logger.info("Failed to move stream {} from region {} to {} : ",
-                                new Object[]{streamName, source, target, cause});
-                    }
-                });
-            }
-        }));
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("StreamMover('").append(source).append("' -> '").append(target).append("')");
-        return sb.toString();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/package-info.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/package-info.java
deleted file mode 100644
index 4ae8d44..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/balancer/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Balancer to move streams around to balance the traffic.
- */
-package com.twitter.distributedlog.service.balancer;
\ No newline at end of file
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/DefaultStreamConfigProvider.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/DefaultStreamConfigProvider.java
deleted file mode 100644
index b45b798..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/DefaultStreamConfigProvider.java
+++ /dev/null
@@ -1,73 +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 com.twitter.distributedlog.service.config;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.config.ConcurrentConstConfiguration;
-import com.twitter.distributedlog.config.ConfigurationSubscription;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.config.FileConfigurationBuilder;
-import com.twitter.distributedlog.config.PropertiesConfigurationBuilder;
-import java.io.File;
-import java.net.MalformedURLException;
-import java.util.List;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.configuration.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * For all streams return the same dynamic config based on configFile.
- */
-public class DefaultStreamConfigProvider implements StreamConfigProvider {
-
-    private static final Logger LOG = LoggerFactory.getLogger(DefaultStreamConfigProvider.class);
-
-    private final Optional<DynamicDistributedLogConfiguration> dynConf;
-    private final ConfigurationSubscription confSub;
-
-    public DefaultStreamConfigProvider(String configFilePath,
-                                       ScheduledExecutorService executorService,
-                                       int reloadPeriod,
-                                       TimeUnit reloadUnit)
-        throws ConfigurationException {
-        try {
-            File configFile = new File(configFilePath);
-            FileConfigurationBuilder properties =
-                new PropertiesConfigurationBuilder(configFile.toURI().toURL());
-            ConcurrentConstConfiguration defaultConf =
-                new ConcurrentConstConfiguration(new DistributedLogConfiguration());
-            DynamicDistributedLogConfiguration conf =
-                new DynamicDistributedLogConfiguration(defaultConf);
-            List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(properties);
-            confSub = new ConfigurationSubscription(
-                conf, fileConfigBuilders, executorService, reloadPeriod, reloadUnit);
-            this.dynConf = Optional.of(conf);
-        } catch (MalformedURLException ex) {
-            throw new ConfigurationException(ex);
-        }
-    }
-
-    @Override
-    public Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName) {
-        return dynConf;
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/NullStreamConfigProvider.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/NullStreamConfigProvider.java
deleted file mode 100644
index 9e51746..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/NullStreamConfigProvider.java
+++ /dev/null
@@ -1,40 +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 com.twitter.distributedlog.service.config;
-
-import com.google.common.base.Optional;
-
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * For all streams return an absent configuration.
- */
-public class NullStreamConfigProvider implements StreamConfigProvider {
-    static final Logger LOG = LoggerFactory.getLogger(NullStreamConfigProvider.class);
-
-    private static final Optional<DynamicDistributedLogConfiguration> nullConf =
-            Optional.<DynamicDistributedLogConfiguration>absent();
-
-    @Override
-    public Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName) {
-        return nullConf;
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/ServerConfiguration.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/ServerConfiguration.java
deleted file mode 100644
index b3b4c4e..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/ServerConfiguration.java
+++ /dev/null
@@ -1,443 +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 com.twitter.distributedlog.service.config;
-
-import static com.google.common.base.Preconditions.checkArgument;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogConstants;
-import com.twitter.distributedlog.service.streamset.IdentityStreamPartitionConverter;
-import com.twitter.distributedlog.service.streamset.StreamPartitionConverter;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.configuration.CompositeConfiguration;
-import org.apache.commons.configuration.ConfigurationException;
-import org.apache.commons.configuration.SystemConfiguration;
-
-/**
- * Configuration for DistributedLog Server.
- */
-public class ServerConfiguration extends CompositeConfiguration {
-
-    private static ClassLoader defaultLoader;
-
-    static {
-        defaultLoader = Thread.currentThread().getContextClassLoader();
-        if (null == defaultLoader) {
-            defaultLoader = DistributedLogConfiguration.class.getClassLoader();
-        }
-    }
-
-    // Server DLSN version
-    protected static final String SERVER_DLSN_VERSION = "server_dlsn_version";
-    protected static final byte SERVER_DLSN_VERSION_DEFAULT = DLSN.VERSION1;
-
-    // Server Durable Write Enable/Disable Flag
-    protected static final String SERVER_DURABLE_WRITE_ENABLED = "server_durable_write_enabled";
-    protected static final boolean SERVER_DURABLE_WRITE_ENABLED_DEFAULT = true;
-
-    // Server Region Id
-    protected static final String SERVER_REGION_ID = "server_region_id";
-    protected static final int SERVER_REGION_ID_DEFAULT = DistributedLogConstants.LOCAL_REGION_ID;
-
-    // Server Port
-    protected static final String SERVER_PORT = "server_port";
-    protected static final int SERVER_PORT_DEFAULT = 0;
-
-    // Server Shard Id
-    protected static final String SERVER_SHARD_ID = "server_shard";
-    protected static final int SERVER_SHARD_ID_DEFAULT = -1;
-
-    // Server Threads
-    protected static final String SERVER_NUM_THREADS = "server_threads";
-    protected static final int SERVER_NUM_THREADS_DEFAULT = Runtime.getRuntime().availableProcessors();
-
-    // Server enable per stream stat
-    protected static final String SERVER_ENABLE_PERSTREAM_STAT = "server_enable_perstream_stat";
-    protected static final boolean SERVER_ENABLE_PERSTREAM_STAT_DEFAULT = true;
-
-    // Server graceful shutdown period (in millis)
-    protected static final String SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS = "server_graceful_shutdown_period_ms";
-    protected static final long SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS_DEFAULT = 0L;
-
-    // Server service timeout
-    public static final String SERVER_SERVICE_TIMEOUT_MS = "server_service_timeout_ms";
-    public static final String SERVER_SERVICE_TIMEOUT_MS_OLD = "serviceTimeoutMs";
-    public static final long SERVER_SERVICE_TIMEOUT_MS_DEFAULT = 0;
-
-    // Server close writer timeout
-    public static final String SERVER_WRITER_CLOSE_TIMEOUT_MS = "server_writer_close_timeout_ms";
-    public static final long SERVER_WRITER_CLOSE_TIMEOUT_MS_DEFAULT = 1000;
-
-    // Server stream probation timeout
-    public static final String SERVER_STREAM_PROBATION_TIMEOUT_MS = "server_stream_probation_timeout_ms";
-    public static final String SERVER_STREAM_PROBATION_TIMEOUT_MS_OLD = "streamProbationTimeoutMs";
-    public static final long SERVER_STREAM_PROBATION_TIMEOUT_MS_DEFAULT = 60 * 1000 * 5;
-
-    // Server stream to partition converter
-    protected static final String SERVER_STREAM_PARTITION_CONVERTER_CLASS = "stream_partition_converter_class";
-
-    // Use hostname as the allocator pool name
-    protected static final String SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME =
-        "server_use_hostname_as_allocator_pool_name";
-    protected static final boolean SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME_DEFAULT = false;
-    //Configure refresh interval for calculating resource placement in seconds
-    public static final String SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_S =
-        "server_resource_placement_refresh_interval_sec";
-    public static final int  SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_DEFAULT = 120;
-
-    public ServerConfiguration() {
-        super();
-        addConfiguration(new SystemConfiguration());
-    }
-
-    /**
-     * Load configurations from {@link DistributedLogConfiguration}.
-     *
-     * @param dlConf
-     *          distributedlog configuration
-     */
-    public void loadConf(DistributedLogConfiguration dlConf) {
-        addConfiguration(dlConf);
-    }
-
-    /**
-     * Set the version to encode dlsn.
-     *
-     * @param version
-     *          dlsn version
-     * @return server configuration
-     */
-    public ServerConfiguration setDlsnVersion(byte version) {
-        setProperty(SERVER_DLSN_VERSION, version);
-        return this;
-    }
-
-    /**
-     * Get the version to encode dlsn.
-     *
-     * @see DLSN
-     * @return version to encode dlsn.
-     */
-    public byte getDlsnVersion() {
-        return getByte(SERVER_DLSN_VERSION, SERVER_DLSN_VERSION_DEFAULT);
-    }
-
-    /**
-     * Set the flag to enable/disable durable write.
-     *
-     * @param enabled
-     *          flag to enable/disable durable write
-     * @return server configuration
-     */
-    public ServerConfiguration enableDurableWrite(boolean enabled) {
-        setProperty(SERVER_DURABLE_WRITE_ENABLED, enabled);
-        return this;
-    }
-
-    /**
-     * Is durable write enabled.
-     *
-     * @return true if waiting writes to be durable. otherwise false.
-     */
-    public boolean isDurableWriteEnabled() {
-        return getBoolean(SERVER_DURABLE_WRITE_ENABLED, SERVER_DURABLE_WRITE_ENABLED_DEFAULT);
-    }
-
-    /**
-     * Set the region id used to instantiate DistributedLogNamespace.
-     *
-     * @param regionId
-     *          region id
-     * @return server configuration
-     */
-    public ServerConfiguration setRegionId(int regionId) {
-        setProperty(SERVER_REGION_ID, regionId);
-        return this;
-    }
-
-    /**
-     * Get the region id used to instantiate {@link com.twitter.distributedlog.namespace.DistributedLogNamespace}.
-     *
-     * @return region id used to instantiate DistributedLogNamespace
-     */
-    public int getRegionId() {
-        return getInt(SERVER_REGION_ID, SERVER_REGION_ID_DEFAULT);
-    }
-
-    /**
-     * Set the server port running for this service.
-     *
-     * @param port
-     *          server port
-     * @return server configuration
-     */
-    public ServerConfiguration setServerPort(int port) {
-        setProperty(SERVER_PORT, port);
-        return this;
-    }
-
-    /**
-     * Get the server port running for this service.
-     *
-     * @return server port
-     */
-    public int getServerPort() {
-        return getInt(SERVER_PORT, SERVER_PORT_DEFAULT);
-    }
-
-    /**
-     * Set the shard id of this server.
-     *
-     * @param shardId
-     *          shard id
-     * @return shard id of this server
-     */
-    public ServerConfiguration setServerShardId(int shardId) {
-        setProperty(SERVER_SHARD_ID, shardId);
-        return this;
-    }
-
-    /**
-     * Get the shard id of this server.
-     *
-     * <p>It would be used to instantiate the client id used for DistributedLogNamespace.
-     *
-     * @return shard id of this server.
-     */
-    public int getServerShardId() {
-        return getInt(SERVER_SHARD_ID, SERVER_SHARD_ID_DEFAULT);
-    }
-
-    /**
-     * Get the number of threads for the executor of this server.
-     *
-     * @return number of threads for the executor running in this server.
-     */
-    public int getServerThreads() {
-        return getInt(SERVER_NUM_THREADS, SERVER_NUM_THREADS_DEFAULT);
-    }
-
-    /**
-     * Set the number of threads for the executor of this server.
-     *
-     * @param numThreads
-     *          number of threads for the executor running in this server.
-     * @return server configuration
-     */
-    public ServerConfiguration setServerThreads(int numThreads) {
-        setProperty(SERVER_NUM_THREADS, numThreads);
-        return this;
-    }
-
-    /**
-     * Enable/Disable per stream stat.
-     *
-     * @param enabled
-     *          flag to enable/disable per stream stat
-     * @return server configuration
-     */
-    public ServerConfiguration setPerStreamStatEnabled(boolean enabled) {
-        setProperty(SERVER_ENABLE_PERSTREAM_STAT, enabled);
-        return this;
-    }
-
-    /**
-     * Whether the per stream stat enabled for not in this server.
-     *
-     * @return true if per stream stat enable, otherwise false.
-     */
-    public boolean isPerStreamStatEnabled() {
-        return getBoolean(SERVER_ENABLE_PERSTREAM_STAT, SERVER_ENABLE_PERSTREAM_STAT_DEFAULT);
-    }
-
-    /**
-     * Set the graceful shutdown period in millis.
-     *
-     * @param periodMs
-     *          graceful shutdown period in millis.
-     * @return server configuration
-     */
-    public ServerConfiguration setGracefulShutdownPeriodMs(long periodMs) {
-        setProperty(SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS, periodMs);
-        return this;
-    }
-
-    /**
-     * Get the graceful shutdown period in millis.
-     *
-     * @return graceful shutdown period in millis.
-     */
-    public long getGracefulShutdownPeriodMs() {
-        return getLong(SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS, SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS_DEFAULT);
-    }
-
-    /**
-     * Get timeout for stream op execution in proxy layer.
-     *
-     * <p>0 disables timeout.
-     *
-     * @return timeout for stream operation in proxy layer.
-     */
-    public long getServiceTimeoutMs() {
-        return getLong(SERVER_SERVICE_TIMEOUT_MS,
-                getLong(SERVER_SERVICE_TIMEOUT_MS_OLD, SERVER_SERVICE_TIMEOUT_MS_DEFAULT));
-    }
-
-    /**
-     * Set timeout for stream op execution in proxy layer.
-     *
-     * <p>0 disables timeout.
-     *
-     * @param timeoutMs
-     *          timeout for stream operation in proxy layer.
-     * @return dl configuration.
-     */
-    public ServerConfiguration setServiceTimeoutMs(long timeoutMs) {
-        setProperty(SERVER_SERVICE_TIMEOUT_MS, timeoutMs);
-        return this;
-    }
-
-    /**
-     * Get timeout for closing writer in proxy layer.
-     *
-     * <p>0 disables timeout.
-     *
-     * @return timeout for closing writer in proxy layer.
-     */
-    public long getWriterCloseTimeoutMs() {
-        return getLong(SERVER_WRITER_CLOSE_TIMEOUT_MS, SERVER_WRITER_CLOSE_TIMEOUT_MS_DEFAULT);
-    }
-
-    /**
-     * Set timeout for closing writer in proxy layer.
-     *
-     * <p>0 disables timeout.
-     *
-     * @param timeoutMs
-     *          timeout for closing writer in proxy layer.
-     * @return dl configuration.
-     */
-    public ServerConfiguration setWriterCloseTimeoutMs(long timeoutMs) {
-        setProperty(SERVER_WRITER_CLOSE_TIMEOUT_MS, timeoutMs);
-        return this;
-    }
-
-    /**
-     * How long should stream be kept in cache in probationary state after service timeout.
-     *
-     * <p>The setting is to prevent reacquire. The unit of this setting is milliseconds.
-     *
-     * @return stream probation timeout in ms.
-     */
-    public long getStreamProbationTimeoutMs() {
-        return getLong(SERVER_STREAM_PROBATION_TIMEOUT_MS,
-                getLong(SERVER_STREAM_PROBATION_TIMEOUT_MS_OLD, SERVER_STREAM_PROBATION_TIMEOUT_MS_DEFAULT));
-    }
-
-    /**
-     * How long should stream be kept in cache in probationary state after service timeout.
-     *
-     * <p>The setting is to prevent reacquire. The unit of this setting is milliseconds.
-     *
-     * @param timeoutMs probation timeout in ms.
-     * @return server configuration
-     */
-    public ServerConfiguration setStreamProbationTimeoutMs(long timeoutMs) {
-        setProperty(SERVER_STREAM_PROBATION_TIMEOUT_MS, timeoutMs);
-        return this;
-    }
-
-    /**
-     * Set the stream partition converter class.
-     *
-     * @param converterClass
-     *          stream partition converter class
-     * @return server configuration
-     */
-    public ServerConfiguration setStreamPartitionConverterClass(
-        Class<? extends StreamPartitionConverter> converterClass) {
-        setProperty(SERVER_STREAM_PARTITION_CONVERTER_CLASS, converterClass.getName());
-        return this;
-    }
-
-    /**
-     * Get the stream partition converter class.
-     *
-     * @return the stream partition converter class.
-     * @throws ConfigurationException
-     */
-    public Class<? extends StreamPartitionConverter> getStreamPartitionConverterClass()
-            throws ConfigurationException {
-        return ReflectionUtils.getClass(
-                this,
-                SERVER_STREAM_PARTITION_CONVERTER_CLASS,
-                IdentityStreamPartitionConverter.class,
-                StreamPartitionConverter.class,
-                defaultLoader);
-    }
-
-     /**
-      * Set if use hostname as the allocator pool name.
-      *
-      * @param useHostname whether to use hostname as the allocator pool name.
-      * @return server configuration
-      * @see #isUseHostnameAsAllocatorPoolName()
-      */
-    public ServerConfiguration setUseHostnameAsAllocatorPoolName(boolean useHostname) {
-        setProperty(SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME, useHostname);
-        return this;
-    }
-
-    /**
-     * Get if use hostname as the allocator pool name.
-     *
-     * @return true if use hostname as the allocator pool name. otherwise, use
-     * {@link #getServerShardId()} as the allocator pool name.
-     * @see #getServerShardId()
-     */
-    public boolean isUseHostnameAsAllocatorPoolName() {
-        return getBoolean(SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME,
-            SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME_DEFAULT);
-    }
-
-    public ServerConfiguration setResourcePlacementRefreshInterval(int refreshIntervalSecs) {
-        setProperty(SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_S, refreshIntervalSecs);
-        return this;
-    }
-
-    public int getResourcePlacementRefreshInterval() {
-        return getInt(SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_S, SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_DEFAULT);
-    }
-
-    /**
-     * Validate the configuration.
-     *
-     * @throws IllegalStateException when there are any invalid settings.
-     */
-    public void validate() {
-        byte dlsnVersion = getDlsnVersion();
-        checkArgument(dlsnVersion >= DLSN.VERSION0 && dlsnVersion <= DLSN.VERSION1,
-                "Unknown dlsn version " + dlsnVersion);
-        checkArgument(getServerThreads() > 0,
-                "Invalid number of server threads : " + getServerThreads());
-        checkArgument(getServerShardId() >= 0,
-                "Invalid server shard id : " + getServerShardId());
-    }
-
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/ServiceStreamConfigProvider.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/ServiceStreamConfigProvider.java
deleted file mode 100644
index 0ee7db4..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/ServiceStreamConfigProvider.java
+++ /dev/null
@@ -1,88 +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 com.twitter.distributedlog.service.config;
-
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.config.DynamicConfigurationFactory;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.service.streamset.StreamPartitionConverter;
-import java.io.File;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.apache.commons.configuration.ConfigurationException;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Provide per stream configuration to DistributedLog service layer.
- */
-public class ServiceStreamConfigProvider implements StreamConfigProvider {
-
-    private static final Logger LOG = LoggerFactory.getLogger(ServiceStreamConfigProvider.class);
-
-    private static final String CONFIG_EXTENSION = "conf";
-
-    private final File configBaseDir;
-    private final File defaultConfigFile;
-    private final StreamPartitionConverter partitionConverter;
-    private final DynamicConfigurationFactory configFactory;
-    private final DynamicDistributedLogConfiguration defaultDynConf;
-
-    public ServiceStreamConfigProvider(String configPath,
-                                       String defaultConfigPath,
-                                       StreamPartitionConverter partitionConverter,
-                                       ScheduledExecutorService executorService,
-                                       int reloadPeriod,
-                                       TimeUnit reloadUnit)
-                                       throws ConfigurationException {
-        this.configBaseDir = new File(configPath);
-        if (!configBaseDir.exists()) {
-            throw new ConfigurationException("Stream configuration base directory "
-                + configPath + " does not exist");
-        }
-        this.defaultConfigFile = new File(configPath);
-        if (!defaultConfigFile.exists()) {
-            throw new ConfigurationException("Stream configuration default config "
-                + defaultConfigPath + " does not exist");
-        }
-
-        // Construct reloading default configuration
-        this.partitionConverter = partitionConverter;
-        this.configFactory = new DynamicConfigurationFactory(executorService, reloadPeriod, reloadUnit);
-        // We know it exists from the check above.
-        this.defaultDynConf = configFactory.getDynamicConfiguration(defaultConfigPath).get();
-    }
-
-    @Override
-    public Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName) {
-        String configName = partitionConverter.convert(streamName).getStream();
-        String configPath = getConfigPath(configName);
-        Optional<DynamicDistributedLogConfiguration> dynConf = Optional.<DynamicDistributedLogConfiguration>absent();
-        try {
-            dynConf = configFactory.getDynamicConfiguration(configPath, defaultDynConf);
-        } catch (ConfigurationException ex) {
-            LOG.warn("Configuration exception for stream {} ({}) at {}",
-                    new Object[] {streamName, configName, configPath, ex});
-        }
-        return dynConf;
-    }
-
-    private String getConfigPath(String configName) {
-        return new File(configBaseDir, String.format("%s.%s", configName, CONFIG_EXTENSION)).getPath();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/StreamConfigProvider.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/StreamConfigProvider.java
deleted file mode 100644
index 949dc70..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/StreamConfigProvider.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 com.twitter.distributedlog.service.config;
-
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-
-/**
- * Expose per-stream configs to dl proxy.
- */
-public interface StreamConfigProvider {
-    /**
-     * Get dynamic per stream config overrides for a given stream.
-     *
-     * @param streamName stream name to return config for
-     * @return Optional dynamic configuration instance
-     */
-    Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName);
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/package-info.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/package-info.java
deleted file mode 100644
index bb0026a..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/config/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Server Configurations.
- */
-package com.twitter.distributedlog.service.config;
\ No newline at end of file
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/package-info.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/package-info.java
deleted file mode 100644
index 4fb3673..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * DistributedLog Proxy Service.
- */
-package com.twitter.distributedlog.service;
\ No newline at end of file
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/EqualLoadAppraiser.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/EqualLoadAppraiser.java
deleted file mode 100644
index fb2d6d2..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/EqualLoadAppraiser.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
- * <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 com.twitter.distributedlog.service.placement;
-
-import com.twitter.util.Future;
-
-/**
- * Equal Load Appraiser.
- *
- * <p>Created for those who hold these truths to be self-evident, that all streams are created equal,
- * that they are endowed by their creator with certain unalienable loads, that among these are
- * Uno, Eins, and One.
- */
-public class EqualLoadAppraiser implements LoadAppraiser {
-    @Override
-    public Future<StreamLoad> getStreamLoad(String stream) {
-        return Future.value(new StreamLoad(stream, 1));
-    }
-
-    @Override
-    public Future<Void> refreshCache() {
-        return Future.value(null);
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/LeastLoadPlacementPolicy.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/LeastLoadPlacementPolicy.java
deleted file mode 100644
index c25c267..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/LeastLoadPlacementPolicy.java
+++ /dev/null
@@ -1,200 +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
- * <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 com.twitter.distributedlog.service.placement;
-
-import com.twitter.distributedlog.client.routing.RoutingService;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.util.Duration;
-import com.twitter.util.Function;
-import com.twitter.util.Future;
-import com.twitter.util.Futures;
-import java.util.ArrayList;
-import java.util.Collection;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.TreeSet;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.BoxedUnit;
-
-/**
- * Least Load Placement Policy.
- *
- * <p>A LoadPlacementPolicy that attempts to place streams in such a way that the load is balanced as
- * evenly as possible across all shards. The LoadAppraiser remains responsible for determining what
- * the load of a server would be. This placement policy then distributes these streams across the
- * servers.
- */
-public class LeastLoadPlacementPolicy extends PlacementPolicy {
-
-    private static final Logger logger = LoggerFactory.getLogger(LeastLoadPlacementPolicy.class);
-
-    private TreeSet<ServerLoad> serverLoads = new TreeSet<ServerLoad>();
-    private Map<String, String> streamToServer = new HashMap<String, String>();
-
-    public LeastLoadPlacementPolicy(LoadAppraiser loadAppraiser, RoutingService routingService,
-                                    DistributedLogNamespace namespace, PlacementStateManager placementStateManager,
-                                    Duration refreshInterval, StatsLogger statsLogger) {
-        super(loadAppraiser, routingService, namespace, placementStateManager, refreshInterval, statsLogger);
-        statsLogger.registerGauge("placement/load.diff", new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return 0;
-            }
-
-            @Override
-            public Number getSample() {
-                if (serverLoads.size() > 0) {
-                    return serverLoads.last().getLoad() - serverLoads.first().getLoad();
-                } else {
-                    return getDefaultValue();
-                }
-            }
-        });
-    }
-
-    private synchronized String getStreamOwner(String stream) {
-        return streamToServer.get(stream);
-    }
-
-    @Override
-    public Future<String> placeStream(String stream) {
-        String streamOwner = getStreamOwner(stream);
-        if (null != streamOwner) {
-            return Future.value(streamOwner);
-        }
-        Future<StreamLoad> streamLoadFuture = loadAppraiser.getStreamLoad(stream);
-        return streamLoadFuture.map(new Function<StreamLoad, String>() {
-            @Override
-            public String apply(StreamLoad streamLoad) {
-                return placeStreamSynchronized(streamLoad);
-            }
-        });
-    }
-
-    private synchronized String placeStreamSynchronized(StreamLoad streamLoad) {
-        ServerLoad serverLoad = serverLoads.pollFirst();
-        serverLoad.addStream(streamLoad);
-        serverLoads.add(serverLoad);
-        return serverLoad.getServer();
-    }
-
-    @Override
-    public void refresh() {
-        logger.info("Refreshing server loads.");
-        Future<Void> refresh = loadAppraiser.refreshCache();
-        final Set<String> servers = getServers();
-        final Set<String> allStreams = getStreams();
-        Future<TreeSet<ServerLoad>> serverLoadsFuture = refresh.flatMap(
-            new Function<Void, Future<TreeSet<ServerLoad>>>() {
-            @Override
-            public Future<TreeSet<ServerLoad>> apply(Void v1) {
-                return calculate(servers, allStreams);
-            }
-        });
-        serverLoadsFuture.map(new Function<TreeSet<ServerLoad>, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(TreeSet<ServerLoad> serverLoads) {
-                try {
-                    updateServerLoads(serverLoads);
-                } catch (PlacementStateManager.StateManagerSaveException e) {
-                    logger.error("The refreshed mapping could not be persisted and will not be used.", e);
-                }
-                return BoxedUnit.UNIT;
-            }
-        });
-    }
-
-    private synchronized void updateServerLoads(TreeSet<ServerLoad> serverLoads)
-        throws PlacementStateManager.StateManagerSaveException {
-        this.placementStateManager.saveOwnership(serverLoads);
-        this.streamToServer = serverLoadsToMap(serverLoads);
-        this.serverLoads = serverLoads;
-    }
-
-    @Override
-    public synchronized void load(TreeSet<ServerLoad> serverLoads) {
-        this.serverLoads = serverLoads;
-        this.streamToServer = serverLoadsToMap(serverLoads);
-    }
-
-    public Future<TreeSet<ServerLoad>> calculate(final Set<String> servers, Set<String> streams) {
-        logger.info("Calculating server loads");
-        final long startTime = System.currentTimeMillis();
-        ArrayList<Future<StreamLoad>> futures = new ArrayList<Future<StreamLoad>>(streams.size());
-
-        for (String stream : streams) {
-            Future<StreamLoad> streamLoad = loadAppraiser.getStreamLoad(stream);
-            futures.add(streamLoad);
-        }
-
-        return Futures.collect(futures).map(new Function<List<StreamLoad>, TreeSet<ServerLoad>>() {
-            @Override
-            public TreeSet<ServerLoad> apply(List<StreamLoad> streamLoads) {
-        /* Sort streamLoads so largest streams are placed first for better balance */
-                TreeSet<StreamLoad> streamQueue = new TreeSet<StreamLoad>();
-                for (StreamLoad streamLoad : streamLoads) {
-                    streamQueue.add(streamLoad);
-                }
-
-                TreeSet<ServerLoad> serverLoads = new TreeSet<ServerLoad>();
-                for (String server : servers) {
-                    ServerLoad serverLoad = new ServerLoad(server);
-                    if (!streamQueue.isEmpty()) {
-                        serverLoad.addStream(streamQueue.pollFirst());
-                    }
-                    serverLoads.add(serverLoad);
-                }
-
-                while (!streamQueue.isEmpty()) {
-                    ServerLoad serverLoad = serverLoads.pollFirst();
-                    serverLoad.addStream(streamQueue.pollFirst());
-                    serverLoads.add(serverLoad);
-                }
-                return serverLoads;
-            }
-        }).onSuccess(new Function<TreeSet<ServerLoad>, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(TreeSet<ServerLoad> serverLoads) {
-                placementCalcStats.registerSuccessfulEvent(System.currentTimeMillis() - startTime);
-                return BoxedUnit.UNIT;
-            }
-        }).onFailure(new Function<Throwable, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Throwable t) {
-                logger.error("Failure calculating loads", t);
-                placementCalcStats.registerFailedEvent(System.currentTimeMillis() - startTime);
-                return BoxedUnit.UNIT;
-            }
-        });
-    }
-
-    private static Map<String, String> serverLoadsToMap(Collection<ServerLoad> serverLoads) {
-        HashMap<String, String> streamToServer = new HashMap<String, String>(serverLoads.size());
-        for (ServerLoad serverLoad : serverLoads) {
-            for (StreamLoad streamLoad : serverLoad.getStreamLoads()) {
-                streamToServer.put(streamLoad.getStream(), serverLoad.getServer());
-            }
-        }
-        return streamToServer;
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/LoadAppraiser.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/LoadAppraiser.java
deleted file mode 100644
index 53c568a..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/LoadAppraiser.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 com.twitter.distributedlog.service.placement;
-
-import com.twitter.util.Future;
-
-/**
- * Interface for load appraiser.
- */
-public interface LoadAppraiser {
-    /**
-     * Retrieve the stream load for a given {@code stream}.
-     *
-     * @param stream name of the stream
-     * @return the stream load of the stream.
-     */
-    Future<StreamLoad> getStreamLoad(String stream);
-
-    /**
-     * Refesch the cache.
-     * @return
-     */
-    Future<Void> refreshCache();
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/PlacementPolicy.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/PlacementPolicy.java
deleted file mode 100644
index 46e8940..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/PlacementPolicy.java
+++ /dev/null
@@ -1,148 +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
- * <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 com.twitter.distributedlog.service.placement;
-
-import com.twitter.distributedlog.client.routing.RoutingService;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.util.Duration;
-import com.twitter.util.Function0;
-import com.twitter.util.Future;
-import com.twitter.util.ScheduledThreadPoolTimer;
-import com.twitter.util.Time;
-import com.twitter.util.Timer;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-import java.util.TreeSet;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.BoxedUnit;
-
-/**
- * A PlacementPolicy assigns streams to servers given an appraisal of the load that the stream contains.
- *
- * <p>The load of a stream is determined by the LoadAppraiser used. The PlacementPolicy will
- * then distributed these StreamLoads to the available servers in a manner defined by the
- * implementation creating ServerLoad objects. It then saves this assignment via the
- * PlacementStateManager.
- */
-public abstract class PlacementPolicy {
-
-    private static final Logger logger = LoggerFactory.getLogger(PlacementPolicy.class);
-
-    protected final LoadAppraiser loadAppraiser;
-    protected final RoutingService routingService;
-    protected final DistributedLogNamespace 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,
-                           Duration refreshInterval, StatsLogger statsLogger) {
-        this.loadAppraiser = loadAppraiser;
-        this.routingService = routingService;
-        this.namespace = namespace;
-        this.placementStateManager = placementStateManager;
-        this.refreshInterval = refreshInterval;
-        placementCalcStats = statsLogger.getOpStatsLogger("placement");
-    }
-
-    public Set<String> getServers() {
-        Set<SocketAddress> hosts = routingService.getHosts();
-        Set<String> servers = new HashSet<String>(hosts.size());
-        for (SocketAddress address : hosts) {
-            servers.add(DLSocketAddress.toString((InetSocketAddress) address));
-        }
-        return servers;
-    }
-
-    public Set<String> getStreams() {
-        Set<String> streams = new HashSet<String>();
-        try {
-            Iterator<String> logs = namespace.getLogs();
-            while (logs.hasNext()) {
-                streams.add(logs.next());
-            }
-        } catch (IOException e) {
-            logger.error("Could not get streams for placement policy.", e);
-        }
-        return streams;
-    }
-
-    public void start(boolean leader) {
-        logger.info("Starting placement policy");
-
-        TreeSet<ServerLoad> emptyServerLoads = new TreeSet<ServerLoad>();
-        for (String server : getServers()) {
-            emptyServerLoads.add(new ServerLoad(server));
-        }
-        load(emptyServerLoads); //Pre-Load so streams don't NPE
-        if (leader) { //this is the leader shard
-            logger.info("Shard is leader. Scheduling timed refresh.");
-            placementRefreshTimer = new ScheduledThreadPoolTimer(1, "timer", true);
-            placementRefreshTimer.schedule(Time.now(), refreshInterval, new Function0<BoxedUnit>() {
-                @Override
-                public BoxedUnit apply() {
-                    refresh();
-                    return BoxedUnit.UNIT;
-                }
-            });
-        } else {
-            logger.info("Shard is not leader. Watching for server load changes.");
-            placementStateManager.watch(new PlacementStateManager.PlacementCallback() {
-                @Override
-                public void callback(TreeSet<ServerLoad> serverLoads) {
-                    if (!serverLoads.isEmpty()) {
-                        load(serverLoads);
-                    }
-                }
-            });
-        }
-    }
-
-    public void close() {
-        if (placementRefreshTimer != null) {
-            placementRefreshTimer.stop();
-        }
-    }
-
-    /**
-     * Places the stream on a server according to the policy.
-     *
-     * <p>It returns a future containing the host that owns the stream upon completion
-     */
-    public abstract Future<String> placeStream(String stream);
-
-    /**
-     * Recalculates the entire placement mapping and updates stores it using the PlacementStateManager.
-     */
-    public abstract void refresh();
-
-    /**
-     * Loads the placement mapping into the node from a TreeSet of ServerLoads.
-     */
-    public abstract void load(TreeSet<ServerLoad> serverLoads);
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/PlacementStateManager.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/PlacementStateManager.java
deleted file mode 100644
index 17e4685..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/PlacementStateManager.java
+++ /dev/null
@@ -1,79 +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
- * <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 com.twitter.distributedlog.service.placement;
-
-import java.util.TreeSet;
-
-/**
- * The PlacementStateManager handles persistence of calculated resource placements.
- */
-public interface PlacementStateManager {
-
-    /**
-     * Saves the ownership mapping as a TreeSet of ServerLoads to persistent storage.
-     */
-    void saveOwnership(TreeSet<ServerLoad> serverLoads) throws StateManagerSaveException;
-
-    /**
-     * Loads the ownership mapping as TreeSet of ServerLoads from persistent storage.
-     */
-    TreeSet<ServerLoad> loadOwnership() throws StateManagerLoadException;
-
-    /**
-     * Watch the persistent storage for changes to the ownership mapping.
-     *
-     * <p>The placementCallback callbacks will be triggered with the new mapping when a change occurs.
-     */
-    void watch(PlacementCallback placementCallback);
-
-    /**
-     * Placement Callback.
-     *
-     * <p>The callback is triggered when server loads are updated.
-     */
-    interface PlacementCallback {
-        void callback(TreeSet<ServerLoad> serverLoads);
-    }
-
-    /**
-     * The base exception thrown when state manager encounters errors.
-     */
-    abstract class StateManagerException extends Exception {
-        public StateManagerException(String message, Exception e) {
-            super(message, e);
-        }
-    }
-
-    /**
-     * Exception thrown when failed to load the ownership mapping.
-     */
-    class StateManagerLoadException extends StateManagerException {
-        public StateManagerLoadException(Exception e) {
-            super("Load of Ownership failed", e);
-        }
-    }
-
-    /**
-     * Exception thrown when failed to save the ownership mapping.
-     */
-    class StateManagerSaveException extends StateManagerException {
-        public StateManagerSaveException(Exception e) {
-            super("Save of Ownership failed", e);
-        }
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ServerLoad.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ServerLoad.java
deleted file mode 100644
index a0b4959..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ServerLoad.java
+++ /dev/null
@@ -1,158 +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
- * <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 com.twitter.distributedlog.service.placement;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.Set;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TJSONProtocol;
-import org.apache.thrift.transport.TMemoryBuffer;
-import org.apache.thrift.transport.TMemoryInputTransport;
-
-/**
- * An object represents the server load.
- *
- * <p>A comparable data object containing the identifier of the server, total appraised load on the
- * server, and all streams assigned to the server by the resource placement mapping. This is
- * comparable first by load and then by server so that a sorted data structure of these will be
- * consistent across multiple calculations.
- */
-public class ServerLoad implements Comparable {
-    private static final int BUFFER_SIZE = 4096000;
-    private final String server;
-    private final HashSet<StreamLoad> streamLoads = new HashSet<StreamLoad>();
-    private long load = 0L;
-
-    public ServerLoad(String server) {
-        this.server = server;
-    }
-
-    public synchronized long addStream(StreamLoad stream) {
-        this.load += stream.getLoad();
-        streamLoads.add(stream);
-        return this.load;
-    }
-
-    public synchronized long removeStream(String stream) {
-        for (StreamLoad streamLoad : streamLoads) {
-            if (streamLoad.stream.equals(stream)) {
-                this.load -= streamLoad.getLoad();
-                streamLoads.remove(streamLoad);
-                return this.load;
-            }
-        }
-        return this.load; //Throwing an exception wouldn't help us as our logic should never reach here
-    }
-
-    public synchronized long getLoad() {
-        return load;
-    }
-
-    public synchronized Set<StreamLoad> getStreamLoads() {
-        return streamLoads;
-    }
-
-    public synchronized String getServer() {
-        return server;
-    }
-
-    protected synchronized com.twitter.distributedlog.service.placement.thrift.ServerLoad toThrift() {
-        com.twitter.distributedlog.service.placement.thrift.ServerLoad tServerLoad =
-            new com.twitter.distributedlog.service.placement.thrift.ServerLoad();
-        tServerLoad.setServer(server);
-        tServerLoad.setLoad(load);
-        ArrayList<com.twitter.distributedlog.service.placement.thrift.StreamLoad> tStreamLoads =
-            new ArrayList<com.twitter.distributedlog.service.placement.thrift.StreamLoad>();
-        for (StreamLoad streamLoad : streamLoads) {
-            tStreamLoads.add(streamLoad.toThrift());
-        }
-        tServerLoad.setStreams(tStreamLoads);
-        return tServerLoad;
-    }
-
-    public byte[] serialize() throws IOException {
-        TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
-        TJSONProtocol protocol = new TJSONProtocol(transport);
-        try {
-            toThrift().write(protocol);
-            transport.flush();
-            return transport.toString(UTF_8.name()).getBytes(UTF_8);
-        } catch (TException e) {
-            throw new IOException("Failed to serialize server load : ", e);
-        } catch (UnsupportedEncodingException uee) {
-            throw new IOException("Failed to serialize server load : ", uee);
-        }
-    }
-
-    public static ServerLoad deserialize(byte[] data) throws IOException {
-        com.twitter.distributedlog.service.placement.thrift.ServerLoad tServerLoad =
-            new com.twitter.distributedlog.service.placement.thrift.ServerLoad();
-        TMemoryInputTransport transport = new TMemoryInputTransport(data);
-        TJSONProtocol protocol = new TJSONProtocol(transport);
-        try {
-            tServerLoad.read(protocol);
-            ServerLoad serverLoad = new ServerLoad(tServerLoad.getServer());
-            if (tServerLoad.isSetStreams()) {
-                for (com.twitter.distributedlog.service.placement.thrift.StreamLoad tStreamLoad :
-                    tServerLoad.getStreams()) {
-                    serverLoad.addStream(new StreamLoad(tStreamLoad.getStream(), tStreamLoad.getLoad()));
-                }
-            }
-            return serverLoad;
-        } catch (TException e) {
-            throw new IOException("Failed to deserialize server load : ", e);
-        }
-    }
-
-    @Override
-    public synchronized int compareTo(Object o) {
-        ServerLoad other = (ServerLoad) o;
-        if (load == other.getLoad()) {
-            return server.compareTo(other.getServer());
-        } else {
-            return Long.compare(load, other.getLoad());
-        }
-    }
-
-    @Override
-    public synchronized boolean equals(Object o) {
-        if (!(o instanceof ServerLoad)) {
-            return false;
-        }
-        ServerLoad other = (ServerLoad) o;
-        return server.equals(other.getServer())
-            && load == other.getLoad()
-            && streamLoads.equals(other.getStreamLoads());
-    }
-
-    @Override
-    public synchronized String toString() {
-        return String.format("ServerLoad<Server: %s, Load: %d, Streams: %s>", server, load, streamLoads);
-    }
-
-    @Override
-    public synchronized int hashCode() {
-        return new HashCodeBuilder().append(server).append(load).append(streamLoads).build();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/StreamLoad.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/StreamLoad.java
deleted file mode 100644
index c0b0ce1..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/StreamLoad.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
- * <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 com.twitter.distributedlog.service.placement;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-import java.io.IOException;
-import java.io.UnsupportedEncodingException;
-import org.apache.commons.lang3.builder.HashCodeBuilder;
-import org.apache.thrift.TException;
-import org.apache.thrift.protocol.TJSONProtocol;
-import org.apache.thrift.transport.TMemoryBuffer;
-import org.apache.thrift.transport.TMemoryInputTransport;
-
-/**
- * An object represent the load of a stream.
- *
- * <p>A comparable data object containing the identifier of the stream and the appraised load produced
- * by the stream.
- */
-public class StreamLoad implements Comparable {
-    private static final int BUFFER_SIZE = 4096;
-    public final String stream;
-    private final int load;
-
-    public StreamLoad(String stream, int load) {
-        this.stream = stream;
-        this.load = load;
-    }
-
-    public int getLoad() {
-        return load;
-    }
-
-    public String getStream() {
-        return stream;
-    }
-
-    protected com.twitter.distributedlog.service.placement.thrift.StreamLoad toThrift() {
-        com.twitter.distributedlog.service.placement.thrift.StreamLoad tStreamLoad =
-            new com.twitter.distributedlog.service.placement.thrift.StreamLoad();
-        return tStreamLoad.setStream(stream).setLoad(load);
-    }
-
-    public byte[] serialize() throws IOException {
-        TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
-        TJSONProtocol protocol = new TJSONProtocol(transport);
-        try {
-            toThrift().write(protocol);
-            transport.flush();
-            return transport.toString(UTF_8.name()).getBytes(UTF_8);
-        } catch (TException e) {
-            throw new IOException("Failed to serialize stream load : ", e);
-        } catch (UnsupportedEncodingException uee) {
-            throw new IOException("Failed to serialize stream load : ", uee);
-        }
-    }
-
-    public static StreamLoad deserialize(byte[] data) throws IOException {
-        com.twitter.distributedlog.service.placement.thrift.StreamLoad tStreamLoad =
-            new com.twitter.distributedlog.service.placement.thrift.StreamLoad();
-        TMemoryInputTransport transport = new TMemoryInputTransport(data);
-        TJSONProtocol protocol = new TJSONProtocol(transport);
-        try {
-            tStreamLoad.read(protocol);
-            return new StreamLoad(tStreamLoad.getStream(), tStreamLoad.getLoad());
-        } catch (TException e) {
-            throw new IOException("Failed to deserialize stream load : ", e);
-        }
-    }
-
-    @Override
-    public int compareTo(Object o) {
-        StreamLoad other = (StreamLoad) o;
-        if (load == other.getLoad()) {
-            return stream.compareTo(other.getStream());
-        } else {
-            return Long.compare(load, other.getLoad());
-        }
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (!(o instanceof StreamLoad)) {
-            return false;
-        }
-        StreamLoad other = (StreamLoad) o;
-        return stream.equals(other.getStream()) && load == other.getLoad();
-    }
-
-    @Override
-    public String toString() {
-        return String.format("StreamLoad<Stream: %s, Load: %d>", stream, load);
-    }
-
-    @Override
-    public int hashCode() {
-        return new HashCodeBuilder().append(stream).append(load).build();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ZKPlacementStateManager.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ZKPlacementStateManager.java
deleted file mode 100644
index 977ae04..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/ZKPlacementStateManager.java
+++ /dev/null
@@ -1,173 +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
- * <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 com.twitter.distributedlog.service.placement;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.util.Utils;
-import java.io.IOException;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.HashSet;
-import java.util.List;
-import java.util.TreeSet;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.Transaction;
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-import org.apache.zookeeper.data.Stat;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * An implementation of the PlacementStateManager that saves data to and loads from Zookeeper to
- * avoid necessitating an additional system for the resource placement.
- */
-public class ZKPlacementStateManager implements PlacementStateManager {
-
-    private static final Logger logger = LoggerFactory.getLogger(ZKPlacementStateManager.class);
-
-    private static final String SERVER_LOAD_DIR = "/.server-load";
-
-    private final String serverLoadPath;
-    private final ZooKeeperClient zkClient;
-
-    private boolean watching = false;
-
-    public ZKPlacementStateManager(URI uri, DistributedLogConfiguration conf, StatsLogger statsLogger) {
-        String zkServers = BKNamespaceDriver.getZKServersFromDLUri(uri);
-        zkClient = BKNamespaceDriver.createZKClientBuilder(
-            String.format("ZKPlacementStateManager-%s", zkServers),
-            conf,
-            zkServers,
-            statsLogger.scope("placement_state_manager")).build();
-        serverLoadPath = uri.getPath() + SERVER_LOAD_DIR;
-    }
-
-    private void createServerLoadPathIfNoExists(byte[] data)
-        throws ZooKeeperClient.ZooKeeperConnectionException, KeeperException, InterruptedException {
-        try {
-            Utils.zkCreateFullPathOptimistic(
-                zkClient, serverLoadPath, data, zkClient.getDefaultACL(), CreateMode.PERSISTENT);
-        } catch (KeeperException.NodeExistsException nee) {
-            logger.debug("the server load path {} is already created by others", serverLoadPath, nee);
-        }
-    }
-
-    @Override
-    public void saveOwnership(TreeSet<ServerLoad> serverLoads) throws StateManagerSaveException {
-        logger.info("saving ownership");
-        try {
-            ZooKeeper zk = zkClient.get();
-            // use timestamp as data so watchers will see any changes
-            byte[] timestamp = ByteBuffer.allocate(8).putLong(System.currentTimeMillis()).array();
-
-            if (zk.exists(serverLoadPath, false) == null) { //create path to rootnode if it does not yet exist
-                createServerLoadPathIfNoExists(timestamp);
-            }
-
-            Transaction tx = zk.transaction();
-            List<String> children = zk.getChildren(serverLoadPath, false);
-            HashSet<String> servers = new HashSet<String>(children);
-            tx.setData(serverLoadPath, timestamp, -1); // trigger the watcher that data has been updated
-            for (ServerLoad serverLoad : serverLoads) {
-                String server = serverToZkFormat(serverLoad.getServer());
-                String serverPath = serverPath(server);
-                if (servers.contains(server)) {
-                    servers.remove(server);
-                    tx.setData(serverPath, serverLoad.serialize(), -1);
-                } else {
-                    tx.create(serverPath, serverLoad.serialize(), zkClient.getDefaultACL(), CreateMode.PERSISTENT);
-                }
-            }
-            for (String server : servers) {
-                tx.delete(serverPath(server), -1);
-            }
-            tx.commit();
-        } catch (InterruptedException | IOException | KeeperException e) {
-            throw new StateManagerSaveException(e);
-        }
-    }
-
-    @Override
-    public TreeSet<ServerLoad> loadOwnership() throws StateManagerLoadException {
-        TreeSet<ServerLoad> ownerships = new TreeSet<ServerLoad>();
-        try {
-            ZooKeeper zk = zkClient.get();
-            List<String> children = zk.getChildren(serverLoadPath, false);
-            for (String server : children) {
-                ownerships.add(ServerLoad.deserialize(zk.getData(serverPath(server), false, new Stat())));
-            }
-            return ownerships;
-        } catch (InterruptedException | IOException | KeeperException e) {
-            throw new StateManagerLoadException(e);
-        }
-    }
-
-    @Override
-    public synchronized void watch(final PlacementCallback callback) {
-        if (watching) {
-            return; // do not double watch
-        }
-        watching = true;
-
-        try {
-            ZooKeeper zk = zkClient.get();
-            try {
-                zk.getData(serverLoadPath, new Watcher() {
-                    @Override
-                    public void process(WatchedEvent watchedEvent) {
-                        try {
-                            callback.callback(loadOwnership());
-                        } catch (StateManagerLoadException e) {
-                            logger.error("Watch of Ownership failed", e);
-                        } finally {
-                            watching = false;
-                            watch(callback);
-                        }
-                    }
-                }, new Stat());
-            } catch (KeeperException.NoNodeException nee) {
-                byte[] timestamp = ByteBuffer.allocate(8).putLong(System.currentTimeMillis()).array();
-                createServerLoadPathIfNoExists(timestamp);
-                watching = false;
-                watch(callback);
-            }
-        } catch (ZooKeeperClient.ZooKeeperConnectionException | InterruptedException | KeeperException e) {
-            logger.error("Watch of Ownership failed", e);
-            watching = false;
-            watch(callback);
-        }
-    }
-
-    public String serverPath(String server) {
-        return String.format("%s/%s", serverLoadPath, server);
-    }
-
-    protected String serverToZkFormat(String server) {
-        return server.replaceAll("/", "--");
-    }
-
-    protected String zkFormatToServer(String zkFormattedServer) {
-        return zkFormattedServer.replaceAll("--", "/");
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/package-info.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/package-info.java
deleted file mode 100644
index 72c134b..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/placement/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Placement Policy to place streams across proxy services.
- */
-package com.twitter.distributedlog.service.placement;
\ No newline at end of file
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/AbstractStreamOp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/AbstractStreamOp.java
deleted file mode 100644
index b513e24..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/AbstractStreamOp.java
+++ /dev/null
@@ -1,175 +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 com.twitter.distributedlog.service.stream;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.exceptions.ChecksumFailedException;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.distributedlog.service.ResponseUtils;
-import com.twitter.distributedlog.thrift.service.ResponseHeader;
-import com.twitter.distributedlog.util.Sequencer;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Promise;
-import com.twitter.util.Return;
-import com.twitter.util.Try;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.Option;
-
-/**
- * Abstract Stream Operation.
- */
-public abstract class AbstractStreamOp<Response> implements StreamOp {
-
-    private static final Logger logger = LoggerFactory.getLogger(AbstractStreamOp.class);
-
-    protected final String stream;
-    protected final OpStatsLogger opStatsLogger;
-    private final Promise<Response> result = new Promise<Response>();
-    protected final Stopwatch stopwatch = Stopwatch.createUnstarted();
-    protected final Long checksum;
-    protected final Feature checksumDisabledFeature;
-
-    public AbstractStreamOp(String stream,
-                            OpStatsLogger statsLogger,
-                            Long checksum,
-                            Feature checksumDisabledFeature) {
-        this.stream = stream;
-        this.opStatsLogger = statsLogger;
-        // start here in case the operation is failed before executing.
-        stopwatch.reset().start();
-        this.checksum = checksum;
-        this.checksumDisabledFeature = checksumDisabledFeature;
-    }
-
-    @Override
-    public String streamName() {
-        return stream;
-    }
-
-    @Override
-    public Stopwatch stopwatch() {
-        return stopwatch;
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!checksumDisabledFeature.isAvailable() && null != checksum) {
-            Long serverChecksum = computeChecksum();
-            if (null != serverChecksum && !checksum.equals(serverChecksum)) {
-                throw new ChecksumFailedException();
-            }
-        }
-    }
-
-    @Override
-    public Long computeChecksum() {
-        return null;
-    }
-
-    @Override
-    public Future<Void> execute(AsyncLogWriter writer, Sequencer sequencer, Object txnLock) {
-        stopwatch.reset().start();
-        return executeOp(writer, sequencer, txnLock)
-                .addEventListener(new FutureEventListener<Response>() {
-            @Override
-            public void onSuccess(Response response) {
-                opStatsLogger.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-                setResponse(response);
-            }
-            @Override
-            public void onFailure(Throwable cause) {
-            }
-        }).voided();
-    }
-
-    /**
-     * Fail with current <i>owner</i> and its reason <i>t</i>.
-     *
-     * @param cause
-     *          failure reason
-     */
-    @Override
-    public void fail(Throwable cause) {
-        if (cause instanceof OwnershipAcquireFailedException) {
-            // Ownership exception is a control exception, not an error, so we don't stat
-            // it with the other errors.
-            OwnershipAcquireFailedException oafe = (OwnershipAcquireFailedException) cause;
-            fail(ResponseUtils.ownerToHeader(oafe.getCurrentOwner()));
-        } else {
-            opStatsLogger.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-            fail(ResponseUtils.exceptionToHeader(cause));
-        }
-    }
-
-    protected void setResponse(Response response) {
-      Return<Response> responseTry = new Return(response);
-      boolean isEmpty = result.updateIfEmpty(responseTry);
-      if (!isEmpty) {
-        Option<Try<Response>> resultTry = result.poll();
-        logger.error("Result set multiple times. Value='{}', New='{}'", resultTry, responseTry);
-      }
-    }
-
-    /**
-     * Return the full response, header and body.
-     *
-     * @return A future containing the response or the exception
-     *      encountered by the op if it failed.
-     */
-    public Future<Response> result() {
-        return result;
-    }
-
-    /**
-     * Execute the operation and return its corresponding response.
-     *
-     * @param writer
-     *          writer to execute the operation.
-     * @param sequencer
-     *          sequencer used for generating transaction id for stream operations
-     * @param txnLock
-     *          transaction lock to guarantee ordering of transaction id
-     * @return future representing the operation.
-     */
-    protected abstract Future<Response> executeOp(AsyncLogWriter writer,
-                                                  Sequencer sequencer,
-                                                  Object txnLock);
-
-    // fail the result with the given response header
-    protected abstract void fail(ResponseHeader header);
-
-    public static OpStatsLogger requestStat(StatsLogger statsLogger, String opName) {
-        return requestLogger(statsLogger).getOpStatsLogger(opName);
-    }
-
-    public static StatsLogger requestLogger(StatsLogger statsLogger) {
-        return statsLogger.scope("request");
-    }
-
-    public static StatsLogger requestScope(StatsLogger statsLogger, String scope) {
-        return requestLogger(statsLogger).scope(scope);
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/AbstractWriteOp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/AbstractWriteOp.java
deleted file mode 100644
index a385b84..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/AbstractWriteOp.java
+++ /dev/null
@@ -1,60 +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 com.twitter.distributedlog.service.stream;
-
-import com.twitter.distributedlog.service.ResponseUtils;
-import com.twitter.distributedlog.thrift.service.ResponseHeader;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.util.ProtocolUtils;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import scala.runtime.AbstractFunction1;
-
-/**
- * Abstract Write Operation.
- */
-public abstract class AbstractWriteOp extends AbstractStreamOp<WriteResponse> {
-
-    protected AbstractWriteOp(String stream,
-                              OpStatsLogger statsLogger,
-                              Long checksum,
-                              Feature checksumDisabledFeature) {
-        super(stream, statsLogger, checksum, checksumDisabledFeature);
-    }
-
-    @Override
-    protected void fail(ResponseHeader header) {
-        setResponse(ResponseUtils.write(header));
-    }
-
-    @Override
-    public Long computeChecksum() {
-        return ProtocolUtils.streamOpCRC32(stream);
-    }
-
-    @Override
-    public Future<ResponseHeader> responseHeader() {
-        return result().map(new AbstractFunction1<WriteResponse, ResponseHeader>() {
-            @Override
-            public ResponseHeader apply(WriteResponse response) {
-                return response.getHeader();
-            }
-        });
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/BulkWriteOp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/BulkWriteOp.java
deleted file mode 100644
index 4d50b66..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/BulkWriteOp.java
+++ /dev/null
@@ -1,253 +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 com.twitter.distributedlog.service.stream;
-
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.exceptions.AlreadyClosedException;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.LockingException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.distributedlog.exceptions.RequestDeniedException;
-import com.twitter.distributedlog.service.ResponseUtils;
-import com.twitter.distributedlog.service.streamset.Partition;
-import com.twitter.distributedlog.service.streamset.StreamPartitionConverter;
-import com.twitter.distributedlog.thrift.service.BulkWriteResponse;
-import com.twitter.distributedlog.thrift.service.ResponseHeader;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.util.Sequencer;
-import com.twitter.util.ConstFuture;
-import com.twitter.util.Future;
-import com.twitter.util.Future$;
-import com.twitter.util.FutureEventListener;
-import com.twitter.util.Try;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import scala.runtime.AbstractFunction1;
-
-/**
- * Bulk Write Operation.
- */
-public class BulkWriteOp extends AbstractStreamOp<BulkWriteResponse> implements WriteOpWithPayload {
-    private final List<ByteBuffer> buffers;
-    private final long payloadSize;
-
-    // Stats
-    private final Counter deniedBulkWriteCounter;
-    private final Counter successRecordCounter;
-    private final Counter failureRecordCounter;
-    private final Counter redirectRecordCounter;
-    private final OpStatsLogger latencyStat;
-    private final Counter bytes;
-    private final Counter bulkWriteBytes;
-
-    private final AccessControlManager accessControlManager;
-
-    // We need to pass these through to preserve ownership change behavior in
-    // client/server. Only include failures which are guaranteed to have failed
-    // all subsequent writes.
-    private boolean isDefiniteFailure(Try<DLSN> result) {
-        boolean def = false;
-        try {
-            result.get();
-        } catch (Exception ex) {
-            if (ex instanceof OwnershipAcquireFailedException
-                || ex instanceof AlreadyClosedException
-                || ex instanceof LockingException) {
-                def = true;
-            }
-        }
-        return def;
-    }
-
-    public BulkWriteOp(String stream,
-                       List<ByteBuffer> buffers,
-                       StatsLogger statsLogger,
-                       StatsLogger perStreamStatsLogger,
-                       StreamPartitionConverter streamPartitionConverter,
-                       Long checksum,
-                       Feature checksumDisabledFeature,
-                       AccessControlManager accessControlManager) {
-        super(stream, requestStat(statsLogger, "bulkWrite"), checksum, checksumDisabledFeature);
-        this.buffers = buffers;
-        long total = 0;
-        // We do this here because the bytebuffers are mutable.
-        for (ByteBuffer bb : buffers) {
-          total += bb.remaining();
-        }
-        this.payloadSize = total;
-
-        final Partition partition = streamPartitionConverter.convert(stream);
-        // Write record stats
-        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-        this.deniedBulkWriteCounter = streamOpStats.requestDeniedCounter("bulkWrite");
-        this.successRecordCounter = streamOpStats.recordsCounter("success");
-        this.failureRecordCounter = streamOpStats.recordsCounter("failure");
-        this.redirectRecordCounter = streamOpStats.recordsCounter("redirect");
-        this.bulkWriteBytes = streamOpStats.scopedRequestCounter("bulkWrite", "bytes");
-        this.latencyStat = streamOpStats.streamRequestLatencyStat(partition, "bulkWrite");
-        this.bytes = streamOpStats.streamRequestCounter(partition, "bulkWrite", "bytes");
-
-        this.accessControlManager = accessControlManager;
-
-        final long size = getPayloadSize();
-        result().addEventListener(new FutureEventListener<BulkWriteResponse>() {
-            @Override
-            public void onSuccess(BulkWriteResponse response) {
-                if (response.getHeader().getCode() == StatusCode.SUCCESS) {
-                    latencyStat.registerSuccessfulEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
-                    bytes.add(size);
-                    bulkWriteBytes.add(size);
-                } else {
-                    latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
-                }
-            }
-            @Override
-            public void onFailure(Throwable cause) {
-                latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
-            }
-        });
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!accessControlManager.allowWrite(stream)) {
-            deniedBulkWriteCounter.inc();
-            throw new RequestDeniedException(stream, "bulkWrite");
-        }
-        super.preExecute();
-    }
-
-    @Override
-    public long getPayloadSize() {
-      return payloadSize;
-    }
-
-    @Override
-    protected Future<BulkWriteResponse> executeOp(AsyncLogWriter writer,
-                                                  Sequencer sequencer,
-                                                  Object txnLock) {
-        // Need to convert input buffers to LogRecords.
-        List<LogRecord> records;
-        Future<List<Future<DLSN>>> futureList;
-        synchronized (txnLock) {
-            records = asRecordList(buffers, sequencer);
-            futureList = writer.writeBulk(records);
-        }
-
-        // Collect into a list of tries to make it easier to extract exception or DLSN.
-        Future<List<Try<DLSN>>> writes = asTryList(futureList);
-
-        Future<BulkWriteResponse> response = writes.flatMap(
-            new AbstractFunction1<List<Try<DLSN>>, Future<BulkWriteResponse>>() {
-                @Override
-                public Future<BulkWriteResponse> apply(List<Try<DLSN>> results) {
-
-                    // Considered a success at batch level even if no individual writes succeeed.
-                    // The reason is that its impossible to make an appropriate decision re retries without
-                    // individual buffer failure reasons.
-                    List<WriteResponse> writeResponses = new ArrayList<WriteResponse>(results.size());
-                    BulkWriteResponse bulkWriteResponse =
-                        ResponseUtils.bulkWriteSuccess().setWriteResponses(writeResponses);
-
-                    // Promote the first result to an op-level failure if we're sure all other writes have
-                    // failed.
-                    if (results.size() > 0) {
-                        Try<DLSN> firstResult = results.get(0);
-                        if (isDefiniteFailure(firstResult)) {
-                            return new ConstFuture(firstResult);
-                        }
-                    }
-
-                    // Translate all futures to write responses.
-                    Iterator<Try<DLSN>> iterator = results.iterator();
-                    while (iterator.hasNext()) {
-                        Try<DLSN> completedFuture = iterator.next();
-                        try {
-                            DLSN dlsn = completedFuture.get();
-                            WriteResponse writeResponse = ResponseUtils.writeSuccess().setDlsn(dlsn.serialize());
-                            writeResponses.add(writeResponse);
-                            successRecordCounter.inc();
-                        } catch (Exception ioe) {
-                            WriteResponse writeResponse = ResponseUtils.write(ResponseUtils.exceptionToHeader(ioe));
-                            writeResponses.add(writeResponse);
-                            if (StatusCode.FOUND == writeResponse.getHeader().getCode()) {
-                                redirectRecordCounter.inc();
-                            } else {
-                                failureRecordCounter.inc();
-                            }
-                        }
-                    }
-
-                    return Future.value(bulkWriteResponse);
-                }
-            }
-        );
-
-        return response;
-    }
-
-    private List<LogRecord> asRecordList(List<ByteBuffer> buffers, Sequencer sequencer) {
-        List<LogRecord> records = new ArrayList<LogRecord>(buffers.size());
-        for (ByteBuffer buffer : buffers) {
-            byte[] payload = new byte[buffer.remaining()];
-            buffer.get(payload);
-            records.add(new LogRecord(sequencer.nextId(), payload));
-        }
-        return records;
-    }
-
-    private Future<List<Try<DLSN>>> asTryList(Future<List<Future<DLSN>>> futureList) {
-        return futureList.flatMap(new AbstractFunction1<List<Future<DLSN>>, Future<List<Try<DLSN>>>>() {
-            @Override
-            public Future<List<Try<DLSN>>> apply(List<Future<DLSN>> results) {
-                return Future$.MODULE$.collectToTry(results);
-            }
-        });
-    }
-
-    @Override
-    protected void fail(ResponseHeader header) {
-        if (StatusCode.FOUND == header.getCode()) {
-            redirectRecordCounter.add(buffers.size());
-        } else {
-            failureRecordCounter.add(buffers.size());
-        }
-        setResponse(ResponseUtils.bulkWrite(header));
-    }
-
-    @Override
-    public Future<ResponseHeader> responseHeader() {
-        return result().map(new AbstractFunction1<BulkWriteResponse, ResponseHeader>() {
-            @Override
-            public ResponseHeader apply(BulkWriteResponse response) {
-                return response.getHeader();
-            }
-        });
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/DeleteOp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/DeleteOp.java
deleted file mode 100644
index e30a989..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/DeleteOp.java
+++ /dev/null
@@ -1,76 +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 com.twitter.distributedlog.service.stream;
-
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.RequestDeniedException;
-import com.twitter.distributedlog.service.ResponseUtils;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.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 scala.runtime.AbstractFunction1;
-
-/**
- * Operation to delete a log stream.
- */
-public class DeleteOp extends AbstractWriteOp {
-    private final StreamManager streamManager;
-    private final Counter deniedDeleteCounter;
-    private final AccessControlManager accessControlManager;
-
-    public DeleteOp(String stream,
-                    StatsLogger statsLogger,
-                    StatsLogger perStreamStatsLogger,
-                    StreamManager streamManager,
-                    Long checksum,
-                    Feature checksumEnabledFeature,
-                    AccessControlManager accessControlManager) {
-        super(stream, requestStat(statsLogger, "delete"), checksum, checksumEnabledFeature);
-        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-        this.deniedDeleteCounter = streamOpStats.requestDeniedCounter("delete");
-        this.accessControlManager = accessControlManager;
-        this.streamManager = streamManager;
-    }
-
-    @Override
-    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
-                                              Sequencer sequencer,
-                                              Object txnLock) {
-        Future<Void> result = streamManager.deleteAndRemoveAsync(streamName());
-        return result.map(new AbstractFunction1<Void, WriteResponse>() {
-            @Override
-            public WriteResponse apply(Void value) {
-                return ResponseUtils.writeSuccess();
-            }
-        });
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!accessControlManager.allowTruncate(stream)) {
-            deniedDeleteCounter.inc();
-            throw new RequestDeniedException(stream, "delete");
-        }
-        super.preExecute();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/HeartbeatOp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/HeartbeatOp.java
deleted file mode 100644
index f34295b..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/HeartbeatOp.java
+++ /dev/null
@@ -1,101 +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 com.twitter.distributedlog.service.stream;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.BKAsyncLogWriter;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.RequestDeniedException;
-import com.twitter.distributedlog.service.ResponseUtils;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.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 scala.runtime.AbstractFunction1;
-
-/**
- * Heartbeat Operation.
- */
-public class HeartbeatOp extends AbstractWriteOp {
-
-    static final byte[] HEARTBEAT_DATA = "heartbeat".getBytes(UTF_8);
-
-    private final AccessControlManager accessControlManager;
-    private final Counter deniedHeartbeatCounter;
-    private final byte dlsnVersion;
-
-    private boolean writeControlRecord = false;
-
-    public HeartbeatOp(String stream,
-                       StatsLogger statsLogger,
-                       StatsLogger perStreamStatsLogger,
-                       byte dlsnVersion,
-                       Long checksum,
-                       Feature checksumDisabledFeature,
-                       AccessControlManager accessControlManager) {
-        super(stream, requestStat(statsLogger, "heartbeat"), checksum, checksumDisabledFeature);
-        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-        this.deniedHeartbeatCounter = streamOpStats.requestDeniedCounter("heartbeat");
-        this.dlsnVersion = dlsnVersion;
-        this.accessControlManager = accessControlManager;
-    }
-
-    public HeartbeatOp setWriteControlRecord(boolean writeControlRecord) {
-        this.writeControlRecord = writeControlRecord;
-        return this;
-    }
-
-    @Override
-    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
-                                              Sequencer sequencer,
-                                              Object txnLock) {
-        // write a control record if heartbeat is the first request of the recovered log segment.
-        if (writeControlRecord) {
-            long txnId;
-            Future<DLSN> writeResult;
-            synchronized (txnLock) {
-                txnId = sequencer.nextId();
-                writeResult = ((BKAsyncLogWriter) writer).writeControlRecord(new LogRecord(txnId, HEARTBEAT_DATA));
-            }
-            return writeResult.map(new AbstractFunction1<DLSN, WriteResponse>() {
-                @Override
-                public WriteResponse apply(DLSN value) {
-                    return ResponseUtils.writeSuccess().setDlsn(value.serialize(dlsnVersion));
-                }
-            });
-        } else {
-            return Future.value(ResponseUtils.writeSuccess());
-        }
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!accessControlManager.allowAcquire(stream)) {
-            deniedHeartbeatCounter.inc();
-            throw new RequestDeniedException(stream, "heartbeat");
-        }
-        super.preExecute();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/ReleaseOp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/ReleaseOp.java
deleted file mode 100644
index aa0f1a7..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/ReleaseOp.java
+++ /dev/null
@@ -1,76 +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 com.twitter.distributedlog.service.stream;
-
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.RequestDeniedException;
-import com.twitter.distributedlog.service.ResponseUtils;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.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 scala.runtime.AbstractFunction1;
-
-/**
- * Operation to release ownership of a log stream.
- */
-public class ReleaseOp extends AbstractWriteOp {
-    private final StreamManager streamManager;
-    private final Counter deniedReleaseCounter;
-    private final AccessControlManager accessControlManager;
-
-    public ReleaseOp(String stream,
-                     StatsLogger statsLogger,
-                     StatsLogger perStreamStatsLogger,
-                     StreamManager streamManager,
-                     Long checksum,
-                     Feature checksumDisabledFeature,
-                     AccessControlManager accessControlManager) {
-        super(stream, requestStat(statsLogger, "release"), checksum, checksumDisabledFeature);
-        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-        this.deniedReleaseCounter = streamOpStats.requestDeniedCounter("release");
-        this.accessControlManager = accessControlManager;
-        this.streamManager = streamManager;
-    }
-
-    @Override
-    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
-                                              Sequencer sequencer,
-                                              Object txnLock) {
-        Future<Void> result = streamManager.closeAndRemoveAsync(streamName());
-        return result.map(new AbstractFunction1<Void, WriteResponse>() {
-            @Override
-            public WriteResponse apply(Void value) {
-                return ResponseUtils.writeSuccess();
-            }
-        });
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!accessControlManager.allowRelease(stream)) {
-            deniedReleaseCounter.inc();
-            throw new RequestDeniedException(stream, "release");
-        }
-        super.preExecute();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/Stream.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/Stream.java
deleted file mode 100644
index e015e29..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/Stream.java
+++ /dev/null
@@ -1,93 +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 com.twitter.distributedlog.service.stream;
-
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.service.streamset.Partition;
-import com.twitter.util.Future;
-import java.io.IOException;
-
-/**
- * Stream is the per stream request handler in the DL service layer.
- *
- * <p>The collection of Streams in the proxy are managed by StreamManager.
- */
-public interface Stream {
-
-    /**
-     * Get the stream configuration for this stream.
-     *
-     * @return stream configuration
-     */
-    DynamicDistributedLogConfiguration getStreamConfiguration();
-
-    /**
-     * Get the stream's last recorded current owner (may be out of date). Used
-     * as a hint for the client.
-     * @return last known owner for the stream
-     */
-    String getOwner();
-
-    /**
-     * Get the stream name.
-     * @return stream name
-     */
-    String getStreamName();
-
-    /**
-     * Get the represented partition name.
-     *
-     * @return represented partition name.
-     */
-    Partition getPartition();
-
-    /**
-     * Expensive initialization code run after stream has been allocated in
-     * StreamManager.
-     *
-     * @throws IOException when encountered exception on initialization
-     */
-    void initialize() throws IOException;
-
-    /**
-     * Another initialize method (actually Thread.start). Should probably be
-     * moved to initialize().
-     */
-    void start();
-
-    /**
-     * Asynchronous close method.
-     * @param reason for closing
-     * @return future satisfied once close complete
-     */
-    Future<Void> requestClose(String reason);
-
-    /**
-     * Delete the stream from DL backend.
-     *
-     * @throws IOException when encountered exception on deleting the stream.
-     */
-    void delete() throws IOException;
-
-    /**
-     * Execute the stream operation against this stream.
-     *
-     * @param op operation to execute
-     */
-    void submit(StreamOp op);
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamFactory.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamFactory.java
deleted file mode 100644
index 0dfbd69..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamFactory.java
+++ /dev/null
@@ -1,38 +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 com.twitter.distributedlog.service.stream;
-
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-
-/**
- * Factory to create a stream with provided stream configuration {@code streamConf}.
- */
-public interface StreamFactory {
-
-    /**
-     * Create a stream object.
-     *
-     * @param name stream name
-     * @param streamConf stream configuration
-     * @param streamManager manager of streams
-     * @return stream object
-     */
-    Stream create(String name,
-                  DynamicDistributedLogConfiguration streamConf,
-                  StreamManager streamManager);
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamFactoryImpl.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamFactoryImpl.java
deleted file mode 100644
index 566ded6..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamFactoryImpl.java
+++ /dev/null
@@ -1,95 +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 com.twitter.distributedlog.service.stream;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.service.FatalErrorHandler;
-import com.twitter.distributedlog.service.config.ServerConfiguration;
-import com.twitter.distributedlog.service.config.StreamConfigProvider;
-import com.twitter.distributedlog.service.streamset.StreamPartitionConverter;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.util.Timer;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.jboss.netty.util.HashedWheelTimer;
-
-/**
- * The implementation of {@link StreamFactory}.
- */
-public class StreamFactoryImpl implements StreamFactory {
-    private final String clientId;
-    private final StreamOpStats streamOpStats;
-    private final ServerConfiguration serverConfig;
-    private final DistributedLogConfiguration dlConfig;
-    private final FeatureProvider featureProvider;
-    private final StreamConfigProvider streamConfigProvider;
-    private final StreamPartitionConverter streamPartitionConverter;
-    private final DistributedLogNamespace dlNamespace;
-    private final OrderedScheduler scheduler;
-    private final FatalErrorHandler fatalErrorHandler;
-    private final HashedWheelTimer requestTimer;
-    private final Timer futureTimer;
-
-    public StreamFactoryImpl(String clientId,
-        StreamOpStats streamOpStats,
-        ServerConfiguration serverConfig,
-        DistributedLogConfiguration dlConfig,
-        FeatureProvider featureProvider,
-        StreamConfigProvider streamConfigProvider,
-        StreamPartitionConverter streamPartitionConverter,
-        DistributedLogNamespace dlNamespace,
-        OrderedScheduler scheduler,
-        FatalErrorHandler fatalErrorHandler,
-        HashedWheelTimer requestTimer) {
-
-        this.clientId = clientId;
-        this.streamOpStats = streamOpStats;
-        this.serverConfig = serverConfig;
-        this.dlConfig = dlConfig;
-        this.featureProvider = featureProvider;
-        this.streamConfigProvider = streamConfigProvider;
-        this.streamPartitionConverter = streamPartitionConverter;
-        this.dlNamespace = dlNamespace;
-        this.scheduler = scheduler;
-        this.fatalErrorHandler = fatalErrorHandler;
-        this.requestTimer = requestTimer;
-        this.futureTimer = new com.twitter.finagle.util.HashedWheelTimer(requestTimer);
-    }
-
-    @Override
-    public Stream create(String name,
-                         DynamicDistributedLogConfiguration streamConf,
-                         StreamManager streamManager) {
-        return new StreamImpl(name,
-            streamPartitionConverter.convert(name),
-            clientId,
-            streamManager,
-            streamOpStats,
-            serverConfig,
-            dlConfig,
-            streamConf,
-            featureProvider,
-            streamConfigProvider,
-            dlNamespace,
-            scheduler,
-            fatalErrorHandler,
-            requestTimer,
-            futureTimer);
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamImpl.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamImpl.java
deleted file mode 100644
index 55c1b48..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamImpl.java
+++ /dev/null
@@ -1,925 +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 com.twitter.distributedlog.service.stream;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.AlreadyClosedException;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.distributedlog.exceptions.StreamNotReadyException;
-import com.twitter.distributedlog.exceptions.StreamUnavailableException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.io.Abortables;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.service.FatalErrorHandler;
-import com.twitter.distributedlog.service.ServerFeatureKeys;
-import com.twitter.distributedlog.service.config.ServerConfiguration;
-import com.twitter.distributedlog.service.config.StreamConfigProvider;
-import com.twitter.distributedlog.service.stream.limiter.StreamRequestLimiter;
-import com.twitter.distributedlog.service.streamset.Partition;
-import com.twitter.distributedlog.stats.BroadCastStatsLogger;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.OrderedScheduler;
-import com.twitter.distributedlog.util.TimeSequencer;
-import com.twitter.distributedlog.util.Utils;
-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.TimeoutException;
-import com.twitter.util.Timer;
-import java.io.IOException;
-import java.util.ArrayDeque;
-import java.util.Queue;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.Gauge;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.jboss.netty.util.HashedWheelTimer;
-import org.jboss.netty.util.Timeout;
-import org.jboss.netty.util.TimerTask;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-import scala.runtime.AbstractFunction1;
-import scala.runtime.BoxedUnit;
-
-/**
- * Implementation of {@link Stream}.
- */
-public class StreamImpl implements Stream {
-
-    private static final Logger logger = LoggerFactory.getLogger(StreamImpl.class);
-
-    /**
-     * The status of the stream.
-     *
-     * <p>The status change of the stream should just go in one direction. If a stream hits
-     * any error, the stream should be put in error state. If a stream is in error state,
-     * it should be removed and not reused anymore.
-     */
-    public enum StreamStatus {
-        UNINITIALIZED(-1),
-        INITIALIZING(0),
-        INITIALIZED(1),
-        CLOSING(-4),
-        CLOSED(-5),
-        // if a stream is in error state, it should be abort during closing.
-        ERROR(-6);
-
-        final int code;
-
-        StreamStatus(int code) {
-            this.code = code;
-        }
-
-        int getCode() {
-            return code;
-        }
-
-        public static boolean isUnavailable(StreamStatus status) {
-            return StreamStatus.ERROR == status || StreamStatus.CLOSING == status || StreamStatus.CLOSED == status;
-        }
-    }
-
-    private final String name;
-    private final Partition partition;
-    private DistributedLogManager manager;
-
-    private volatile AsyncLogWriter writer;
-    private volatile StreamStatus status;
-    private volatile String owner;
-    private volatile Throwable lastException;
-    private volatile Queue<StreamOp> pendingOps = new ArrayDeque<StreamOp>();
-
-    private final Promise<Void> closePromise = new Promise<Void>();
-    private final Object txnLock = new Object();
-    private final TimeSequencer sequencer = new TimeSequencer();
-    private final StreamRequestLimiter limiter;
-    private final DynamicDistributedLogConfiguration dynConf;
-    private final DistributedLogConfiguration dlConfig;
-    private final DistributedLogNamespace dlNamespace;
-    private final String clientId;
-    private final OrderedScheduler scheduler;
-    private final ReentrantReadWriteLock closeLock = new ReentrantReadWriteLock();
-    private final Feature featureRateLimitDisabled;
-    private final StreamManager streamManager;
-    private final StreamConfigProvider streamConfigProvider;
-    private final FatalErrorHandler fatalErrorHandler;
-    private final long streamProbationTimeoutMs;
-    private final long serviceTimeoutMs;
-    private final long writerCloseTimeoutMs;
-    private final boolean failFastOnStreamNotReady;
-    private final HashedWheelTimer requestTimer;
-    private final Timer futureTimer;
-
-    // Stats
-    private final StatsLogger streamLogger;
-    private final StatsLogger streamExceptionStatLogger;
-    private final StatsLogger limiterStatLogger;
-    private final Counter serviceTimeout;
-    private final OpStatsLogger streamAcquireStat;
-    private final OpStatsLogger writerCloseStatLogger;
-    private final Counter pendingOpsCounter;
-    private final Counter unexpectedExceptions;
-    private final Counter writerCloseTimeoutCounter;
-    private final StatsLogger exceptionStatLogger;
-    private final ConcurrentHashMap<String, Counter> exceptionCounters =
-        new ConcurrentHashMap<String, Counter>();
-    private final Gauge<Number> streamStatusGauge;
-
-    // Since we may create and discard streams at initialization if there's a race,
-    // must not do any expensive initialization here (particularly any locking or
-    // significant resource allocation etc.).
-    StreamImpl(final String name,
-               final Partition partition,
-               String clientId,
-               StreamManager streamManager,
-               StreamOpStats streamOpStats,
-               ServerConfiguration serverConfig,
-               DistributedLogConfiguration dlConfig,
-               DynamicDistributedLogConfiguration streamConf,
-               FeatureProvider featureProvider,
-               StreamConfigProvider streamConfigProvider,
-               DistributedLogNamespace dlNamespace,
-               OrderedScheduler scheduler,
-               FatalErrorHandler fatalErrorHandler,
-               HashedWheelTimer requestTimer,
-               Timer futureTimer) {
-        this.clientId = clientId;
-        this.dlConfig = dlConfig;
-        this.streamManager = streamManager;
-        this.name = name;
-        this.partition = partition;
-        this.status = StreamStatus.UNINITIALIZED;
-        this.lastException = new IOException("Fail to write record to stream " + name);
-        this.streamConfigProvider = streamConfigProvider;
-        this.dlNamespace = dlNamespace;
-        this.featureRateLimitDisabled = featureProvider.getFeature(
-            ServerFeatureKeys.SERVICE_RATE_LIMIT_DISABLED.name().toLowerCase());
-        this.scheduler = scheduler;
-        this.serviceTimeoutMs = serverConfig.getServiceTimeoutMs();
-        this.streamProbationTimeoutMs = serverConfig.getStreamProbationTimeoutMs();
-        this.writerCloseTimeoutMs = serverConfig.getWriterCloseTimeoutMs();
-        this.failFastOnStreamNotReady = dlConfig.getFailFastOnStreamNotReady();
-        this.fatalErrorHandler = fatalErrorHandler;
-        this.dynConf = streamConf;
-        StatsLogger limiterStatsLogger = BroadCastStatsLogger.two(
-            streamOpStats.baseScope("stream_limiter"),
-            streamOpStats.streamRequestScope(partition, "limiter"));
-        this.limiter = new StreamRequestLimiter(name, dynConf, limiterStatsLogger, featureRateLimitDisabled);
-        this.requestTimer = requestTimer;
-        this.futureTimer = futureTimer;
-
-        // Stats
-        this.streamLogger = streamOpStats.streamRequestStatsLogger(partition);
-        this.limiterStatLogger = streamOpStats.baseScope("request_limiter");
-        this.streamExceptionStatLogger = streamLogger.scope("exceptions");
-        this.serviceTimeout = streamOpStats.baseCounter("serviceTimeout");
-        StatsLogger streamsStatsLogger = streamOpStats.baseScope("streams");
-        this.streamAcquireStat = streamsStatsLogger.getOpStatsLogger("acquire");
-        this.pendingOpsCounter = streamOpStats.baseCounter("pending_ops");
-        this.unexpectedExceptions = streamOpStats.baseCounter("unexpected_exceptions");
-        this.exceptionStatLogger = streamOpStats.requestScope("exceptions");
-        this.writerCloseStatLogger = streamsStatsLogger.getOpStatsLogger("writer_close");
-        this.writerCloseTimeoutCounter = streamsStatsLogger.getCounter("writer_close_timeouts");
-        // Gauges
-        this.streamStatusGauge = new Gauge<Number>() {
-            @Override
-            public Number getDefaultValue() {
-                return StreamStatus.UNINITIALIZED.getCode();
-            }
-            @Override
-            public Number getSample() {
-                return status.getCode();
-            }
-        };
-    }
-
-    @Override
-    public String getOwner() {
-        return owner;
-    }
-
-    @Override
-    public String getStreamName() {
-        return name;
-    }
-
-    @Override
-    public DynamicDistributedLogConfiguration getStreamConfiguration() {
-        return dynConf;
-    }
-
-    @Override
-    public Partition getPartition() {
-        return partition;
-    }
-
-    private DistributedLogManager openLog(String name) throws IOException {
-        Optional<DistributedLogConfiguration> dlConf = Optional.<DistributedLogConfiguration>absent();
-        Optional<DynamicDistributedLogConfiguration> dynDlConf = Optional.of(dynConf);
-        Optional<StatsLogger> perStreamStatsLogger = Optional.of(streamLogger);
-        return dlNamespace.openLog(name, dlConf, dynDlConf, perStreamStatsLogger);
-    }
-
-    // Expensive initialization, only called once per stream.
-    @Override
-    public void initialize() throws IOException {
-        manager = openLog(name);
-
-        // Better to avoid registering the gauge multiple times, so do this in init
-        // which only gets called once.
-        streamLogger.registerGauge("stream_status", this.streamStatusGauge);
-
-        // Signal initialization is complete, should be last in this method.
-        status = StreamStatus.INITIALIZING;
-    }
-
-    @Override
-    public String toString() {
-        return String.format("Stream:%s, %s, %s Status:%s", name, manager, writer, status);
-    }
-
-    @Override
-    public void start() {
-        // acquire the stream
-        acquireStream().addEventListener(new FutureEventListener<Boolean>() {
-                @Override
-                public void onSuccess(Boolean success) {
-                    if (!success) {
-                        // failed to acquire the stream. set the stream in error status and close it.
-                        setStreamInErrorStatus();
-                        requestClose("Failed to acquire the ownership");
-                    }
-                }
-
-                @Override
-                public void onFailure(Throwable cause) {
-                    // unhandled exceptions
-                    logger.error("Stream {} threw unhandled exception : ", name, cause);
-                    // failed to acquire the stream. set the stream in error status and close it.
-                    setStreamInErrorStatus();
-                    requestClose("Unhandled exception");
-                }
-            });
-    }
-
-    //
-    // Stats Operations
-    //
-
-    void countException(Throwable t, StatsLogger streamExceptionLogger) {
-        String exceptionName = null == t ? "null" : t.getClass().getName();
-        Counter counter = exceptionCounters.get(exceptionName);
-        if (null == counter) {
-            counter = exceptionStatLogger.getCounter(exceptionName);
-            Counter oldCounter = exceptionCounters.putIfAbsent(exceptionName, counter);
-            if (null != oldCounter) {
-                counter = oldCounter;
-            }
-        }
-        counter.inc();
-        streamExceptionLogger.getCounter(exceptionName).inc();
-    }
-
-    boolean isCriticalException(Throwable cause) {
-        return !(cause instanceof OwnershipAcquireFailedException);
-    }
-
-    //
-    // Service Timeout:
-    // - schedule a timeout function to handle operation timeouts: {@link #handleServiceTimeout(String)}
-    // - if the operation is completed within timeout period, cancel the timeout.
-    //
-
-    void scheduleTimeout(final StreamOp op) {
-        final Timeout timeout = requestTimer.newTimeout(new TimerTask() {
-            @Override
-            public void run(Timeout timeout) throws Exception {
-                if (!timeout.isCancelled()) {
-                    serviceTimeout.inc();
-                    handleServiceTimeout("Operation " + op.getClass().getName() + " timeout");
-                }
-            }
-        }, serviceTimeoutMs, TimeUnit.MILLISECONDS);
-        op.responseHeader().ensure(new Function0<BoxedUnit>() {
-            @Override
-            public BoxedUnit apply() {
-                timeout.cancel();
-                return null;
-            }
-        });
-    }
-
-    /**
-     * Close the stream and schedule cache eviction at some point in the future.
-     * We delay this as a way to place the stream in a probationary state--cached
-     * in the proxy but unusable.
-     * This mechanism helps the cluster adapt to situations where a proxy has
-     * persistent connectivity/availability issues, because it keeps an affected
-     * stream off the proxy for a period of time, hopefully long enough for the
-     * issues to be resolved, or for whoop to kick in and kill the shard.
-     */
-    void handleServiceTimeout(String reason) {
-        synchronized (this) {
-            if (StreamStatus.isUnavailable(status)) {
-                return;
-            }
-            // Mark stream in error state
-            setStreamInErrorStatus();
-        }
-
-        // Async close request, and schedule eviction when its done.
-        Future<Void> closeFuture = requestClose(reason, false /* dont remove */);
-        closeFuture.onSuccess(new AbstractFunction1<Void, BoxedUnit>() {
-            @Override
-            public BoxedUnit apply(Void result) {
-                streamManager.scheduleRemoval(StreamImpl.this, streamProbationTimeoutMs);
-                return BoxedUnit.UNIT;
-            }
-        });
-    }
-
-    //
-    // Submit the operation to the stream.
-    //
-
-    /**
-     * Execute the StreamOp. If reacquire is needed, this may initiate reacquire and queue the op for
-     * execution once complete.
-     *
-     * @param op
-     *          stream operation to execute.
-     */
-    @Override
-    public void submit(StreamOp op) {
-        try {
-            limiter.apply(op);
-        } catch (OverCapacityException ex) {
-            op.fail(ex);
-            return;
-        }
-
-        // Timeout stream op if requested.
-        if (serviceTimeoutMs > 0) {
-            scheduleTimeout(op);
-        }
-
-        boolean completeOpNow = false;
-        boolean success = true;
-        if (StreamStatus.isUnavailable(status)) {
-            // Stream is closed, fail the op immediately
-            op.fail(new StreamUnavailableException("Stream " + name + " is closed."));
-            return;
-        } else if (StreamStatus.INITIALIZED == status && writer != null) {
-            completeOpNow = true;
-            success = true;
-        } else {
-            synchronized (this) {
-                if (StreamStatus.isUnavailable(status)) {
-                    // Stream is closed, fail the op immediately
-                    op.fail(new StreamUnavailableException("Stream " + name + " is closed."));
-                    return;
-                } else if (StreamStatus.INITIALIZED == status) {
-                    completeOpNow = true;
-                    success = true;
-                } else if (failFastOnStreamNotReady) {
-                    op.fail(new StreamNotReadyException("Stream " + name + " is not ready; status = " + status));
-                    return;
-                } else { // the stream is still initializing
-                    pendingOps.add(op);
-                    pendingOpsCounter.inc();
-                    if (1 == pendingOps.size()) {
-                        if (op instanceof HeartbeatOp) {
-                            ((HeartbeatOp) op).setWriteControlRecord(true);
-                        }
-                    }
-                }
-            }
-        }
-        if (completeOpNow) {
-            executeOp(op, success);
-        }
-    }
-
-    //
-    // Execute operations and handle exceptions on operations
-    //
-
-    /**
-     * Execute the <i>op</i> immediately.
-     *
-     * @param op
-     *          stream operation to execute.
-     * @param success
-     *          whether the operation is success or not.
-     */
-    void executeOp(final StreamOp op, boolean success) {
-        final AsyncLogWriter writer;
-        final Throwable lastException;
-        synchronized (this) {
-            writer = this.writer;
-            lastException = this.lastException;
-        }
-        if (null != writer && success) {
-            op.execute(writer, sequencer, txnLock)
-                    .addEventListener(new FutureEventListener<Void>() {
-                @Override
-                public void onSuccess(Void value) {
-                    // nop
-                }
-                @Override
-                public void onFailure(Throwable cause) {
-                    boolean countAsException = true;
-                    if (cause instanceof DLException) {
-                        final DLException dle = (DLException) cause;
-                        switch (dle.getCode()) {
-                        case FOUND:
-                            assert(cause instanceof OwnershipAcquireFailedException);
-                            countAsException = false;
-                            handleExceptionOnStreamOp(op, cause);
-                            break;
-                        case ALREADY_CLOSED:
-                            assert(cause instanceof AlreadyClosedException);
-                            op.fail(cause);
-                            handleAlreadyClosedException((AlreadyClosedException) cause);
-                            break;
-                        // exceptions that mostly from client (e.g. too large record)
-                        case NOT_IMPLEMENTED:
-                        case METADATA_EXCEPTION:
-                        case LOG_EMPTY:
-                        case LOG_NOT_FOUND:
-                        case TRUNCATED_TRANSACTION:
-                        case END_OF_STREAM:
-                        case TRANSACTION_OUT_OF_ORDER:
-                        case INVALID_STREAM_NAME:
-                        case TOO_LARGE_RECORD:
-                        case STREAM_NOT_READY:
-                        case OVER_CAPACITY:
-                            op.fail(cause);
-                            break;
-                        // the DL writer hits exception, simple set the stream to error status
-                        // and fail the request
-                        default:
-                            handleExceptionOnStreamOp(op, cause);
-                            break;
-                        }
-                    } else {
-                        handleExceptionOnStreamOp(op, cause);
-                    }
-                    if (countAsException) {
-                        countException(cause, streamExceptionStatLogger);
-                    }
-                }
-            });
-        } else {
-            if (null != lastException) {
-                op.fail(lastException);
-            } else {
-                op.fail(new StreamUnavailableException("Stream " + name + " is closed."));
-            }
-        }
-    }
-
-    /**
-     * Handle exception when executing <i>op</i>.
-     *
-     * @param op
-     *          stream operation executing
-     * @param cause
-     *          exception received when executing <i>op</i>
-     */
-    private void handleExceptionOnStreamOp(StreamOp op, final Throwable cause) {
-        AsyncLogWriter oldWriter = null;
-        boolean statusChanged = false;
-        synchronized (this) {
-            if (StreamStatus.INITIALIZED == status) {
-                oldWriter = setStreamStatus(StreamStatus.ERROR, StreamStatus.INITIALIZED, null, cause);
-                statusChanged = true;
-            }
-        }
-        if (statusChanged) {
-            Abortables.asyncAbort(oldWriter, false);
-            if (isCriticalException(cause)) {
-                logger.error("Failed to write data into stream {} : ", name, cause);
-            } else {
-                logger.warn("Failed to write data into stream {} : {}", name, cause.getMessage());
-            }
-            requestClose("Failed to write data into stream " + name + " : " + cause.getMessage());
-        }
-        op.fail(cause);
-    }
-
-    /**
-     * Handling already closed exception.
-     */
-    private void handleAlreadyClosedException(AlreadyClosedException ace) {
-        unexpectedExceptions.inc();
-        logger.error("Encountered unexpected exception when writing data into stream {} : ", name, ace);
-        fatalErrorHandler.notifyFatalError();
-    }
-
-    //
-    // Acquire streams
-    //
-
-    Future<Boolean> acquireStream() {
-        final Stopwatch stopwatch = Stopwatch.createStarted();
-        final Promise<Boolean> acquirePromise = new Promise<Boolean>();
-        manager.openAsyncLogWriter().addEventListener(
-            FutureUtils.OrderedFutureEventListener.of(new FutureEventListener<AsyncLogWriter>() {
-
-            @Override
-            public void onSuccess(AsyncLogWriter w) {
-                onAcquireStreamSuccess(w, stopwatch, acquirePromise);
-            }
-
-            @Override
-            public void onFailure(Throwable cause) {
-                onAcquireStreamFailure(cause, stopwatch, acquirePromise);
-            }
-
-        }, scheduler, getStreamName()));
-        return acquirePromise;
-    }
-
-    private void onAcquireStreamSuccess(AsyncLogWriter w,
-                                        Stopwatch stopwatch,
-                                        Promise<Boolean> acquirePromise) {
-        synchronized (txnLock) {
-            sequencer.setLastId(w.getLastTxId());
-        }
-        AsyncLogWriter oldWriter;
-        Queue<StreamOp> oldPendingOps;
-        boolean success;
-        synchronized (StreamImpl.this) {
-            oldWriter = setStreamStatus(StreamStatus.INITIALIZED,
-                    StreamStatus.INITIALIZING, w, null);
-            oldPendingOps = pendingOps;
-            pendingOps = new ArrayDeque<StreamOp>();
-            success = true;
-        }
-        // check if the stream is allowed to be acquired
-        if (!streamManager.allowAcquire(StreamImpl.this)) {
-            if (null != oldWriter) {
-                Abortables.asyncAbort(oldWriter, true);
-            }
-            int maxAcquiredPartitions = dynConf.getMaxAcquiredPartitionsPerProxy();
-            StreamUnavailableException sue = new StreamUnavailableException("Stream " + partition.getStream()
-                    + " is not allowed to acquire more than " + maxAcquiredPartitions + " partitions");
-            countException(sue, exceptionStatLogger);
-            logger.error("Failed to acquire stream {} because it is unavailable : {}",
-                    name, sue.getMessage());
-            synchronized (this) {
-                oldWriter = setStreamStatus(StreamStatus.ERROR,
-                        StreamStatus.INITIALIZED, null, sue);
-                // we don't switch the pending ops since they are already switched
-                // when setting the status to initialized
-                success = false;
-            }
-        }
-        processPendingRequestsAfterAcquire(success, oldWriter, oldPendingOps, stopwatch, acquirePromise);
-    }
-
-    private void onAcquireStreamFailure(Throwable cause,
-                                        Stopwatch stopwatch,
-                                        Promise<Boolean> acquirePromise) {
-        AsyncLogWriter oldWriter;
-        Queue<StreamOp> oldPendingOps;
-        boolean success;
-        if (cause instanceof AlreadyClosedException) {
-            countException(cause, streamExceptionStatLogger);
-            handleAlreadyClosedException((AlreadyClosedException) cause);
-            return;
-        } else {
-            if (isCriticalException(cause)) {
-                countException(cause, streamExceptionStatLogger);
-                logger.error("Failed to acquire stream {} : ", name, cause);
-            } else {
-                logger.warn("Failed to acquire stream {} : {}", name, cause.getMessage());
-            }
-            synchronized (StreamImpl.this) {
-                oldWriter = setStreamStatus(StreamStatus.ERROR,
-                        StreamStatus.INITIALIZING, null, cause);
-                oldPendingOps = pendingOps;
-                pendingOps = new ArrayDeque<StreamOp>();
-                success = false;
-            }
-        }
-        processPendingRequestsAfterAcquire(success, oldWriter, oldPendingOps, stopwatch, acquirePromise);
-    }
-
-    /**
-     * Process the pending request after acquired stream.
-     *
-     * @param success whether the acquisition succeed or not
-     * @param oldWriter the old writer to abort
-     * @param oldPendingOps the old pending ops to execute
-     * @param stopwatch stopwatch to measure the time spent on acquisition
-     * @param acquirePromise the promise to complete the acquire operation
-     */
-    void processPendingRequestsAfterAcquire(boolean success,
-                                            AsyncLogWriter oldWriter,
-                                            Queue<StreamOp> oldPendingOps,
-                                            Stopwatch stopwatch,
-                                            Promise<Boolean> acquirePromise) {
-        if (success) {
-            streamAcquireStat.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-        } else {
-            streamAcquireStat.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
-        }
-        for (StreamOp op : oldPendingOps) {
-            executeOp(op, success);
-            pendingOpsCounter.dec();
-        }
-        Abortables.asyncAbort(oldWriter, true);
-        FutureUtils.setValue(acquirePromise, success);
-    }
-
-    //
-    // Stream Status Changes
-    //
-
-    synchronized void setStreamInErrorStatus() {
-        if (StreamStatus.CLOSING == status || StreamStatus.CLOSED == status) {
-            return;
-        }
-        this.status = StreamStatus.ERROR;
-    }
-
-    /**
-     * Update the stream status. The changes are only applied when there isn't status changed.
-     *
-     * @param newStatus
-     *          new status
-     * @param oldStatus
-     *          old status
-     * @param writer
-     *          new log writer
-     * @param t
-     *          new exception
-     * @return old writer if it exists
-     */
-    synchronized AsyncLogWriter setStreamStatus(StreamStatus newStatus,
-                                                StreamStatus oldStatus,
-                                                AsyncLogWriter writer,
-                                                Throwable t) {
-        if (oldStatus != this.status) {
-            logger.info("Stream {} status already changed from {} -> {} when trying to change it to {}",
-                    new Object[] { name, oldStatus, this.status, newStatus });
-            return null;
-        }
-
-        String owner = null;
-        if (t instanceof OwnershipAcquireFailedException) {
-            owner = ((OwnershipAcquireFailedException) t).getCurrentOwner();
-        }
-
-        AsyncLogWriter oldWriter = this.writer;
-        this.writer = writer;
-        if (null != owner && owner.equals(clientId)) {
-            unexpectedExceptions.inc();
-            logger.error("I am waiting myself {} to release lock on stream {}, so have to shut myself down :",
-                         new Object[] { owner, name, t });
-            // I lost the ownership but left a lock over zookeeper
-            // I should not ask client to redirect to myself again as I can't handle it :(
-            // shutdown myself
-            fatalErrorHandler.notifyFatalError();
-            this.owner = null;
-        } else {
-            this.owner = owner;
-        }
-        this.lastException = t;
-        this.status = newStatus;
-        if (StreamStatus.INITIALIZED == newStatus) {
-            streamManager.notifyAcquired(this);
-            logger.info("Inserted acquired stream {} -> writer {}", name, this);
-        } else {
-            streamManager.notifyReleased(this);
-            logger.info("Removed acquired stream {} -> writer {}", name, this);
-        }
-        return oldWriter;
-    }
-
-    //
-    // Stream Close Functions
-    //
-
-    void close(DistributedLogManager dlm) {
-        if (null != dlm) {
-            try {
-                dlm.close();
-            } catch (IOException ioe) {
-                logger.warn("Failed to close dlm for {} : ", name, ioe);
-            }
-        }
-    }
-
-    @Override
-    public Future<Void> requestClose(String reason) {
-        return requestClose(reason, true);
-    }
-
-    Future<Void> requestClose(String reason, boolean uncache) {
-        final boolean abort;
-        closeLock.writeLock().lock();
-        try {
-            if (StreamStatus.CLOSING == status
-                || StreamStatus.CLOSED == status) {
-                return closePromise;
-            }
-            logger.info("Request to close stream {} : {}", getStreamName(), reason);
-            // if the stream isn't closed from INITIALIZED state, we abort the stream instead of closing it.
-            abort = StreamStatus.INITIALIZED != status;
-            status = StreamStatus.CLOSING;
-            streamManager.notifyReleased(this);
-        } finally {
-            closeLock.writeLock().unlock();
-        }
-        // we will fail the requests that are coming in between closing and closed only
-        // after the async writer is closed. so we could clear up the lock before redirect
-        // them.
-        close(abort, uncache);
-        return closePromise;
-    }
-
-    @Override
-    public void delete() throws IOException {
-        if (null != writer) {
-            Utils.close(writer);
-            synchronized (this) {
-                writer = null;
-                lastException = new StreamUnavailableException("Stream was deleted");
-            }
-        }
-        if (null == manager) {
-            throw new UnexpectedException("No stream " + name + " to delete");
-        }
-        manager.delete();
-    }
-
-    /**
-     * Post action executed after closing.
-     */
-    private void postClose(boolean uncache) {
-        closeManagerAndErrorOutPendingRequests();
-        unregisterGauge();
-        if (uncache) {
-            if (null != owner) {
-                long probationTimeoutMs = 2 * dlConfig.getZKSessionTimeoutMilliseconds() / 3;
-                streamManager.scheduleRemoval(this, probationTimeoutMs);
-            } else {
-                streamManager.notifyRemoved(this);
-                logger.info("Removed cached stream {}.", getStreamName());
-            }
-        }
-        FutureUtils.setValue(closePromise, null);
-    }
-
-    /**
-     * Shouldn't call close directly. The callers should call #requestClose instead
-     *
-     * @param shouldAbort shall we abort the stream instead of closing
-     */
-    private Future<Void> close(boolean shouldAbort, final boolean uncache) {
-        boolean abort;
-        closeLock.writeLock().lock();
-        try {
-            if (StreamStatus.CLOSED == status) {
-                return closePromise;
-            }
-            abort = shouldAbort || (StreamStatus.INITIALIZED != status && StreamStatus.CLOSING != status);
-            status = StreamStatus.CLOSED;
-            streamManager.notifyReleased(this);
-        } finally {
-            closeLock.writeLock().unlock();
-        }
-        logger.info("Closing stream {} ...", name);
-        // Close the writers to release the locks before failing the requests
-        Future<Void> closeWriterFuture;
-        if (abort) {
-            closeWriterFuture = Abortables.asyncAbort(writer, true);
-        } else {
-            closeWriterFuture = Utils.asyncClose(writer, true);
-        }
-        // close the manager and error out pending requests after close writer
-        Duration closeWaitDuration;
-        if (writerCloseTimeoutMs <= 0) {
-            closeWaitDuration = Duration.Top();
-        } else {
-            closeWaitDuration = Duration.fromMilliseconds(writerCloseTimeoutMs);
-        }
-
-        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);
-                    }
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        if (cause instanceof TimeoutException) {
-                            writerCloseTimeoutCounter.inc();
-                        }
-                        postClose(uncache);
-                    }
-                }, scheduler, name));
-        return closePromise;
-    }
-
-    private void closeManagerAndErrorOutPendingRequests() {
-        close(manager);
-        // Failed the pending requests.
-        Queue<StreamOp> oldPendingOps;
-        synchronized (this) {
-            oldPendingOps = pendingOps;
-            pendingOps = new ArrayDeque<StreamOp>();
-        }
-        StreamUnavailableException closingException =
-                new StreamUnavailableException("Stream " + name + " is closed.");
-        for (StreamOp op : oldPendingOps) {
-            op.fail(closingException);
-            pendingOpsCounter.dec();
-        }
-        limiter.close();
-        logger.info("Closed stream {}.", name);
-    }
-
-    /**
-     * clean up the gauge to help GC.
-     */
-    private void unregisterGauge(){
-        streamLogger.unregisterGauge("stream_status", this.streamStatusGauge);
-    }
-
-    // Test-only apis
-
-    @VisibleForTesting
-    public int numPendingOps() {
-        Queue<StreamOp> queue = pendingOps;
-        return null == queue ? 0 : queue.size();
-    }
-
-    @VisibleForTesting
-    public StreamStatus getStatus() {
-        return status;
-    }
-
-    @VisibleForTesting
-    public void setStatus(StreamStatus status) {
-        this.status = status;
-    }
-
-    @VisibleForTesting
-    public AsyncLogWriter getWriter() {
-        return writer;
-    }
-
-    @VisibleForTesting
-    public DistributedLogManager getManager() {
-        return manager;
-    }
-
-    @VisibleForTesting
-    public Throwable getLastException() {
-        return lastException;
-    }
-
-    @VisibleForTesting
-    public Future<Void> getCloseFuture() {
-        return closePromise;
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamManager.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamManager.java
deleted file mode 100644
index 7f7d44e..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamManager.java
+++ /dev/null
@@ -1,142 +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 com.twitter.distributedlog.service.stream;
-
-import com.google.common.base.Optional;
-import com.twitter.util.Future;
-import java.io.IOException;
-import java.util.List;
-import java.util.Map;
-
-/**
- * Manage lifecycle of streams.
- *
- * <p>StreamManager is responsible for creating, destroying, and keeping track of Stream objects.
- *
- * <p>Stream objects, which are managed by StreamManager and created by StreamFactory, are essentially the
- * per stream request handlers, responsible fo dispatching ex. write requests to an underlying AsyncLogWriter,
- * managing stream lock, interpreting exceptions, error conditions, and etc.
- */
-public interface StreamManager {
-
-    /**
-     * Get a cached stream, returning null if it doesnt exist.
-     * @param stream name
-     * @return the cached stream
-     */
-    Stream getStream(String stream);
-
-    /**
-     * Get a cached stream and create a new one if it doesnt exist.
-     * @param streamName stream name
-     * @param start whether to start the stream after it is created.
-     * @return future satisfied once close complete
-     */
-    Stream getOrCreateStream(String streamName, boolean start) throws IOException;
-
-    /**
-     * Asynchronously create a new stream.
-     * @param stream
-     * @return Future satisfied once the stream is created
-     */
-    Future<Void> createStreamAsync(String stream);
-
-    /**
-     * Is acquiring stream allowed?
-     *
-     * @param stream
-     *          stream instance
-     * @return true if it is allowed to acquire this stream, otherwise false.
-     */
-    boolean allowAcquire(Stream stream);
-
-    /**
-     * Notify the manager that a stream was acquired.
-     * @param stream being acquired
-     */
-    void notifyAcquired(Stream stream);
-
-    /**
-     * Notify the manager that a stream was released.
-     * @param stream being released
-     */
-    void notifyReleased(Stream stream);
-
-    /**
-     * Notify the manager that a stream was completely removed.
-     * @param stream being uncached
-     * @return whether the stream existed or not
-     */
-    boolean notifyRemoved(Stream stream);
-
-    /**
-     * Asynchronous delete method.
-     * @param streamName stream name
-     * @return future satisfied once delete complete
-     */
-    Future<Void> deleteAndRemoveAsync(String streamName);
-
-    /**
-     * Asynchronous close and uncache method.
-     * @param streamName stream name
-     * @return future satisfied once close and uncache complete
-     */
-    Future<Void> closeAndRemoveAsync(String streamName);
-
-    /**
-     * Close and uncache after delayMs.
-     * @param stream to remove
-     */
-    void scheduleRemoval(Stream stream, long delayMs);
-
-    /**
-     * Close all stream.
-     * @return future satisfied all streams closed
-     */
-    Future<List<Void>> closeStreams();
-
-    /**
-     * Return map with stream ownership info.
-     * @param regex for filtering streams
-     * @return map containing ownership info
-     */
-    Map<String, String> getStreamOwnershipMap(Optional<String> regex);
-
-    /**
-     * Number of acquired streams.
-     * @return number of acquired streams
-     */
-    int numAcquired();
-
-    /**
-     * Number of cached streams.
-     * @return number of cached streams
-     */
-    int numCached();
-
-    /**
-     * Is the stream denoted by streamName in the acquired state.
-     * @return true if the stream is in the acquired state
-     */
-    boolean isAcquired(String streamName);
-
-    /**
-     * Close manager and disallow further activity.
-     */
-    void close();
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamManagerImpl.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamManagerImpl.java
deleted file mode 100644
index 8b36d3b..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamManagerImpl.java
+++ /dev/null
@@ -1,413 +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 com.twitter.distributedlog.service.stream;
-
-import com.google.common.annotations.VisibleForTesting;
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.ServiceUnavailableException;
-import com.twitter.distributedlog.exceptions.StreamUnavailableException;
-import com.twitter.distributedlog.exceptions.UnexpectedException;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.service.config.StreamConfigProvider;
-import com.twitter.distributedlog.service.streamset.Partition;
-import com.twitter.distributedlog.service.streamset.PartitionMap;
-import com.twitter.distributedlog.service.streamset.StreamPartitionConverter;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.util.Future;
-import com.twitter.util.Promise;
-
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.RejectedExecutionException;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.locks.ReentrantReadWriteLock;
-
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * StreamManagerImpl is the default implementation responsible for creating, destroying, and keeping track
- * of Streams.
- *
- * <p>StreamFactory, supplied to StreamManagerImpl in the constructor below, is reposible simply for creating
- * a stream object in isolation from the rest of the system. We pass a StreamFactory in instead of simply
- * creating StreamImpl's ourselves in order to inject dependencies without bloating the StreamManagerImpl
- * constructor.
- */
-public class StreamManagerImpl implements StreamManager {
-
-    private static final Logger logger = LoggerFactory.getLogger(StreamManagerImpl.class);
-
-    private final ConcurrentHashMap<String, Stream> streams =
-        new ConcurrentHashMap<String, Stream>();
-    private final AtomicInteger numCached = new AtomicInteger(0);
-
-    private final ConcurrentHashMap<String, Stream> acquiredStreams =
-        new ConcurrentHashMap<String, Stream>();
-    private final AtomicInteger numAcquired = new AtomicInteger(0);
-
-    //
-    // Partitions
-    //
-    private final StreamPartitionConverter partitionConverter;
-    private final PartitionMap cachedPartitions = new PartitionMap();
-    private final PartitionMap acquiredPartitions = new PartitionMap();
-
-    final ReentrantReadWriteLock closeLock = new ReentrantReadWriteLock();
-    private final ScheduledExecutorService executorService;
-    private final DistributedLogConfiguration dlConfig;
-    private final StreamConfigProvider streamConfigProvider;
-    private final String clientId;
-    private boolean closed = false;
-    private final StreamFactory streamFactory;
-    private final DistributedLogNamespace dlNamespace;
-
-    public StreamManagerImpl(String clientId,
-                             DistributedLogConfiguration dlConfig,
-                             ScheduledExecutorService executorService,
-                             StreamFactory streamFactory,
-                             StreamPartitionConverter partitionConverter,
-                             StreamConfigProvider streamConfigProvider,
-                             DistributedLogNamespace dlNamespace) {
-        this.clientId = clientId;
-        this.executorService = executorService;
-        this.streamFactory = streamFactory;
-        this.partitionConverter = partitionConverter;
-        this.dlConfig = dlConfig;
-        this.streamConfigProvider = streamConfigProvider;
-        this.dlNamespace = dlNamespace;
-    }
-
-    private DynamicDistributedLogConfiguration getDynConf(String streamName) {
-        Optional<DynamicDistributedLogConfiguration> dynDlConf =
-                streamConfigProvider.getDynamicStreamConfig(streamName);
-        if (dynDlConf.isPresent()) {
-            return dynDlConf.get();
-        } else {
-            return ConfUtils.getConstDynConf(dlConfig);
-        }
-    }
-
-    @Override
-    public boolean allowAcquire(Stream stream) {
-        return acquiredPartitions.addPartition(
-                stream.getPartition(),
-                stream.getStreamConfiguration().getMaxAcquiredPartitionsPerProxy());
-    }
-
-    /**
-     * Must be enqueued to an executor to avoid deadlocks (close and execute-op both
-     * try to acquire the same read-write lock).
-     */
-    @Override
-    public Future<Void> deleteAndRemoveAsync(final String stream) {
-        final Promise<Void> result = new Promise<Void>();
-        java.util.concurrent.Future<?> scheduleFuture = schedule(new Runnable() {
-            @Override
-            public void run() {
-                result.become(doDeleteAndRemoveAsync(stream));
-            }
-        }, 0);
-        if (null == scheduleFuture) {
-            return Future.exception(
-                new ServiceUnavailableException("Couldn't schedule a delete task."));
-        }
-        return result;
-    }
-
-    /**
-     * Must be enqueued to an executor to avoid deadlocks (close and execute-op both
-     * try to acquire the same read-write lock).
-     */
-    @Override
-    public Future<Void> closeAndRemoveAsync(final String streamName) {
-        final Promise<Void> releasePromise = new Promise<Void>();
-        java.util.concurrent.Future<?> scheduleFuture = schedule(new Runnable() {
-            @Override
-            public void run() {
-                releasePromise.become(doCloseAndRemoveAsync(streamName));
-            }
-        }, 0);
-        if (null == scheduleFuture) {
-            return Future.exception(
-                new ServiceUnavailableException("Couldn't schedule a release task."));
-        }
-        return releasePromise;
-    }
-
-    @Override
-    public Future<Void> createStreamAsync(final String stream) {
-        final Promise<Void> createPromise = new Promise<Void>();
-        java.util.concurrent.Future<?> scheduleFuture = schedule(new Runnable() {
-            @Override
-            public void run() {
-                try {
-                    dlNamespace.createLog(stream);
-                    createPromise.setValue(null);
-                } catch (Exception e) {
-                    createPromise.setException(e);
-                }
-            }
-        }, 0);
-        if (null == scheduleFuture) {
-            return Future.exception(
-                new ServiceUnavailableException("Couldn't schedule a create task."));
-        }
-        return createPromise;
-    }
-
-    @Override
-    public void notifyReleased(Stream stream) {
-        acquiredPartitions.removePartition(stream.getPartition());
-        if (acquiredStreams.remove(stream.getStreamName(), stream)) {
-            numAcquired.getAndDecrement();
-        }
-    }
-
-    @Override
-    public void notifyAcquired(Stream stream) {
-        if (null == acquiredStreams.put(stream.getStreamName(), stream)) {
-            numAcquired.getAndIncrement();
-        }
-    }
-
-    @Override
-    public boolean notifyRemoved(Stream stream) {
-        cachedPartitions.removePartition(stream.getPartition());
-        if (streams.remove(stream.getStreamName(), stream)) {
-            numCached.getAndDecrement();
-            return true;
-        }
-        return false;
-    }
-
-    @Override
-    public Map<String, String> getStreamOwnershipMap(Optional<String> regex) {
-        Map<String, String> ownershipMap = new HashMap<String, String>();
-        for (Map.Entry<String, Stream> entry : acquiredStreams.entrySet()) {
-            String name = entry.getKey();
-            if (regex.isPresent() && !name.matches(regex.get())) {
-                continue;
-            }
-            Stream stream = entry.getValue();
-            if (null == stream) {
-                continue;
-            }
-            String owner = stream.getOwner();
-            if (null == owner) {
-                ownershipMap.put(name, clientId);
-            }
-        }
-        return ownershipMap;
-    }
-
-    @Override
-    public Stream getStream(String stream) {
-        return streams.get(stream);
-    }
-
-    @Override
-    public Stream getOrCreateStream(String streamName, boolean start) throws IOException {
-        Stream stream = streams.get(streamName);
-        if (null == stream) {
-            closeLock.readLock().lock();
-            try {
-                if (closed) {
-                    return null;
-                }
-                DynamicDistributedLogConfiguration dynConf = getDynConf(streamName);
-                int maxCachedPartitions = dynConf.getMaxCachedPartitionsPerProxy();
-
-                // get partition from the stream name
-                Partition partition = partitionConverter.convert(streamName);
-
-                // add partition to cached map
-                if (!cachedPartitions.addPartition(partition, maxCachedPartitions)) {
-                    throw new StreamUnavailableException("Stream " + streamName
-                            + " is not allowed to cache more than " + maxCachedPartitions + " partitions");
-                }
-
-                stream = newStream(streamName, dynConf);
-                Stream oldWriter = streams.putIfAbsent(streamName, stream);
-                if (null != oldWriter) {
-                    stream = oldWriter;
-                } else {
-                    numCached.getAndIncrement();
-                    logger.info("Inserted mapping stream name {} -> stream {}", streamName, stream);
-                    stream.initialize();
-                    if (start) {
-                        stream.start();
-                    }
-                }
-            } finally {
-                closeLock.readLock().unlock();
-            }
-        }
-        return stream;
-    }
-
-    @Override
-    public Future<List<Void>> closeStreams() {
-        int numAcquired = acquiredStreams.size();
-        int numCached = streams.size();
-        logger.info("Closing all acquired streams : acquired = {}, cached = {}.",
-            numAcquired, numCached);
-        Set<Stream> streamsToClose = new HashSet<Stream>();
-        streamsToClose.addAll(streams.values());
-        return closeStreams(streamsToClose, Optional.<RateLimiter>absent());
-    }
-
-    @Override
-    public void scheduleRemoval(final Stream stream, long delayMs) {
-        if (delayMs > 0) {
-            logger.info("Scheduling removal of stream {} from cache after {} sec.",
-                    stream.getStreamName(), delayMs);
-        }
-        schedule(new Runnable() {
-            @Override
-            public void run() {
-                if (notifyRemoved(stream)) {
-                    logger.info("Removed cached stream {} after probation.", stream.getStreamName());
-                } else {
-                    logger.info("Cached stream {} already removed.", stream.getStreamName());
-                }
-            }
-        }, delayMs);
-    }
-
-    @Override
-    public int numAcquired() {
-        return numAcquired.get();
-    }
-
-    @Override
-    public int numCached() {
-        return numCached.get();
-    }
-
-    @Override
-    public boolean isAcquired(String streamName) {
-        return acquiredStreams.containsKey(streamName);
-    }
-
-    @Override
-    public void close() {
-        closeLock.writeLock().lock();
-        try {
-            if (closed) {
-                return;
-            }
-            closed = true;
-        } finally {
-            closeLock.writeLock().unlock();
-        }
-    }
-
-    private Future<List<Void>> closeStreams(Set<Stream> streamsToClose, Optional<RateLimiter> rateLimiter) {
-        if (streamsToClose.isEmpty()) {
-            logger.info("No streams to close.");
-            List<Void> emptyList = new ArrayList<Void>();
-            return Future.value(emptyList);
-        }
-        List<Future<Void>> futures = new ArrayList<Future<Void>>(streamsToClose.size());
-        for (Stream stream : streamsToClose) {
-            if (rateLimiter.isPresent()) {
-                rateLimiter.get().acquire();
-            }
-            futures.add(stream.requestClose("Close Streams"));
-        }
-        return Future.collect(futures);
-    }
-
-    private Stream newStream(String name, DynamicDistributedLogConfiguration streamConf) {
-        return streamFactory.create(name, streamConf, this);
-    }
-
-    public Future<Void> doCloseAndRemoveAsync(final String streamName) {
-        Stream stream = streams.get(streamName);
-        if (null == stream) {
-            logger.info("No stream {} to release.", streamName);
-            return Future.value(null);
-        } else {
-            return stream.requestClose("release ownership");
-        }
-    }
-
-    /**
-     * Dont schedule if we're closed - closeLock is acquired to close, so if we acquire the
-     * lock and discover we're not closed, we won't schedule.
-     */
-    private java.util.concurrent.Future<?> schedule(Runnable runnable, long delayMs) {
-        closeLock.readLock().lock();
-        try {
-            if (closed) {
-                return null;
-            } else if (delayMs > 0) {
-                return executorService.schedule(runnable, delayMs, TimeUnit.MILLISECONDS);
-            } else {
-                return executorService.submit(runnable);
-            }
-        } catch (RejectedExecutionException ree) {
-            logger.error("Failed to schedule task {} in {} ms : ",
-                    new Object[] { runnable, delayMs, ree });
-            return null;
-        } finally {
-            closeLock.readLock().unlock();
-        }
-    }
-
-    private Future<Void> doDeleteAndRemoveAsync(final String streamName) {
-        Stream stream = streams.get(streamName);
-        if (null == stream) {
-            logger.warn("No stream {} to delete.", streamName);
-            return Future.exception(new UnexpectedException("No stream " + streamName + " to delete."));
-        } else {
-            Future<Void> result;
-            logger.info("Deleting stream {}, {}", streamName, stream);
-            try {
-                stream.delete();
-                result = stream.requestClose("Stream Deleted");
-            } catch (IOException e) {
-                logger.error("Failed on removing stream {} : ", streamName, e);
-                result = Future.exception(e);
-            }
-            return result;
-        }
-    }
-
-    @VisibleForTesting
-    public ConcurrentHashMap<String, Stream> getCachedStreams() {
-        return streams;
-    }
-
-    @VisibleForTesting
-    public ConcurrentHashMap<String, Stream> getAcquiredStreams() {
-        return acquiredStreams;
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamOp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamOp.java
deleted file mode 100644
index a2cbc80..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamOp.java
+++ /dev/null
@@ -1,75 +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 com.twitter.distributedlog.service.stream;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.thrift.service.ResponseHeader;
-import com.twitter.distributedlog.util.Sequencer;
-import com.twitter.util.Future;
-
-/**
- * An operation applied to a stream.
- */
-public interface StreamOp {
-    /**
-     * Execute a stream op with the supplied writer.
-     *
-     * @param writer active writer for applying the change
-     * @param sequencer sequencer used for generating transaction id for stream operations
-     * @param txnLock transaction lock to guarantee ordering of transaction id
-     * @return a future satisfied when the operation completes execution
-     */
-    Future<Void> execute(AsyncLogWriter writer,
-                         Sequencer sequencer,
-                         Object txnLock);
-
-    /**
-     * Invoked before the stream op is executed.
-     */
-    void preExecute() throws DLException;
-
-    /**
-     * Return the response header (containing the status code etc.).
-     *
-     * @return A future containing the response header or the exception
-     *      encountered by the op if it failed.
-     */
-    Future<ResponseHeader> responseHeader();
-
-    /**
-     * Abort the operation with the givem exception.
-     */
-    void fail(Throwable t);
-
-    /**
-     * Return the stream name.
-     */
-    String streamName();
-
-    /**
-     * Stopwatch gives the start time of the operation.
-     */
-    Stopwatch stopwatch();
-
-    /**
-     * Compute checksum from arguments.
-     */
-    Long computeChecksum();
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamOpStats.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamOpStats.java
deleted file mode 100644
index c1019c6..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/StreamOpStats.java
+++ /dev/null
@@ -1,104 +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 com.twitter.distributedlog.service.stream;
-
-import com.twitter.distributedlog.service.streamset.Partition;
-import com.twitter.distributedlog.stats.BroadCastStatsLogger;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Encapsulate stream op stats construction to make it easier to access stream
- * op stats consistently from different scopes.
- */
-public class StreamOpStats {
-    private final StatsLogger baseStatsLogger;
-    private final StatsLogger requestStatsLogger;
-    private final StatsLogger recordsStatsLogger;
-    private final StatsLogger requestDeniedStatsLogger;
-    private final StatsLogger streamStatsLogger;
-
-    public StreamOpStats(StatsLogger statsLogger,
-                         StatsLogger perStreamStatsLogger) {
-        this.baseStatsLogger = statsLogger;
-        this.requestStatsLogger = statsLogger.scope("request");
-        this.recordsStatsLogger = statsLogger.scope("records");
-        this.requestDeniedStatsLogger = statsLogger.scope("denied");
-        this.streamStatsLogger = perStreamStatsLogger;
-    }
-
-    public StatsLogger baseStatsLogger(String opName) {
-        return baseStatsLogger;
-    }
-
-    public Counter baseCounter(String opName) {
-        return baseStatsLogger.getCounter(opName);
-    }
-
-    public StatsLogger baseScope(String opName) {
-        return baseStatsLogger.scope(opName);
-    }
-
-    public OpStatsLogger requestLatencyStat(String opName) {
-        return requestStatsLogger.getOpStatsLogger(opName);
-    }
-
-    public StatsLogger requestScope(String scopeName) {
-        return requestStatsLogger.scope(scopeName);
-    }
-
-    public Counter scopedRequestCounter(String opName, String counterName) {
-        return requestScope(opName).getCounter(counterName);
-    }
-
-    public Counter requestCounter(String counterName) {
-        return requestStatsLogger.getCounter(counterName);
-    }
-
-    public Counter requestPendingCounter(String counterName) {
-        return requestCounter(counterName);
-    }
-
-    public Counter requestDeniedCounter(String counterName) {
-        return requestDeniedStatsLogger.getCounter(counterName);
-    }
-
-    public Counter recordsCounter(String counterName) {
-        return recordsStatsLogger.getCounter(counterName);
-    }
-
-    public StatsLogger streamRequestStatsLogger(Partition partition) {
-        return BroadCastStatsLogger.masterslave(
-            streamStatsLogger.scope(partition.getStream()).scope("partition")
-                .scope(partition.getPaddedId()), streamStatsLogger.scope(partition.getStream())
-                .scope("aggregate"));
-    }
-
-    public StatsLogger streamRequestScope(Partition partition, String scopeName) {
-        return streamRequestStatsLogger(partition).scope(scopeName);
-    }
-
-    public OpStatsLogger streamRequestLatencyStat(Partition partition, String opName) {
-        return streamRequestStatsLogger(partition).getOpStatsLogger(opName);
-    }
-
-    public Counter streamRequestCounter(Partition partition, String opName, String counterName) {
-        return streamRequestScope(partition, opName).getCounter(counterName);
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/TruncateOp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/TruncateOp.java
deleted file mode 100644
index b0b4df2..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/TruncateOp.java
+++ /dev/null
@@ -1,91 +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 com.twitter.distributedlog.service.stream;
-
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.RequestDeniedException;
-import com.twitter.distributedlog.service.ResponseUtils;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.util.ProtocolUtils;
-import com.twitter.distributedlog.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.
- */
-public class TruncateOp extends AbstractWriteOp {
-
-    private static final Logger logger = LoggerFactory.getLogger(TruncateOp.class);
-
-    private final Counter deniedTruncateCounter;
-    private final DLSN dlsn;
-    private final AccessControlManager accessControlManager;
-
-    public TruncateOp(String stream,
-                      DLSN dlsn,
-                      StatsLogger statsLogger,
-                      StatsLogger perStreamStatsLogger,
-                      Long checksum,
-                      Feature checksumDisabledFeature,
-                      AccessControlManager accessControlManager) {
-        super(stream, requestStat(statsLogger, "truncate"), checksum, checksumDisabledFeature);
-        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-        this.deniedTruncateCounter = streamOpStats.requestDeniedCounter("truncate");
-        this.accessControlManager = accessControlManager;
-        this.dlsn = dlsn;
-    }
-
-    @Override
-    public Long computeChecksum() {
-        return ProtocolUtils.truncateOpCRC32(stream, dlsn);
-    }
-
-    @Override
-    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
-                                              Sequencer sequencer,
-                                              Object txnLock) {
-        if (!stream.equals(writer.getStreamName())) {
-            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();
-            }
-        });
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!accessControlManager.allowTruncate(stream)) {
-            deniedTruncateCounter.inc();
-            throw new RequestDeniedException(stream, "truncate");
-        }
-        super.preExecute();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/WriteOp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/WriteOp.java
deleted file mode 100644
index 69739dc..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/WriteOp.java
+++ /dev/null
@@ -1,173 +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 com.twitter.distributedlog.service.stream;
-
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.RequestDeniedException;
-import com.twitter.distributedlog.service.ResponseUtils;
-import com.twitter.distributedlog.service.config.ServerConfiguration;
-import com.twitter.distributedlog.service.streamset.Partition;
-import com.twitter.distributedlog.service.streamset.StreamPartitionConverter;
-import com.twitter.distributedlog.thrift.service.ResponseHeader;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.util.ProtocolUtils;
-import com.twitter.distributedlog.util.Sequencer;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import java.nio.ByteBuffer;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.feature.Feature;
-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 scala.runtime.AbstractFunction1;
-
-/**
- * Operation to write a single record to a log stream.
- */
-public class WriteOp extends AbstractWriteOp implements WriteOpWithPayload {
-
-    private static final Logger logger = LoggerFactory.getLogger(WriteOp.class);
-
-    private final byte[] payload;
-    private final boolean isRecordSet;
-
-    // Stats
-    private final Counter deniedWriteCounter;
-    private final Counter successRecordCounter;
-    private final Counter failureRecordCounter;
-    private final Counter redirectRecordCounter;
-    private final OpStatsLogger latencyStat;
-    private final Counter bytes;
-    private final Counter writeBytes;
-
-    private final byte dlsnVersion;
-    private final AccessControlManager accessControlManager;
-
-    public WriteOp(String stream,
-                   ByteBuffer data,
-                   StatsLogger statsLogger,
-                   StatsLogger perStreamStatsLogger,
-                   StreamPartitionConverter streamPartitionConverter,
-                   ServerConfiguration conf,
-                   byte dlsnVersion,
-                   Long checksum,
-                   boolean isRecordSet,
-                   Feature checksumDisabledFeature,
-                   AccessControlManager accessControlManager) {
-        super(stream, requestStat(statsLogger, "write"), checksum, checksumDisabledFeature);
-        payload = new byte[data.remaining()];
-        data.get(payload);
-        this.isRecordSet = isRecordSet;
-
-        final Partition partition = streamPartitionConverter.convert(stream);
-        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
-        this.successRecordCounter = streamOpStats.recordsCounter("success");
-        this.failureRecordCounter = streamOpStats.recordsCounter("failure");
-        this.redirectRecordCounter = streamOpStats.recordsCounter("redirect");
-        this.deniedWriteCounter = streamOpStats.requestDeniedCounter("write");
-        this.writeBytes = streamOpStats.scopedRequestCounter("write", "bytes");
-        this.latencyStat = streamOpStats.streamRequestLatencyStat(partition, "write");
-        this.bytes = streamOpStats.streamRequestCounter(partition, "write", "bytes");
-
-        this.dlsnVersion = dlsnVersion;
-        this.accessControlManager = accessControlManager;
-
-        final long size = getPayloadSize();
-        result().addEventListener(new FutureEventListener<WriteResponse>() {
-            @Override
-            public void onSuccess(WriteResponse response) {
-                if (response.getHeader().getCode() == StatusCode.SUCCESS) {
-                    latencyStat.registerSuccessfulEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
-                    bytes.add(size);
-                    writeBytes.add(size);
-                } else {
-                    latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
-                }
-            }
-            @Override
-            public void onFailure(Throwable cause) {
-                latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
-            }
-        });
-    }
-
-    @Override
-    public long getPayloadSize() {
-      return payload.length;
-    }
-
-    @Override
-    public Long computeChecksum() {
-        return ProtocolUtils.writeOpCRC32(stream, payload);
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!accessControlManager.allowWrite(stream)) {
-            deniedWriteCounter.inc();
-            throw new RequestDeniedException(stream, "write");
-        }
-        super.preExecute();
-    }
-
-    @Override
-    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
-                                              Sequencer sequencer,
-                                              Object txnLock) {
-        if (!stream.equals(writer.getStreamName())) {
-            logger.error("Write: Stream Name Mismatch in the Stream Map {}, {}", stream, writer.getStreamName());
-            return Future.exception(new IllegalStateException("The stream mapping is incorrect, fail the request"));
-        }
-
-        long txnId;
-        Future<DLSN> writeResult;
-        synchronized (txnLock) {
-            txnId = sequencer.nextId();
-            LogRecord record = new LogRecord(txnId, payload);
-            if (isRecordSet) {
-                record.setRecordSet();
-            }
-            writeResult = writer.write(record);
-        }
-        return writeResult.map(new AbstractFunction1<DLSN, WriteResponse>() {
-            @Override
-            public WriteResponse apply(DLSN value) {
-                successRecordCounter.inc();
-                return ResponseUtils.writeSuccess().setDlsn(value.serialize(dlsnVersion));
-            }
-        });
-    }
-
-    @Override
-    protected void fail(ResponseHeader header) {
-        if (StatusCode.FOUND == header.getCode()) {
-            redirectRecordCounter.inc();
-        } else {
-            failureRecordCounter.inc();
-        }
-        super.fail(header);
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/WriteOpWithPayload.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/WriteOpWithPayload.java
deleted file mode 100644
index 6cc9063..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/WriteOpWithPayload.java
+++ /dev/null
@@ -1,27 +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 com.twitter.distributedlog.service.stream;
-
-/**
- * A write operation with payload.
- */
-public interface WriteOpWithPayload {
-
-    // Return the payload size in bytes
-    long getPayloadSize();
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/admin/AdminOp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/admin/AdminOp.java
deleted file mode 100644
index 6d2d2ea..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/admin/AdminOp.java
+++ /dev/null
@@ -1,40 +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 com.twitter.distributedlog.service.stream.admin;
-
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.util.Future;
-
-/**
- * Admin operation interface.
- */
-public interface AdminOp<RespT> {
-
-    /**
-     * Invoked before the stream op is executed.
-     */
-    void preExecute() throws DLException;
-
-    /**
-     * Execute the operation.
-     *
-     * @return the future represents the response of the operation
-     */
-    Future<RespT> execute();
-
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/admin/CreateOp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/admin/CreateOp.java
deleted file mode 100644
index 478201e..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/admin/CreateOp.java
+++ /dev/null
@@ -1,57 +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 com.twitter.distributedlog.service.stream.admin;
-
-import static com.twitter.distributedlog.service.stream.AbstractStreamOp.requestStat;
-
-import com.twitter.distributedlog.service.ResponseUtils;
-import com.twitter.distributedlog.service.stream.StreamManager;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.StatsLogger;
-import scala.runtime.AbstractFunction1;
-
-/**
- * Operation to create log stream.
- */
-public class CreateOp extends StreamAdminOp {
-
-  public CreateOp(String stream,
-                  StatsLogger statsLogger,
-                  StreamManager streamManager,
-                  Long checksum,
-                  Feature checksumEnabledFeature) {
-    super(stream,
-            streamManager,
-            requestStat(statsLogger, "create"),
-            checksum,
-            checksumEnabledFeature);
-  }
-
-  @Override
-  protected Future<WriteResponse> executeOp() {
-    Future<Void> result = streamManager.createStreamAsync(stream);
-    return result.map(new AbstractFunction1<Void, WriteResponse>() {
-      @Override
-      public WriteResponse apply(Void value) {
-        return ResponseUtils.writeSuccess();
-      }
-    });
-  }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/admin/StreamAdminOp.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/admin/StreamAdminOp.java
deleted file mode 100644
index 4fad542..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/admin/StreamAdminOp.java
+++ /dev/null
@@ -1,100 +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 com.twitter.distributedlog.service.stream.admin;
-
-import com.google.common.base.Stopwatch;
-import com.twitter.distributedlog.exceptions.ChecksumFailedException;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.service.ResponseUtils;
-import com.twitter.distributedlog.service.stream.StreamManager;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.util.ProtocolUtils;
-import com.twitter.util.Future;
-import com.twitter.util.FutureTransformer;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.OpStatsLogger;
-
-/**
- * Stream admin op.
- */
-public abstract class StreamAdminOp implements AdminOp<WriteResponse> {
-
-    protected final String stream;
-    protected final StreamManager streamManager;
-    protected final OpStatsLogger opStatsLogger;
-    protected final Stopwatch stopwatch = Stopwatch.createUnstarted();
-    protected final Long checksum;
-    protected final Feature checksumDisabledFeature;
-
-    protected StreamAdminOp(String stream,
-                            StreamManager streamManager,
-                            OpStatsLogger statsLogger,
-                            Long checksum,
-                            Feature checksumDisabledFeature) {
-        this.stream = stream;
-        this.streamManager = streamManager;
-        this.opStatsLogger = statsLogger;
-        // start here in case the operation is failed before executing.
-        stopwatch.reset().start();
-        this.checksum = checksum;
-        this.checksumDisabledFeature = checksumDisabledFeature;
-    }
-
-    protected Long computeChecksum() {
-        return ProtocolUtils.streamOpCRC32(stream);
-    }
-
-    @Override
-    public void preExecute() throws DLException {
-        if (!checksumDisabledFeature.isAvailable() && null != checksum) {
-            Long serverChecksum = computeChecksum();
-            if (null != serverChecksum && !checksum.equals(serverChecksum)) {
-                throw new ChecksumFailedException();
-            }
-        }
-    }
-
-    /**
-     * Execute the operation.
-     *
-     * @return execute operation
-     */
-    protected abstract Future<WriteResponse> executeOp();
-
-    @Override
-    public Future<WriteResponse> execute() {
-        return executeOp().transformedBy(new FutureTransformer<WriteResponse, WriteResponse>() {
-
-            @Override
-            public WriteResponse map(WriteResponse response) {
-                opStatsLogger.registerSuccessfulEvent(
-                        stopwatch.elapsed(TimeUnit.MICROSECONDS));
-                return response;
-            }
-
-            @Override
-            public WriteResponse handle(Throwable cause) {
-                opStatsLogger.registerFailedEvent(
-                        stopwatch.elapsed(TimeUnit.MICROSECONDS));
-                return ResponseUtils.write(ResponseUtils.exceptionToHeader(cause));
-            }
-
-        });
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/admin/package-info.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/admin/package-info.java
deleted file mode 100644
index 5ec997c..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/admin/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Stream Related Admin Operations.
- */
-package com.twitter.distributedlog.service.stream.admin;
\ No newline at end of file
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/DynamicRequestLimiter.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/DynamicRequestLimiter.java
deleted file mode 100644
index d684de5..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/DynamicRequestLimiter.java
+++ /dev/null
@@ -1,94 +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 com.twitter.distributedlog.service.stream.limiter;
-
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-import com.twitter.distributedlog.limiter.RequestLimiter;
-import java.io.Closeable;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.apache.commons.configuration.event.ConfigurationEvent;
-import org.apache.commons.configuration.event.ConfigurationListener;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Dynamically rebuild a rate limiter when the supplied dynamic config changes.
- *
- * <p>Subclasses implement build() to build the limiter. DynamicRequestLimiter must be closed to deregister
- * the config listener.
- */
-public abstract class DynamicRequestLimiter<Req> implements RequestLimiter<Req>, Closeable {
-    private static final Logger LOG = LoggerFactory.getLogger(DynamicRequestLimiter.class);
-
-    private final ConfigurationListener listener;
-    private final Feature rateLimitDisabledFeature;
-    volatile RequestLimiter<Req> limiter;
-    final DynamicDistributedLogConfiguration dynConf;
-
-    public DynamicRequestLimiter(DynamicDistributedLogConfiguration dynConf,
-                                 StatsLogger statsLogger,
-                                 Feature rateLimitDisabledFeature) {
-        final StatsLogger limiterStatsLogger = statsLogger.scope("dynamic");
-        this.dynConf = dynConf;
-        this.rateLimitDisabledFeature = rateLimitDisabledFeature;
-        this.listener = new ConfigurationListener() {
-            @Override
-            public void configurationChanged(ConfigurationEvent event) {
-                // Note that this method may be called several times if several config options
-                // are changed. The effect is harmless except that we create and discard more
-                // objects than we need to.
-                LOG.debug("Config changed callback invoked with event {} {} {} {}", new Object[] {
-                        event.getPropertyName(), event.getPropertyValue(), event.getType(),
-                        event.isBeforeUpdate()});
-                if (!event.isBeforeUpdate()) {
-                    limiterStatsLogger.getCounter("config_changed").inc();
-                    LOG.debug("Rebuilding limiter");
-                    limiter = build();
-                }
-            }
-        };
-        LOG.debug("Registering config changed callback");
-        dynConf.addConfigurationListener(listener);
-    }
-
-    public void initialize() {
-        this.limiter = build();
-    }
-
-    @Override
-    public void apply(Req request) throws OverCapacityException {
-        if (rateLimitDisabledFeature.isAvailable()) {
-            return;
-        }
-        limiter.apply(request);
-    }
-
-    @Override
-    public void close() {
-        boolean success = dynConf.removeConfigurationListener(listener);
-        LOG.debug("Deregistering config changed callback success={}", success);
-    }
-
-   /**
-    * Build the underlying limiter. Called when DynamicRequestLimiter detects config has changed.
-    * This may be called multiple times so the method should be cheap.
-    */
-    protected abstract RequestLimiter<Req> build();
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/RequestLimiterBuilder.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/RequestLimiterBuilder.java
deleted file mode 100644
index c1a37bb..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/RequestLimiterBuilder.java
+++ /dev/null
@@ -1,116 +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 com.twitter.distributedlog.service.stream.limiter;
-
-import static com.google.common.base.Preconditions.checkNotNull;
-
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-import com.twitter.distributedlog.limiter.ComposableRequestLimiter;
-import com.twitter.distributedlog.limiter.ComposableRequestLimiter.CostFunction;
-import com.twitter.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
-import com.twitter.distributedlog.limiter.GuavaRateLimiter;
-import com.twitter.distributedlog.limiter.RateLimiter;
-import com.twitter.distributedlog.limiter.RequestLimiter;
-import com.twitter.distributedlog.service.stream.StreamOp;
-import com.twitter.distributedlog.service.stream.WriteOpWithPayload;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Request limiter builder.
- */
-public class RequestLimiterBuilder {
-    private OverlimitFunction<StreamOp> overlimitFunction = NOP_OVERLIMIT_FUNCTION;
-    private RateLimiter limiter;
-    private CostFunction<StreamOp> costFunction;
-    private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
-
-    /**
-     * Function to calculate the `RPS` (Request per second) cost of a given stream operation.
-     */
-    public static final CostFunction<StreamOp> RPS_COST_FUNCTION = new CostFunction<StreamOp>() {
-        @Override
-        public int apply(StreamOp op) {
-            if (op instanceof WriteOpWithPayload) {
-                return 1;
-            } else {
-                return 0;
-            }
-        }
-    };
-
-    /**
-     * Function to calculate the `BPS` (Bytes per second) cost of a given stream operation.
-     */
-    public static final CostFunction<StreamOp> BPS_COST_FUNCTION = new CostFunction<StreamOp>() {
-        @Override
-        public int apply(StreamOp op) {
-            if (op instanceof WriteOpWithPayload) {
-                WriteOpWithPayload writeOp = (WriteOpWithPayload) op;
-                return (int) Math.min(writeOp.getPayloadSize(), Integer.MAX_VALUE);
-            } else {
-                return 0;
-            }
-        }
-    };
-
-    /**
-     * Function to check if a stream operation will cause {@link OverCapacityException}.
-     */
-    public static final OverlimitFunction<StreamOp> NOP_OVERLIMIT_FUNCTION = new OverlimitFunction<StreamOp>() {
-        @Override
-        public void apply(StreamOp op) throws OverCapacityException {
-            return;
-        }
-    };
-
-    public RequestLimiterBuilder limit(int limit) {
-        this.limiter = GuavaRateLimiter.of(limit);
-        return this;
-    }
-
-    public RequestLimiterBuilder overlimit(OverlimitFunction<StreamOp> overlimitFunction) {
-        this.overlimitFunction = overlimitFunction;
-        return this;
-    }
-
-    public RequestLimiterBuilder cost(CostFunction<StreamOp> costFunction) {
-        this.costFunction = costFunction;
-        return this;
-    }
-
-    public RequestLimiterBuilder statsLogger(StatsLogger statsLogger) {
-        this.statsLogger = statsLogger;
-        return this;
-    }
-
-    public static RequestLimiterBuilder newRpsLimiterBuilder() {
-        return new RequestLimiterBuilder().cost(RPS_COST_FUNCTION);
-    }
-
-    public static RequestLimiterBuilder newBpsLimiterBuilder() {
-        return new RequestLimiterBuilder().cost(BPS_COST_FUNCTION);
-    }
-
-    public RequestLimiter<StreamOp> build() {
-        checkNotNull(limiter);
-        checkNotNull(overlimitFunction);
-        checkNotNull(costFunction);
-        return new ComposableRequestLimiter(limiter, overlimitFunction, costFunction, statsLogger);
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/ServiceRequestLimiter.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/ServiceRequestLimiter.java
deleted file mode 100644
index a3e1efb..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/ServiceRequestLimiter.java
+++ /dev/null
@@ -1,103 +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 com.twitter.distributedlog.service.stream.limiter;
-
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-import com.twitter.distributedlog.limiter.ChainedRequestLimiter;
-import com.twitter.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
-import com.twitter.distributedlog.limiter.RequestLimiter;
-import com.twitter.distributedlog.rate.MovingAverageRate;
-import com.twitter.distributedlog.service.stream.StreamManager;
-import com.twitter.distributedlog.service.stream.StreamOp;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * Request limiter for the service instance (global request limiter).
- */
-public class ServiceRequestLimiter extends DynamicRequestLimiter<StreamOp> {
-    private final StatsLogger limiterStatLogger;
-    private final MovingAverageRate serviceRps;
-    private final MovingAverageRate serviceBps;
-    private final StreamManager streamManager;
-
-    public ServiceRequestLimiter(DynamicDistributedLogConfiguration dynConf,
-                                 StatsLogger statsLogger,
-                                 MovingAverageRate serviceRps,
-                                 MovingAverageRate serviceBps,
-                                 StreamManager streamManager,
-                                 Feature disabledFeature) {
-        super(dynConf, statsLogger, disabledFeature);
-        this.limiterStatLogger = statsLogger;
-        this.streamManager = streamManager;
-        this.serviceRps = serviceRps;
-        this.serviceBps = serviceBps;
-        this.limiter = build();
-    }
-
-    @Override
-    public RequestLimiter<StreamOp> build() {
-        int rpsStreamAcquireLimit = dynConf.getRpsStreamAcquireServiceLimit();
-        int rpsSoftServiceLimit = dynConf.getRpsSoftServiceLimit();
-        int rpsHardServiceLimit = dynConf.getRpsHardServiceLimit();
-        int bpsStreamAcquireLimit = dynConf.getBpsStreamAcquireServiceLimit();
-        int bpsSoftServiceLimit = dynConf.getBpsSoftServiceLimit();
-        int bpsHardServiceLimit = dynConf.getBpsHardServiceLimit();
-
-        RequestLimiterBuilder rpsHardLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("rps_hard_limit"))
-            .limit(rpsHardServiceLimit)
-            .overlimit(new OverlimitFunction<StreamOp>() {
-                @Override
-                public void apply(StreamOp request) throws OverCapacityException {
-                    throw new OverCapacityException("Being rate limited: RPS limit exceeded for the service instance");
-                }
-            });
-
-        RequestLimiterBuilder rpsSoftLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("rps_soft_limit"))
-            .limit(rpsSoftServiceLimit);
-
-        RequestLimiterBuilder bpsHardLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("bps_hard_limit"))
-            .limit(bpsHardServiceLimit)
-            .overlimit(new OverlimitFunction<StreamOp>() {
-                @Override
-                public void apply(StreamOp request) throws OverCapacityException {
-                    throw new OverCapacityException("Being rate limited: BPS limit exceeded for the service instance");
-                }
-            });
-
-        RequestLimiterBuilder bpsSoftLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("bps_soft_limit"))
-            .limit(bpsSoftServiceLimit);
-
-        ChainedRequestLimiter.Builder<StreamOp> builder = new ChainedRequestLimiter.Builder<StreamOp>();
-        builder.addLimiter(new StreamAcquireLimiter(
-            streamManager, serviceRps, rpsStreamAcquireLimit, limiterStatLogger.scope("rps_acquire")));
-        builder.addLimiter(new StreamAcquireLimiter(
-            streamManager, serviceBps, bpsStreamAcquireLimit, limiterStatLogger.scope("bps_acquire")));
-        builder.addLimiter(bpsHardLimiterBuilder.build());
-        builder.addLimiter(bpsSoftLimiterBuilder.build());
-        builder.addLimiter(rpsHardLimiterBuilder.build());
-        builder.addLimiter(rpsSoftLimiterBuilder.build());
-        builder.statsLogger(limiterStatLogger);
-        return builder.build();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/StreamAcquireLimiter.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/StreamAcquireLimiter.java
deleted file mode 100644
index 5015751..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/StreamAcquireLimiter.java
+++ /dev/null
@@ -1,56 +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 com.twitter.distributedlog.service.stream.limiter;
-
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-import com.twitter.distributedlog.exceptions.TooManyStreamsException;
-import com.twitter.distributedlog.limiter.RequestLimiter;
-import com.twitter.distributedlog.rate.MovingAverageRate;
-import com.twitter.distributedlog.service.stream.StreamManager;
-import com.twitter.distributedlog.service.stream.StreamOp;
-import org.apache.bookkeeper.stats.Counter;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * A special limiter on limiting acquiring new streams.
- */
-public class StreamAcquireLimiter implements RequestLimiter<StreamOp> {
-    private final StreamManager streamManager;
-    private final MovingAverageRate serviceRps;
-    private final double serviceRpsLimit;
-    private final Counter overlimitCounter;
-
-    public StreamAcquireLimiter(StreamManager streamManager,
-                                MovingAverageRate serviceRps,
-                                double serviceRpsLimit,
-                                StatsLogger statsLogger) {
-        this.streamManager = streamManager;
-        this.serviceRps = serviceRps;
-        this.serviceRpsLimit = serviceRpsLimit;
-        this.overlimitCounter = statsLogger.getCounter("overlimit");
-    }
-
-    @Override
-    public void apply(StreamOp op) throws OverCapacityException {
-        String streamName = op.streamName();
-        if (serviceRpsLimit > -1 && serviceRps.get() > serviceRpsLimit && !streamManager.isAcquired(streamName)) {
-            overlimitCounter.inc();
-            throw new TooManyStreamsException("Request rate is too high to accept new stream " + streamName + ".");
-        }
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/StreamRequestLimiter.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/StreamRequestLimiter.java
deleted file mode 100644
index fa601d1..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/StreamRequestLimiter.java
+++ /dev/null
@@ -1,87 +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 com.twitter.distributedlog.service.stream.limiter;
-
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-import com.twitter.distributedlog.limiter.ChainedRequestLimiter;
-import com.twitter.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
-import com.twitter.distributedlog.limiter.RequestLimiter;
-import com.twitter.distributedlog.service.stream.StreamOp;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.stats.StatsLogger;
-
-/**
- * A dynamic request limiter on limiting stream operations.
- */
-public class StreamRequestLimiter extends DynamicRequestLimiter<StreamOp> {
-    private final DynamicDistributedLogConfiguration dynConf;
-    private final StatsLogger limiterStatLogger;
-    private final String streamName;
-
-    public StreamRequestLimiter(String streamName,
-                                DynamicDistributedLogConfiguration dynConf,
-                                StatsLogger statsLogger,
-                                Feature disabledFeature) {
-        super(dynConf, statsLogger, disabledFeature);
-        this.limiterStatLogger = statsLogger;
-        this.dynConf = dynConf;
-        this.streamName = streamName;
-        this.limiter = build();
-    }
-
-    @Override
-    public RequestLimiter<StreamOp> build() {
-
-        // RPS hard, soft limits
-        RequestLimiterBuilder rpsHardLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("rps_hard_limit"))
-            .limit(dynConf.getRpsHardWriteLimit())
-            .overlimit(new OverlimitFunction<StreamOp>() {
-                @Override
-                public void apply(StreamOp op) throws OverCapacityException {
-                    throw new OverCapacityException("Being rate limited: RPS limit exceeded for stream " + streamName);
-                }
-            });
-        RequestLimiterBuilder rpsSoftLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("rps_soft_limit"))
-            .limit(dynConf.getRpsSoftWriteLimit());
-
-        // BPS hard, soft limits
-        RequestLimiterBuilder bpsHardLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("bps_hard_limit"))
-            .limit(dynConf.getBpsHardWriteLimit())
-            .overlimit(new OverlimitFunction<StreamOp>() {
-                @Override
-                public void apply(StreamOp op) throws OverCapacityException {
-                    throw new OverCapacityException("Being rate limited: BPS limit exceeded for stream " + streamName);
-                }
-            });
-        RequestLimiterBuilder bpsSoftLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
-            .statsLogger(limiterStatLogger.scope("bps_soft_limit"))
-            .limit(dynConf.getBpsSoftWriteLimit());
-
-        ChainedRequestLimiter.Builder<StreamOp> builder = new ChainedRequestLimiter.Builder<StreamOp>();
-        builder.addLimiter(rpsSoftLimiterBuilder.build());
-        builder.addLimiter(rpsHardLimiterBuilder.build());
-        builder.addLimiter(bpsSoftLimiterBuilder.build());
-        builder.addLimiter(bpsHardLimiterBuilder.build());
-        builder.statsLogger(limiterStatLogger);
-        return builder.build();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/package-info.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/package-info.java
deleted file mode 100644
index 533c75a..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/limiter/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Request Rate Limiting.
- */
-package com.twitter.distributedlog.service.stream.limiter;
\ No newline at end of file
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/package-info.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/package-info.java
deleted file mode 100644
index 389acd9..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/stream/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Stream Related Operations.
- */
-package com.twitter.distributedlog.service.stream;
\ No newline at end of file
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/CacheableStreamPartitionConverter.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/CacheableStreamPartitionConverter.java
deleted file mode 100644
index 9afd234..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/CacheableStreamPartitionConverter.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 com.twitter.distributedlog.service.streamset;
-
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-
-/**
- * A stream-to-partition converter that caches the mapping between stream and partitions.
- */
-public abstract class CacheableStreamPartitionConverter implements StreamPartitionConverter {
-
-    private final ConcurrentMap<String, Partition> partitions;
-
-    protected CacheableStreamPartitionConverter() {
-        this.partitions = new ConcurrentHashMap<String, Partition>();
-    }
-
-    @Override
-    public Partition convert(String streamName) {
-        Partition p = partitions.get(streamName);
-        if (null != p) {
-            return p;
-        }
-        // not found
-        Partition newPartition = newPartition(streamName);
-        Partition oldPartition = partitions.putIfAbsent(streamName, newPartition);
-        if (null == oldPartition) {
-            return newPartition;
-        } else {
-            return oldPartition;
-        }
-    }
-
-    /**
-     * Create the partition from <code>streamName</code>.
-     *
-     * @param streamName
-     *          stream name
-     * @return partition id of the stream
-     */
-    protected abstract Partition newPartition(String streamName);
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/DelimiterStreamPartitionConverter.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/DelimiterStreamPartitionConverter.java
deleted file mode 100644
index d69a393..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/DelimiterStreamPartitionConverter.java
+++ /dev/null
@@ -1,50 +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 com.twitter.distributedlog.service.streamset;
-
-import org.apache.commons.lang3.StringUtils;
-
-/**
- * Stream Partition Converter that converts the stream name into a stream-to-partition mapping by delimiter.
- */
-public class DelimiterStreamPartitionConverter extends CacheableStreamPartitionConverter {
-
-    private final String delimiter;
-
-    public DelimiterStreamPartitionConverter() {
-        this("_");
-    }
-
-    public DelimiterStreamPartitionConverter(String delimiter) {
-        this.delimiter = delimiter;
-    }
-
-    @Override
-    protected Partition newPartition(String streamName) {
-        String[] parts = StringUtils.split(streamName, delimiter);
-        if (null != parts && parts.length == 2) {
-            try {
-                int partition = Integer.parseInt(parts[1]);
-                return new Partition(parts[0], partition);
-            } catch (NumberFormatException nfe) {
-                // ignore the exception
-            }
-        }
-        return new Partition(streamName, 0);
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/IdentityStreamPartitionConverter.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/IdentityStreamPartitionConverter.java
deleted file mode 100644
index 3b822b5..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/IdentityStreamPartitionConverter.java
+++ /dev/null
@@ -1,28 +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 com.twitter.distributedlog.service.streamset;
-
-/**
- * Map stream name to partition of the same name.
- */
-public class IdentityStreamPartitionConverter extends CacheableStreamPartitionConverter {
-    @Override
-    protected Partition newPartition(String streamName) {
-        return new Partition(streamName, 0);
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/Partition.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/Partition.java
deleted file mode 100644
index 770c631..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/Partition.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 com.twitter.distributedlog.service.streamset;
-
-import com.google.common.base.Objects;
-
-/**
- * `Partition` defines the relationship between a `virtual` stream and a
- * physical DL stream.
- *
- * <p>A `virtual` stream could be partitioned into multiple partitions
- * and each partition is effectively a DL stream.
- */
-public class Partition {
-
-    // Name of its parent stream.
-    private final String stream;
-
-    // Unique id of the partition within the stream.
-    // It can be just simply an index id.
-    public final int id;
-
-    public Partition(String stream, int id) {
-        this.stream = stream;
-        this.id = id;
-    }
-
-    /**
-     * Get the `virtual` stream name.
-     *
-     * @return the stream name.
-     */
-    public String getStream() {
-        return stream;
-    }
-
-    /**
-     * Get the partition id of this partition.
-     *
-     * @return partition id
-     */
-    public int getId() {
-        return id;
-    }
-
-    /**
-     * Get the 6 digit 0 padded id of this partition as a String.
-     * @return partition id
-     */
-    public String getPaddedId() {
-        return String.format("%06d", getId());
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        if (this == o) {
-            return true;
-        }
-        if (!(o instanceof Partition)) {
-            return false;
-        }
-        Partition partition = (Partition) o;
-
-        return id == partition.id && Objects.equal(stream, partition.stream);
-    }
-
-    @Override
-    public int hashCode() {
-        int result = stream.hashCode();
-        result = 31 * result + id;
-        return result;
-    }
-
-    @Override
-    public String toString() {
-        StringBuilder sb = new StringBuilder();
-        sb.append("Partition(")
-          .append(stream)
-          .append(", ")
-          .append(id)
-          .append(")");
-        return sb.toString();
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/PartitionMap.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/PartitionMap.java
deleted file mode 100644
index 1962e5f..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/PartitionMap.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 com.twitter.distributedlog.service.streamset;
-
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-/**
- * A mapping between a logical stream and a set of physical partitions.
- */
-public class PartitionMap {
-
-    private final Map<String, Set<Partition>> partitionMap;
-
-    public PartitionMap() {
-        partitionMap = new HashMap<String, Set<Partition>>();
-    }
-
-    public synchronized boolean addPartition(Partition partition, int maxPartitions) {
-        if (maxPartitions <= 0) {
-            return true;
-        }
-        Set<Partition> partitions = partitionMap.get(partition.getStream());
-        if (null == partitions) {
-            partitions = new HashSet<Partition>();
-            partitions.add(partition);
-            partitionMap.put(partition.getStream(), partitions);
-            return true;
-        }
-        if (partitions.contains(partition) || partitions.size() < maxPartitions) {
-            partitions.add(partition);
-            return true;
-        }
-        return false;
-    }
-
-    public synchronized boolean removePartition(Partition partition) {
-        Set<Partition> partitions = partitionMap.get(partition.getStream());
-        return null != partitions && partitions.remove(partition);
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/StreamPartitionConverter.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/StreamPartitionConverter.java
deleted file mode 100644
index 150643e..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/StreamPartitionConverter.java
+++ /dev/null
@@ -1,35 +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 com.twitter.distributedlog.service.streamset;
-
-/**
- * Map stream name to a partition.
- *
- * @see Partition
- */
-public interface StreamPartitionConverter {
-
-    /**
-     * Convert the stream name to partition.
-     *
-     * @param streamName
-     *          stream name
-     * @return partition
-     */
-    Partition convert(String streamName);
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/package-info.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/package-info.java
deleted file mode 100644
index 3888e40..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/streamset/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * StreamSet - A logical set of streams.
- */
-package com.twitter.distributedlog.service.streamset;
\ No newline at end of file
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/tools/ProxyTool.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/tools/ProxyTool.java
deleted file mode 100644
index 8ff2f26..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/tools/ProxyTool.java
+++ /dev/null
@@ -1,350 +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 com.twitter.distributedlog.service.tools;
-
-import com.google.common.util.concurrent.RateLimiter;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
-import com.twitter.distributedlog.client.serverset.DLZkServerSet;
-import com.twitter.distributedlog.service.ClientUtils;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.distributedlog.tools.Tool;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import java.net.InetSocketAddress;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import org.apache.commons.cli.CommandLine;
-import org.apache.commons.cli.Options;
-import org.apache.commons.cli.ParseException;
-import org.apache.commons.lang3.tuple.Pair;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Tools to interact with proxies.
- */
-public class ProxyTool extends Tool {
-
-    private static final Logger logger = LoggerFactory.getLogger(ProxyTool.class);
-
-    /**
-     * Abstract Cluster level command.
-     */
-    protected abstract static class ClusterCommand extends OptsCommand {
-
-        protected Options options = new Options();
-        protected URI uri;
-        protected final List<String> streams = new ArrayList<String>();
-
-        protected ClusterCommand(String name, String description) {
-            super(name, description);
-            options.addOption("u", "uri", true, "DistributedLog URI");
-            options.addOption("r", "prefix", true, "Prefix of stream name. E.g. 'QuantumLeapTest-'.");
-            options.addOption("e", "expression", true, "Expression to generate stream suffix. "
-                + "Currently we support range '0-9', list '1,2,3' and name '143'");
-        }
-
-        @Override
-        protected int runCmd(CommandLine commandLine) throws Exception {
-            try {
-                parseCommandLine(commandLine);
-            } catch (ParseException pe) {
-                System.err.println("ERROR: failed to parse commandline : '" + pe.getMessage() + "'");
-                printUsage();
-                return -1;
-            }
-
-            DLZkServerSet serverSet = DLZkServerSet.of(uri, 60000);
-            logger.info("Created serverset for {}", uri);
-            try {
-                DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
-                        .name("proxy_tool")
-                        .clientId(ClientId$.MODULE$.apply("proxy_tool"))
-                        .maxRedirects(2)
-                        .serverSet(serverSet.getServerSet())
-                        .clientBuilder(ClientBuilder.get()
-                            .connectionTimeout(Duration.fromSeconds(2))
-                            .tcpConnectTimeout(Duration.fromSeconds(2))
-                            .requestTimeout(Duration.fromSeconds(10))
-                            .hostConnectionLimit(1)
-                            .hostConnectionCoresize(1)
-                            .keepAlive(true)
-                            .failFast(false))
-                        .build();
-                try {
-                    return runCmd(client);
-                } finally {
-                    client.close();
-                }
-            } finally {
-                serverSet.close();
-            }
-        }
-
-        protected abstract int runCmd(DistributedLogClient client) throws Exception;
-
-        @Override
-        protected Options getOptions() {
-            return options;
-        }
-
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            if (!cmdline.hasOption("u")) {
-                throw new ParseException("No distributedlog uri provided.");
-            }
-            this.uri = URI.create(cmdline.getOptionValue("u"));
-
-            // get stream names
-            String streamPrefix = cmdline.hasOption("r") ? cmdline.getOptionValue("r") : "";
-            String streamExpression = null;
-            if (cmdline.hasOption("e")) {
-                streamExpression = cmdline.getOptionValue("e");
-            }
-            if (null == streamPrefix || null == streamExpression) {
-                throw new ParseException("Please specify stream prefix & expression.");
-            }
-            // parse the stream expression
-            if (streamExpression.contains("-")) {
-                // a range expression
-                String[] parts = streamExpression.split("-");
-                if (parts.length != 2) {
-                    throw new ParseException("Invalid stream index range : " + streamExpression);
-                }
-                try {
-                    int start = Integer.parseInt(parts[0]);
-                    int end = Integer.parseInt(parts[1]);
-                    if (start > end) {
-                        throw new ParseException("Invalid stream index range : " + streamExpression);
-                    }
-                    for (int i = start; i <= end; i++) {
-                        streams.add(streamPrefix + i);
-                    }
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid stream index range : " + streamExpression);
-                }
-            } else if (streamExpression.contains(",")) {
-                // a list expression
-                String[] parts = streamExpression.split(",");
-                try {
-                    for (String part : parts) {
-                        streams.add(streamPrefix + part);
-                    }
-                } catch (NumberFormatException nfe) {
-                    throw new ParseException("Invalid stream suffix list : " + streamExpression);
-                }
-            } else {
-                streams.add(streamPrefix + streamExpression);
-            }
-        }
-    }
-
-    /**
-     * Command to release ownership of a log stream.
-     */
-    static class ReleaseCommand extends ClusterCommand {
-
-        double rate = 100f;
-
-        ReleaseCommand() {
-            super("release", "Release Stream Ownerships");
-            options.addOption("t", "rate", true, "Rate to release streams");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (cmdline.hasOption("t")) {
-                rate = Double.parseDouble(cmdline.getOptionValue("t", "100"));
-            }
-        }
-
-        @Override
-        protected int runCmd(DistributedLogClient client) throws Exception {
-            RateLimiter rateLimiter = RateLimiter.create(rate);
-            for (String stream : streams) {
-                rateLimiter.acquire();
-                try {
-                    Await.result(client.release(stream));
-                    System.out.println("Release ownership of stream " + stream);
-                } catch (Exception e) {
-                    System.err.println("Failed to release ownership of stream " + stream);
-                    throw e;
-                }
-            }
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "release [options]";
-        }
-    }
-
-    /**
-     * Command to truncate a log stream.
-     */
-    static class TruncateCommand extends ClusterCommand {
-
-        DLSN dlsn = DLSN.InitialDLSN;
-
-        TruncateCommand() {
-            super("truncate", "Truncate streams until given dlsn.");
-            options.addOption("d", "dlsn", true, "DLSN to truncate until");
-        }
-
-        @Override
-        protected int runCmd(DistributedLogClient client) throws Exception {
-            System.out.println("Truncating streams : " + streams);
-            for (String stream : streams) {
-                boolean success = Await.result(client.truncate(stream, dlsn));
-                System.out.println("Truncate " + stream + " to " + dlsn + " : " + success);
-            }
-            return 0;
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("d")) {
-                throw new ParseException("No DLSN provided");
-            }
-            String[] dlsnStrs = cmdline.getOptionValue("d").split(",");
-            if (dlsnStrs.length != 3) {
-                throw new ParseException("Invalid DLSN : " + cmdline.getOptionValue("d"));
-            }
-            dlsn = new DLSN(Long.parseLong(dlsnStrs[0]), Long.parseLong(dlsnStrs[1]), Long.parseLong(dlsnStrs[2]));
-        }
-
-        @Override
-        protected String getUsage() {
-            return "truncate [options]";
-        }
-    }
-
-    /**
-     * Abstract command to operate on a single proxy server.
-     */
-    protected abstract static class ProxyCommand extends OptsCommand {
-
-        protected Options options = new Options();
-        protected InetSocketAddress address;
-
-        protected ProxyCommand(String name, String description) {
-            super(name, description);
-            options.addOption("H", "host", true, "Single Proxy Address");
-        }
-
-        @Override
-        protected Options getOptions() {
-            return options;
-        }
-
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            if (!cmdline.hasOption("H")) {
-                throw new ParseException("No proxy address provided");
-            }
-            address = DLSocketAddress.parseSocketAddress(cmdline.getOptionValue("H"));
-        }
-
-        @Override
-        protected int runCmd(CommandLine commandLine) throws Exception {
-            try {
-                parseCommandLine(commandLine);
-            } catch (ParseException pe) {
-                System.err.println("ERROR: failed to parse commandline : '" + pe.getMessage() + "'");
-                printUsage();
-                return -1;
-            }
-
-            DistributedLogClientBuilder clientBuilder = DistributedLogClientBuilder.newBuilder()
-                    .name("proxy_tool")
-                    .clientId(ClientId$.MODULE$.apply("proxy_tool"))
-                    .maxRedirects(2)
-                    .host(address)
-                    .clientBuilder(ClientBuilder.get()
-                            .connectionTimeout(Duration.fromSeconds(2))
-                            .tcpConnectTimeout(Duration.fromSeconds(2))
-                            .requestTimeout(Duration.fromSeconds(10))
-                            .hostConnectionLimit(1)
-                            .hostConnectionCoresize(1)
-                            .keepAlive(true)
-                            .failFast(false));
-            Pair<DistributedLogClient, MonitorServiceClient> clientPair =
-                    ClientUtils.buildClient(clientBuilder);
-            try {
-                return runCmd(clientPair);
-            } finally {
-                clientPair.getLeft().close();
-            }
-        }
-
-        protected abstract int runCmd(Pair<DistributedLogClient, MonitorServiceClient> client) throws Exception;
-    }
-
-    /**
-     * Command to enable/disable accepting new streams.
-     */
-    static class AcceptNewStreamCommand extends ProxyCommand {
-
-        boolean enabled = false;
-
-        AcceptNewStreamCommand() {
-            super("accept-new-stream", "Enable/Disable accepting new streams for one proxy");
-            options.addOption("e", "enabled", true, "Enable/Disable accepting new streams");
-        }
-
-        @Override
-        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
-            super.parseCommandLine(cmdline);
-            if (!cmdline.hasOption("e")) {
-                throw new ParseException("No action 'enable/disable' provided");
-            }
-            enabled = Boolean.parseBoolean(cmdline.getOptionValue("e"));
-        }
-
-        @Override
-        protected int runCmd(Pair<DistributedLogClient, MonitorServiceClient> client)
-                throws Exception {
-            Await.result(client.getRight().setAcceptNewStream(enabled));
-            return 0;
-        }
-
-        @Override
-        protected String getUsage() {
-            return "accept-new-stream [options]";
-        }
-    }
-
-    public ProxyTool() {
-        super();
-        addCommand(new ReleaseCommand());
-        addCommand(new TruncateCommand());
-        addCommand(new AcceptNewStreamCommand());
-    }
-
-    @Override
-    protected String getName() {
-        return "proxy_tool";
-    }
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/tools/package-info.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/tools/package-info.java
deleted file mode 100644
index 1e32fd3..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/tools/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Service related tools.
- */
-package com.twitter.distributedlog.service.tools;
\ No newline at end of file
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/utils/ServerUtils.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/utils/ServerUtils.java
deleted file mode 100644
index af65b11..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/utils/ServerUtils.java
+++ /dev/null
@@ -1,49 +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 com.twitter.distributedlog.service.utils;
-
-import java.io.IOException;
-import java.net.InetAddress;
-
-/**
- * Utils that used by servers.
- */
-public class ServerUtils {
-
-  /**
-   * Retrieve the ledger allocator pool name.
-   *
-   * @param serverRegionId region id that that server is running
-   * @param shardId shard id of the server
-   * @param useHostname whether to use hostname as the ledger allocator pool name
-   * @return ledger allocator pool name
-   * @throws IOException
-   */
-    public static String getLedgerAllocatorPoolName(int serverRegionId,
-                                                    int shardId,
-                                                    boolean useHostname)
-        throws IOException {
-        if (useHostname) {
-            return String.format("allocator_%04d_%s", serverRegionId,
-                InetAddress.getLocalHost().getHostAddress());
-        } else {
-            return String.format("allocator_%04d_%010d", serverRegionId, shardId);
-        }
-    }
-
-}
diff --git a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/utils/package-info.java b/distributedlog-service/src/main/java/com/twitter/distributedlog/service/utils/package-info.java
deleted file mode 100644
index e6dcec6..0000000
--- a/distributedlog-service/src/main/java/com/twitter/distributedlog/service/utils/package-info.java
+++ /dev/null
@@ -1,21 +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.
- */
-/**
- * Utilities used by proxy servers.
- */
-package com.twitter.distributedlog.service.utils;
\ No newline at end of file
diff --git a/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/package-info.java b/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/package-info.java
index d00b64d..5bdb3ce 100644
--- a/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/package-info.java
+++ b/distributedlog-service/src/main/java/org/apache/bookkeeper/stats/package-info.java
@@ -18,4 +18,4 @@
 /**
  * Extension of {@link org.apache.bookkeeper.stats.CodahaleMetricsProvider}.
  */
-package org.apache.bookkeeper.stats;
\ No newline at end of file
+package org.apache.bookkeeper.stats;
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/ClientUtils.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/ClientUtils.java
new file mode 100644
index 0000000..96bc338
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/ClientUtils.java
@@ -0,0 +1,33 @@
+/**
+ * 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.service;
+
+import org.apache.distributedlog.client.DistributedLogClientImpl;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.commons.lang3.tuple.Pair;
+
+/**
+ * DistributedLog Client Related Utils.
+ */
+public class ClientUtils {
+
+    public static Pair<DistributedLogClient, MonitorServiceClient> buildClient(DistributedLogClientBuilder builder) {
+        DistributedLogClientImpl clientImpl = builder.buildClient();
+        return Pair.of((DistributedLogClient) clientImpl, (MonitorServiceClient) clientImpl);
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogCluster.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogCluster.java
new file mode 100644
index 0000000..aadd71e
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogCluster.java
@@ -0,0 +1,352 @@
+/**
+ * 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.service;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.LocalDLMEmulator;
+import org.apache.distributedlog.client.routing.SingleHostRoutingService;
+import org.apache.distributedlog.impl.metadata.BKDLConfig;
+import org.apache.distributedlog.metadata.DLMetadata;
+import org.apache.distributedlog.service.placement.EqualLoadAppraiser;
+import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
+import com.twitter.finagle.builder.Server;
+import java.io.File;
+import java.net.BindException;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.conf.ServerConfiguration;
+import org.apache.bookkeeper.shims.zk.ZooKeeperServerShim;
+import org.apache.bookkeeper.stats.NullStatsProvider;
+import org.apache.bookkeeper.util.IOUtils;
+import org.apache.bookkeeper.util.LocalBookKeeper;
+import org.apache.commons.io.FileUtils;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * DistributedLog Cluster is an emulator to run distributedlog components.
+ */
+public class DistributedLogCluster {
+
+    private static final Logger LOG = LoggerFactory.getLogger(DistributedLogCluster.class);
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    /**
+     * Builder to build distributedlog cluster.
+     */
+    public static class Builder {
+
+        int numBookies = 3;
+        boolean shouldStartZK = true;
+        String zkHost = "127.0.0.1";
+        int zkPort = 0;
+        boolean shouldStartProxy = true;
+        int proxyPort = 7000;
+        boolean thriftmux = false;
+        DistributedLogConfiguration dlConf = new DistributedLogConfiguration()
+                .setLockTimeout(10)
+                .setOutputBufferSize(0)
+                .setImmediateFlushEnabled(true);
+        ServerConfiguration bkConf = new ServerConfiguration();
+
+        private Builder() {}
+
+        /**
+         * How many bookies to run. By default is 3.
+         *
+         * @return builder
+         */
+        public Builder numBookies(int numBookies) {
+            this.numBookies = numBookies;
+            return this;
+        }
+
+        /**
+         * Whether to start zookeeper? By default is true.
+         *
+         * @param startZK
+         *          flag to start zookeeper?
+         * @return builder
+         */
+        public Builder shouldStartZK(boolean startZK) {
+            this.shouldStartZK = startZK;
+            return this;
+        }
+
+        /**
+         * ZooKeeper server to run. By default it runs locally on '127.0.0.1'.
+         *
+         * @param zkServers
+         *          zk servers
+         * @return builder
+         */
+        public Builder zkServers(String zkServers) {
+            this.zkHost = zkServers;
+            return this;
+        }
+
+        /**
+         * ZooKeeper server port to listen on. By default it listens on 2181.
+         *
+         * @param zkPort
+         *          zookeeper server port.
+         * @return builder.
+         */
+        public Builder zkPort(int zkPort) {
+            this.zkPort = zkPort;
+            return this;
+        }
+
+        /**
+         * Whether to start proxy or not. By default is true.
+         *
+         * @param startProxy
+         *          whether to start proxy or not.
+         * @return builder
+         */
+        public Builder shouldStartProxy(boolean startProxy) {
+            this.shouldStartProxy = startProxy;
+            return this;
+        }
+
+        /**
+         * Port that proxy server to listen on. By default is 7000.
+         *
+         * @param proxyPort
+         *          port that proxy server to listen on.
+         * @return builder
+         */
+        public Builder proxyPort(int proxyPort) {
+            this.proxyPort = proxyPort;
+            return this;
+        }
+
+        /**
+         * Set the distributedlog configuration.
+         *
+         * @param dlConf
+         *          distributedlog configuration
+         * @return builder
+         */
+        public Builder dlConf(DistributedLogConfiguration dlConf) {
+            this.dlConf = dlConf;
+            return this;
+        }
+
+        /**
+         * Set the Bookkeeper server configuration.
+         *
+         * @param bkConf
+         *          bookkeeper server configuration
+         * @return builder
+         */
+        public Builder bkConf(ServerConfiguration bkConf) {
+            this.bkConf = bkConf;
+            return this;
+        }
+
+        /**
+         * Enable thriftmux for the dl server.
+         *
+         * @param enabled flag to enable thriftmux
+         * @return builder
+         */
+        public Builder thriftmux(boolean enabled) {
+            this.thriftmux = enabled;
+            return this;
+        }
+
+        public DistributedLogCluster build() throws Exception {
+            // build the cluster
+            return new DistributedLogCluster(
+                dlConf,
+                bkConf,
+                numBookies,
+                shouldStartZK,
+                zkHost,
+                zkPort,
+                shouldStartProxy,
+                proxyPort,
+                thriftmux);
+        }
+    }
+
+    /**
+     * Run a distributedlog proxy server.
+     */
+    public static class DLServer {
+
+        static final int MAX_RETRIES = 20;
+        static final int MIN_PORT = 1025;
+        static final int MAX_PORT = 65535;
+
+        int proxyPort;
+
+        public final InetSocketAddress address;
+        public final Pair<DistributedLogServiceImpl, Server> dlServer;
+        private final SingleHostRoutingService routingService = SingleHostRoutingService.of(null);
+
+        protected DLServer(DistributedLogConfiguration dlConf,
+                           URI uri,
+                           int basePort,
+                           boolean thriftmux) throws Exception {
+            proxyPort = basePort;
+
+            boolean success = false;
+            int retries = 0;
+            Pair<DistributedLogServiceImpl, Server> serverPair = null;
+            while (!success) {
+                try {
+                    org.apache.distributedlog.service.config.ServerConfiguration serverConf =
+                            new org.apache.distributedlog.service.config.ServerConfiguration();
+                    serverConf.loadConf(dlConf);
+                    serverConf.setServerShardId(proxyPort);
+                    serverPair = DistributedLogServer.runServer(
+                            serverConf,
+                            dlConf,
+                            uri,
+                            new IdentityStreamPartitionConverter(),
+                            routingService,
+                            new NullStatsProvider(),
+                            proxyPort,
+                            thriftmux,
+                            new EqualLoadAppraiser());
+                    routingService.setAddress(DLSocketAddress.getSocketAddress(proxyPort));
+                    routingService.startService();
+                    serverPair.getLeft().startPlacementPolicy();
+                    success = true;
+                } catch (BindException be) {
+                    retries++;
+                    if (retries > MAX_RETRIES) {
+                        throw be;
+                    }
+                    proxyPort++;
+                    if (proxyPort > MAX_PORT) {
+                        proxyPort = MIN_PORT;
+                    }
+                }
+            }
+
+            LOG.info("Running DL on port {}", proxyPort);
+
+            dlServer = serverPair;
+            address = DLSocketAddress.getSocketAddress(proxyPort);
+        }
+
+        public InetSocketAddress getAddress() {
+            return address;
+        }
+
+        public void shutdown() {
+            DistributedLogServer.closeServer(dlServer, 0, TimeUnit.MILLISECONDS);
+            routingService.stopService();
+        }
+    }
+
+    private final DistributedLogConfiguration dlConf;
+    private final ZooKeeperServerShim zks;
+    private final LocalDLMEmulator dlmEmulator;
+    private DLServer dlServer;
+    private final boolean shouldStartProxy;
+    private final int proxyPort;
+    private final boolean thriftmux;
+    private final List<File> tmpDirs = new ArrayList<File>();
+
+    private DistributedLogCluster(DistributedLogConfiguration dlConf,
+                                  ServerConfiguration bkConf,
+                                  int numBookies,
+                                  boolean shouldStartZK,
+                                  String zkServers,
+                                  int zkPort,
+                                  boolean shouldStartProxy,
+                                  int proxyPort,
+                                  boolean thriftmux) throws Exception {
+        this.dlConf = dlConf;
+        if (shouldStartZK) {
+            File zkTmpDir = IOUtils.createTempDir("zookeeper", "distrlog");
+            tmpDirs.add(zkTmpDir);
+            if (0 == zkPort) {
+                Pair<ZooKeeperServerShim, Integer> serverAndPort = LocalDLMEmulator.runZookeeperOnAnyPort(zkTmpDir);
+                this.zks = serverAndPort.getLeft();
+                zkPort = serverAndPort.getRight();
+            } else {
+                this.zks = LocalBookKeeper.runZookeeper(1000, zkPort, zkTmpDir);
+            }
+        } else {
+            this.zks = null;
+        }
+        this.dlmEmulator = LocalDLMEmulator.newBuilder()
+                .numBookies(numBookies)
+                .zkHost(zkServers)
+                .zkPort(zkPort)
+                .serverConf(bkConf)
+                .shouldStartZK(false)
+                .build();
+        this.shouldStartProxy = shouldStartProxy;
+        this.proxyPort = proxyPort;
+        this.thriftmux = thriftmux;
+    }
+
+    public void start() throws Exception {
+        this.dlmEmulator.start();
+        BKDLConfig bkdlConfig = new BKDLConfig(this.dlmEmulator.getZkServers(), "/ledgers").setACLRootPath(".acl");
+        DLMetadata.create(bkdlConfig).update(this.dlmEmulator.getUri());
+        if (shouldStartProxy) {
+            this.dlServer = new DLServer(
+                    dlConf,
+                    this.dlmEmulator.getUri(),
+                    proxyPort,
+                    thriftmux);
+        } else {
+            this.dlServer = null;
+        }
+    }
+
+    public void stop() throws Exception {
+        if (null != dlServer) {
+            this.dlServer.shutdown();
+        }
+        this.dlmEmulator.teardown();
+        if (null != this.zks) {
+            this.zks.stop();
+        }
+        for (File dir : tmpDirs) {
+            FileUtils.deleteDirectory(dir);
+        }
+    }
+
+    public URI getUri() {
+        return this.dlmEmulator.getUri();
+    }
+
+    public String getZkServers() {
+        return this.dlmEmulator.getZkServers();
+    }
+
+    public String getProxyFinagleStr() {
+        return "inet!" + (dlServer == null ? "127.0.0.1:" + proxyPort : dlServer.getAddress().toString());
+    }
+
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java
new file mode 100644
index 0000000..7887445
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServer.java
@@ -0,0 +1,460 @@
+/**
+ * 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.service;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.config.DynamicConfigurationFactory;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.service.announcer.Announcer;
+import org.apache.distributedlog.service.announcer.NOPAnnouncer;
+import org.apache.distributedlog.service.announcer.ServerSetAnnouncer;
+import org.apache.distributedlog.service.config.DefaultStreamConfigProvider;
+import org.apache.distributedlog.service.config.NullStreamConfigProvider;
+import org.apache.distributedlog.service.config.ServerConfiguration;
+import org.apache.distributedlog.service.config.ServiceStreamConfigProvider;
+import org.apache.distributedlog.service.config.StreamConfigProvider;
+import org.apache.distributedlog.service.placement.EqualLoadAppraiser;
+import org.apache.distributedlog.service.placement.LoadAppraiser;
+import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
+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 com.twitter.finagle.Stack;
+import com.twitter.finagle.ThriftMuxServer$;
+import com.twitter.finagle.builder.Server;
+import com.twitter.finagle.builder.ServerBuilder;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.finagle.thrift.ClientIdRequiredFilter;
+import com.twitter.finagle.thrift.ThriftServerFramedCodec;
+import com.twitter.finagle.transport.Transport;
+import com.twitter.util.Duration;
+import java.io.File;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.apache.thrift.protocol.TBinaryProtocol;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+import scala.Tuple2;
+
+/**
+ * Running the distributedlog proxy server.
+ */
+public class DistributedLogServer {
+
+    private static final Logger logger = LoggerFactory.getLogger(DistributedLogServer.class);
+    private static final String DEFAULT_LOAD_APPRIASER = EqualLoadAppraiser.class.getCanonicalName();
+
+    private DistributedLogServiceImpl dlService = null;
+    private Server server = null;
+    private RoutingService routingService;
+    private StatsProvider statsProvider;
+    private Announcer announcer = null;
+    private ScheduledExecutorService configExecutorService;
+    private long gracefulShutdownMs = 0L;
+
+    private final StatsReceiver statsReceiver;
+    private final CountDownLatch keepAliveLatch = new CountDownLatch(1);
+    private final Optional<String> uri;
+    private final Optional<String> conf;
+    private final Optional<String> streamConf;
+    private final Optional<Integer> port;
+    private final Optional<Integer> statsPort;
+    private final Optional<Integer> shardId;
+    private final Optional<Boolean> announceServerSet;
+    private final Optional<String> loadAppraiserClassStr;
+    private final Optional<Boolean> thriftmux;
+
+    DistributedLogServer(Optional<String> uri,
+                         Optional<String> conf,
+                         Optional<String> streamConf,
+                         Optional<Integer> port,
+                         Optional<Integer> statsPort,
+                         Optional<Integer> shardId,
+                         Optional<Boolean> announceServerSet,
+                         Optional<String> loadAppraiserClass,
+                         Optional<Boolean> thriftmux,
+                         RoutingService routingService,
+                         StatsReceiver statsReceiver,
+                         StatsProvider statsProvider) {
+        this.uri = uri;
+        this.conf = conf;
+        this.streamConf = streamConf;
+        this.port = port;
+        this.statsPort = statsPort;
+        this.shardId = shardId;
+        this.announceServerSet = announceServerSet;
+        this.thriftmux = thriftmux;
+        this.routingService = routingService;
+        this.statsReceiver = statsReceiver;
+        this.statsProvider = statsProvider;
+        this.loadAppraiserClassStr = loadAppraiserClass;
+    }
+
+    public void runServer()
+        throws ConfigurationException, IllegalArgumentException, IOException, ClassNotFoundException {
+        if (!uri.isPresent()) {
+            throw new IllegalArgumentException("No distributedlog uri provided.");
+        }
+        URI dlUri = URI.create(uri.get());
+        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
+        if (conf.isPresent()) {
+            String configFile = conf.get();
+            try {
+                dlConf.loadConf(new File(configFile).toURI().toURL());
+            } catch (ConfigurationException e) {
+                throw new IllegalArgumentException("Failed to load distributedlog configuration from "
+                    + configFile + ".");
+            } catch (MalformedURLException e) {
+                throw new IllegalArgumentException("Failed to load distributedlog configuration from malformed "
+                        + configFile + ".");
+            }
+        }
+
+        this.configExecutorService = Executors.newScheduledThreadPool(1,
+                new ThreadFactoryBuilder()
+                        .setNameFormat("DistributedLogService-Dyncfg-%d")
+                        .setDaemon(true)
+                        .build());
+
+        // server configuration and dynamic configuration
+        ServerConfiguration serverConf = new ServerConfiguration();
+        serverConf.loadConf(dlConf);
+
+        // overwrite the shard id if it is provided in the args
+        if (shardId.isPresent()) {
+            serverConf.setServerShardId(shardId.get());
+        }
+
+        serverConf.validate();
+
+        DynamicDistributedLogConfiguration dynDlConf = getServiceDynConf(dlConf);
+
+        logger.info("Starting stats provider : {}", statsProvider.getClass());
+        statsProvider.start(dlConf);
+
+        if (announceServerSet.isPresent() && announceServerSet.get()) {
+            announcer = new ServerSetAnnouncer(
+                    dlUri,
+                    port.or(0),
+                    statsPort.or(0),
+                    shardId.or(0));
+        } else {
+            announcer = new NOPAnnouncer();
+        }
+
+        // Build the stream partition converter
+        StreamPartitionConverter converter;
+        try {
+            converter = ReflectionUtils.newInstance(serverConf.getStreamPartitionConverterClass());
+        } catch (ConfigurationException e) {
+            logger.warn("Failed to load configured stream-to-partition converter. Fallback to use {}",
+                    IdentityStreamPartitionConverter.class.getName());
+            converter = new IdentityStreamPartitionConverter();
+        }
+        Class loadAppraiserClass = Class.forName(loadAppraiserClassStr.or(DEFAULT_LOAD_APPRIASER));
+        LoadAppraiser loadAppraiser = (LoadAppraiser) ReflectionUtils.newInstance(loadAppraiserClass);
+        logger.info("Supplied load appraiser class is " + loadAppraiserClassStr.get()
+            + " Instantiated " + loadAppraiser.getClass().getCanonicalName());
+
+        StreamConfigProvider streamConfProvider =
+                getStreamConfigProvider(dlConf, converter);
+
+        // pre-run
+        preRun(dlConf, serverConf);
+
+        Pair<DistributedLogServiceImpl, Server> serverPair = runServer(
+                serverConf,
+                dlConf,
+                dynDlConf,
+                dlUri,
+                converter,
+                routingService,
+                statsProvider,
+                port.or(0),
+                keepAliveLatch,
+                statsReceiver,
+                thriftmux.isPresent(),
+                streamConfProvider,
+                loadAppraiser);
+
+        this.dlService = serverPair.getLeft();
+        this.server = serverPair.getRight();
+
+        // announce the service
+        announcer.announce();
+        // start the routing service after announced
+        routingService.startService();
+        logger.info("Started the routing service.");
+        dlService.startPlacementPolicy();
+        logger.info("Started the placement policy.");
+    }
+
+    protected void preRun(DistributedLogConfiguration conf, ServerConfiguration serverConf) {
+        this.gracefulShutdownMs = serverConf.getGracefulShutdownPeriodMs();
+        if (!serverConf.isDurableWriteEnabled()) {
+            conf.setDurableWriteEnabled(false);
+        }
+    }
+
+    private DynamicDistributedLogConfiguration getServiceDynConf(DistributedLogConfiguration dlConf)
+        throws ConfigurationException {
+        Optional<DynamicDistributedLogConfiguration> dynConf = Optional.absent();
+        if (conf.isPresent()) {
+            DynamicConfigurationFactory configFactory = new DynamicConfigurationFactory(
+                    configExecutorService, dlConf.getDynamicConfigReloadIntervalSec(), TimeUnit.SECONDS);
+            dynConf = configFactory.getDynamicConfiguration(conf.get());
+        }
+        if (dynConf.isPresent()) {
+            return dynConf.get();
+        } else {
+            return ConfUtils.getConstDynConf(dlConf);
+        }
+    }
+
+    private StreamConfigProvider getStreamConfigProvider(DistributedLogConfiguration dlConf,
+                                                         StreamPartitionConverter partitionConverter)
+            throws ConfigurationException {
+        StreamConfigProvider streamConfProvider = new NullStreamConfigProvider();
+        if (streamConf.isPresent() && conf.isPresent()) {
+            String dynConfigPath = streamConf.get();
+            String defaultConfigFile = conf.get();
+            streamConfProvider = new ServiceStreamConfigProvider(
+                    dynConfigPath,
+                    defaultConfigFile,
+                    partitionConverter,
+                    configExecutorService,
+                    dlConf.getDynamicConfigReloadIntervalSec(),
+                    TimeUnit.SECONDS);
+        } else if (conf.isPresent()) {
+            String configFile = conf.get();
+            streamConfProvider = new DefaultStreamConfigProvider(configFile, configExecutorService,
+                    dlConf.getDynamicConfigReloadIntervalSec(), TimeUnit.SECONDS);
+        }
+        return streamConfProvider;
+    }
+
+    static Pair<DistributedLogServiceImpl, Server> runServer(
+            ServerConfiguration serverConf,
+            DistributedLogConfiguration dlConf,
+            URI dlUri,
+            StreamPartitionConverter converter,
+            RoutingService routingService,
+            StatsProvider provider,
+            int port,
+            boolean thriftmux,
+            LoadAppraiser loadAppraiser) throws IOException {
+
+        return runServer(serverConf,
+                dlConf,
+                ConfUtils.getConstDynConf(dlConf),
+                dlUri,
+                converter,
+                routingService,
+                provider,
+                port,
+                new CountDownLatch(0),
+                new NullStatsReceiver(),
+                thriftmux,
+                new NullStreamConfigProvider(),
+                loadAppraiser);
+    }
+
+    static Pair<DistributedLogServiceImpl, Server> runServer(
+            ServerConfiguration serverConf,
+            DistributedLogConfiguration dlConf,
+            DynamicDistributedLogConfiguration dynDlConf,
+            URI dlUri,
+            StreamPartitionConverter partitionConverter,
+            RoutingService routingService,
+            StatsProvider provider,
+            int port,
+            CountDownLatch keepAliveLatch,
+            StatsReceiver statsReceiver,
+            boolean thriftmux,
+            StreamConfigProvider streamConfProvider,
+            LoadAppraiser loadAppraiser) throws IOException {
+        logger.info("Running server @ uri {}.", dlUri);
+
+        boolean perStreamStatsEnabled = serverConf.isPerStreamStatEnabled();
+        StatsLogger perStreamStatsLogger;
+        if (perStreamStatsEnabled) {
+            perStreamStatsLogger = provider.getStatsLogger("stream");
+        } else {
+            perStreamStatsLogger = NullStatsLogger.INSTANCE;
+        }
+
+        // dl service
+        DistributedLogServiceImpl dlService = new DistributedLogServiceImpl(
+            serverConf,
+            dlConf,
+            dynDlConf,
+            streamConfProvider,
+            dlUri,
+            partitionConverter,
+            routingService,
+            provider.getStatsLogger(""),
+            perStreamStatsLogger,
+            keepAliveLatch,
+            loadAppraiser);
+
+        StatsReceiver serviceStatsReceiver = statsReceiver.scope("service");
+        StatsLogger serviceStatsLogger = provider.getStatsLogger("service");
+
+        ServerBuilder serverBuilder = ServerBuilder.get()
+                .name("DistributedLogServer")
+                .codec(ThriftServerFramedCodec.get())
+                .reportTo(statsReceiver)
+                .keepAlive(true)
+                .bindTo(new InetSocketAddress(port));
+
+        if (thriftmux) {
+            logger.info("Using thriftmux.");
+            Tuple2<Transport.Liveness, Stack.Param<Transport.Liveness>> livenessParam = new Transport.Liveness(
+                    Duration.Top(), Duration.Top(), Option.apply((Object) Boolean.valueOf(true))).mk();
+            serverBuilder = serverBuilder.stack(
+                ThriftMuxServer$.MODULE$.configured(livenessParam._1(), livenessParam._2()));
+        }
+
+        logger.info("DistributedLogServer running with the following configuration : \n{}", dlConf.getPropsAsString());
+
+        // starts dl server
+        Server server = ServerBuilder.safeBuild(
+                new ClientIdRequiredFilter<byte[], byte[]>(serviceStatsReceiver).andThen(
+                    new StatsFilter<byte[], byte[]>(serviceStatsLogger).andThen(
+                        new DistributedLogService.Service(dlService, new TBinaryProtocol.Factory()))),
+                serverBuilder);
+
+        logger.info("Started DistributedLog Server.");
+        return Pair.of(dlService, server);
+    }
+
+    static void closeServer(Pair<DistributedLogServiceImpl, Server> pair,
+                            long gracefulShutdownPeriod,
+                            TimeUnit timeUnit) {
+        if (null != pair.getLeft()) {
+            pair.getLeft().shutdown();
+            if (gracefulShutdownPeriod > 0) {
+                try {
+                    timeUnit.sleep(gracefulShutdownPeriod);
+                } catch (InterruptedException e) {
+                    logger.info("Interrupted on waiting service shutting down state propagated to all clients : ", e);
+                }
+            }
+        }
+        if (null != pair.getRight()) {
+            logger.info("Closing dl thrift server.");
+            pair.getRight().close();
+            logger.info("Closed dl thrift server.");
+        }
+    }
+
+    /**
+     * Close the server.
+     */
+    public void close() {
+        if (null != announcer) {
+            try {
+                announcer.unannounce();
+            } catch (IOException e) {
+                logger.warn("Error on unannouncing service : ", e);
+            }
+            announcer.close();
+        }
+        closeServer(Pair.of(dlService, server), gracefulShutdownMs, TimeUnit.MILLISECONDS);
+        routingService.stopService();
+        if (null != statsProvider) {
+            statsProvider.stop();
+        }
+        SchedulerUtils.shutdownScheduler(configExecutorService, 60, TimeUnit.SECONDS);
+        keepAliveLatch.countDown();
+    }
+
+    public void join() throws InterruptedException {
+        keepAliveLatch.await();
+    }
+
+    /**
+     * Running distributedlog server.
+     *
+     * @param uri distributedlog namespace
+     * @param conf distributedlog configuration file location
+     * @param streamConf per stream configuration dir location
+     * @param port listen port
+     * @param statsPort stats port
+     * @param shardId shard id
+     * @param announceServerSet whether to announce itself to server set
+     * @param thriftmux flag to enable thrift mux
+     * @param statsReceiver receiver to receive finagle stats
+     * @param statsProvider provider to receive dl stats
+     * @return distributedlog server
+     * @throws ConfigurationException
+     * @throws IllegalArgumentException
+     * @throws IOException
+     * @throws ClassNotFoundException
+     */
+    public static DistributedLogServer runServer(
+               Optional<String> uri,
+               Optional<String> conf,
+               Optional<String> streamConf,
+               Optional<Integer> port,
+               Optional<Integer> statsPort,
+               Optional<Integer> shardId,
+               Optional<Boolean> announceServerSet,
+               Optional<String> loadAppraiserClass,
+               Optional<Boolean> thriftmux,
+               RoutingService routingService,
+               StatsReceiver statsReceiver,
+               StatsProvider statsProvider)
+        throws ConfigurationException, IllegalArgumentException, IOException, ClassNotFoundException {
+
+        final DistributedLogServer server = new DistributedLogServer(
+                uri,
+                conf,
+                streamConf,
+                port,
+                statsPort,
+                shardId,
+                announceServerSet,
+                loadAppraiserClass,
+                thriftmux,
+                routingService,
+                statsReceiver,
+                statsProvider);
+
+        server.runServer();
+        return server;
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java
new file mode 100644
index 0000000..a1642f9
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServerApp.java
@@ -0,0 +1,187 @@
+/**
+ * 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.service;
+
+import static com.google.common.base.Preconditions.checkArgument;
+import static org.apache.distributedlog.util.CommandLineUtils.getOptionalBooleanArg;
+import static org.apache.distributedlog.util.CommandLineUtils.getOptionalIntegerArg;
+import static org.apache.distributedlog.util.CommandLineUtils.getOptionalStringArg;
+
+import com.google.common.base.Function;
+import com.google.common.base.Optional;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.client.routing.RoutingUtils;
+import org.apache.distributedlog.client.serverset.DLZkServerSet;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.io.File;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+import javax.annotation.Nullable;
+import org.apache.bookkeeper.stats.NullStatsProvider;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.cli.BasicParser;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.configuration.ConfigurationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The launcher of the distributedlog proxy server.
+ */
+public class DistributedLogServerApp {
+
+    private static final Logger logger = LoggerFactory.getLogger(DistributedLogServerApp.class);
+
+    private static final String USAGE = "DistributedLogServerApp [-u <uri>] [-c <conf>]";
+    private final String[] args;
+    private final Options options = new Options();
+
+    private DistributedLogServerApp(String[] args) {
+        this.args = args;
+
+        // prepare options
+        options.addOption("u", "uri", true, "DistributedLog URI");
+        options.addOption("c", "conf", true, "DistributedLog Configuration File");
+        options.addOption("sc", "stream-conf", true, "Per Stream Configuration Directory");
+        options.addOption("p", "port", true, "DistributedLog Server Port");
+        options.addOption("sp", "stats-port", true, "DistributedLog Stats Port");
+        options.addOption("pd", "stats-provider", true, "DistributedLog Stats Provider");
+        options.addOption("si", "shard-id", true, "DistributedLog Shard ID");
+        options.addOption("a", "announce", false, "ServerSet Path to Announce");
+        options.addOption("la", "load-appraiser", true, "LoadAppraiser Implementation to Use");
+        options.addOption("mx", "thriftmux", false, "Is thriftmux enabled");
+    }
+
+    private void printUsage() {
+        HelpFormatter helpFormatter = new HelpFormatter();
+        helpFormatter.printHelp(USAGE, options);
+    }
+
+    private void run() {
+        try {
+            logger.info("Running distributedlog server : args = {}", Arrays.toString(args));
+            BasicParser parser = new BasicParser();
+            CommandLine cmdline = parser.parse(options, args);
+            runCmd(cmdline);
+        } catch (ParseException pe) {
+            logger.error("Argument error : {}", pe.getMessage());
+            printUsage();
+            Runtime.getRuntime().exit(-1);
+        } catch (IllegalArgumentException iae) {
+            logger.error("Argument error : {}", iae.getMessage());
+            printUsage();
+            Runtime.getRuntime().exit(-1);
+        } catch (ConfigurationException ce) {
+            logger.error("Configuration error : {}", ce.getMessage());
+            printUsage();
+            Runtime.getRuntime().exit(-1);
+        } catch (IOException ie) {
+            logger.error("Failed to start distributedlog server : ", ie);
+            Runtime.getRuntime().exit(-1);
+        } catch (ClassNotFoundException cnf) {
+          logger.error("Failed to start distributedlog server : ", cnf);
+          Runtime.getRuntime().exit(-1);
+        }
+    }
+
+    private void runCmd(CommandLine cmdline)
+        throws IllegalArgumentException, IOException, ConfigurationException, ClassNotFoundException {
+        final StatsReceiver statsReceiver = NullStatsReceiver.get();
+        Optional<String> confOptional = getOptionalStringArg(cmdline, "c");
+        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
+        if (confOptional.isPresent()) {
+            String configFile = confOptional.get();
+            try {
+                dlConf.loadConf(new File(configFile).toURI().toURL());
+            } catch (ConfigurationException e) {
+                throw new IllegalArgumentException("Failed to load distributedlog configuration from "
+                    + configFile + ".");
+            } catch (MalformedURLException e) {
+                throw new IllegalArgumentException("Failed to load distributedlog configuration from malformed "
+                        + configFile + ".");
+            }
+        }
+        // load the stats provider
+        final StatsProvider statsProvider = getOptionalStringArg(cmdline, "pd")
+                .transform(new Function<String, StatsProvider>() {
+                    @Nullable
+                    @Override
+                    public StatsProvider apply(@Nullable String name) {
+                        return ReflectionUtils.newInstance(name, StatsProvider.class);
+                    }
+                }).or(new NullStatsProvider());
+
+        final Optional<String> uriOption = getOptionalStringArg(cmdline, "u");
+        checkArgument(uriOption.isPresent(), "No distributedlog uri provided.");
+        URI dlUri = URI.create(uriOption.get());
+
+        DLZkServerSet serverSet = DLZkServerSet.of(dlUri, (int) TimeUnit.SECONDS.toMillis(60));
+        RoutingService routingService = RoutingUtils.buildRoutingService(serverSet.getServerSet())
+                .statsReceiver(statsReceiver.scope("routing"))
+                .build();
+
+        final DistributedLogServer server = DistributedLogServer.runServer(
+                uriOption,
+                confOptional,
+                getOptionalStringArg(cmdline, "sc"),
+                getOptionalIntegerArg(cmdline, "p"),
+                getOptionalIntegerArg(cmdline, "sp"),
+                getOptionalIntegerArg(cmdline, "si"),
+                getOptionalBooleanArg(cmdline, "a"),
+                getOptionalStringArg(cmdline, "la"),
+                getOptionalBooleanArg(cmdline, "mx"),
+                routingService,
+                statsReceiver,
+                statsProvider);
+
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                logger.info("Closing DistributedLog Server.");
+                server.close();
+                logger.info("Closed DistributedLog Server.");
+                statsProvider.stop();
+            }
+        });
+
+        try {
+            server.join();
+        } catch (InterruptedException e) {
+            logger.warn("Interrupted when waiting distributedlog server to be finished : ", e);
+        }
+
+        logger.info("DistributedLog Service Interrupted.");
+        server.close();
+        logger.info("Closed DistributedLog Server.");
+        statsProvider.stop();
+    }
+
+    public static void main(String[] args) {
+        final DistributedLogServerApp launcher = new DistributedLogServerApp(args);
+        launcher.run();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java
new file mode 100644
index 0000000..c37cd53
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/DistributedLogServiceImpl.java
@@ -0,0 +1,794 @@
+/**
+ * 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.service;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Stopwatch;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import com.twitter.common.net.InetSocketAddressHelper;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import org.apache.distributedlog.client.resolver.RegionResolver;
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.RegionUnavailableException;
+import org.apache.distributedlog.exceptions.ServiceUnavailableException;
+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.service.config.ServerConfiguration;
+import org.apache.distributedlog.service.config.StreamConfigProvider;
+import org.apache.distributedlog.service.placement.LeastLoadPlacementPolicy;
+import org.apache.distributedlog.service.placement.LoadAppraiser;
+import org.apache.distributedlog.service.placement.PlacementPolicy;
+import org.apache.distributedlog.service.placement.ZKPlacementStateManager;
+import org.apache.distributedlog.service.stream.BulkWriteOp;
+import org.apache.distributedlog.service.stream.DeleteOp;
+import org.apache.distributedlog.service.stream.HeartbeatOp;
+import org.apache.distributedlog.service.stream.ReleaseOp;
+import org.apache.distributedlog.service.stream.Stream;
+import org.apache.distributedlog.service.stream.StreamFactory;
+import org.apache.distributedlog.service.stream.StreamFactoryImpl;
+import org.apache.distributedlog.service.stream.StreamManager;
+import org.apache.distributedlog.service.stream.StreamManagerImpl;
+import org.apache.distributedlog.service.stream.StreamOp;
+import org.apache.distributedlog.service.stream.StreamOpStats;
+import org.apache.distributedlog.service.stream.TruncateOp;
+import org.apache.distributedlog.service.stream.WriteOp;
+import org.apache.distributedlog.service.stream.WriteOpWithPayload;
+import org.apache.distributedlog.service.stream.admin.CreateOp;
+import org.apache.distributedlog.service.stream.admin.StreamAdminOp;
+import org.apache.distributedlog.service.stream.limiter.ServiceRequestLimiter;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.distributedlog.service.utils.ServerUtils;
+import org.apache.distributedlog.thrift.service.BulkWriteResponse;
+import org.apache.distributedlog.thrift.service.ClientInfo;
+import org.apache.distributedlog.thrift.service.DistributedLogService;
+import org.apache.distributedlog.thrift.service.HeartbeatOptions;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.thrift.service.ServerInfo;
+import org.apache.distributedlog.thrift.service.ServerStatus;
+import org.apache.distributedlog.thrift.service.StatusCode;
+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.OrderedScheduler;
+import org.apache.distributedlog.util.SchedulerUtils;
+import com.twitter.util.Await;
+import com.twitter.util.Duration;
+import com.twitter.util.Function;
+import com.twitter.util.Function0;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.ScheduledThreadPoolTimer;
+import com.twitter.util.Timer;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Implementation of distributedlog thrift service.
+ */
+public class DistributedLogServiceImpl implements DistributedLogService.ServiceIface,
+                                                  FatalErrorHandler {
+
+    private static final Logger logger = LoggerFactory.getLogger(DistributedLogServiceImpl.class);
+
+    private static final int MOVING_AVERAGE_WINDOW_SECS = 60;
+
+    private final ServerConfiguration serverConfig;
+    private final DistributedLogConfiguration dlConfig;
+    private final DistributedLogNamespace dlNamespace;
+    private final int serverRegionId;
+    private final PlacementPolicy placementPolicy;
+    private ServerStatus serverStatus = ServerStatus.WRITE_AND_ACCEPT;
+    private final ReentrantReadWriteLock closeLock =
+            new ReentrantReadWriteLock();
+    private final CountDownLatch keepAliveLatch;
+    private final byte dlsnVersion;
+    private final String clientId;
+    private final OrderedScheduler scheduler;
+    private final AccessControlManager accessControlManager;
+    private final StreamConfigProvider streamConfigProvider;
+    private final StreamManager streamManager;
+    private final StreamFactory streamFactory;
+    private final RoutingService routingService;
+    private final RegionResolver regionResolver;
+    private final MovingAverageRateFactory movingAvgFactory;
+    private final MovingAverageRate windowedRps;
+    private final MovingAverageRate windowedBps;
+    private final ServiceRequestLimiter limiter;
+    private final Timer timer;
+    private final HashedWheelTimer requestTimer;
+
+    // Features
+    private final FeatureProvider featureProvider;
+    private final Feature featureRegionStopAcceptNewStream;
+    private final Feature featureChecksumDisabled;
+    private final Feature limiterDisabledFeature;
+
+    // Stats
+    private final StatsLogger statsLogger;
+    private final StatsLogger perStreamStatsLogger;
+    private final StreamPartitionConverter streamPartitionConverter;
+    private final StreamOpStats streamOpStats;
+    private final Counter bulkWritePendingStat;
+    private final Counter writePendingStat;
+    private final Counter redirects;
+    private final Counter receivedRecordCounter;
+    private final StatsLogger statusCodeStatLogger;
+    private final ConcurrentHashMap<StatusCode, Counter> statusCodeCounters =
+            new ConcurrentHashMap<StatusCode, Counter>();
+    private final Counter statusCodeTotal;
+    private final Gauge<Number> proxyStatusGauge;
+    private final Gauge<Number> movingAvgRpsGauge;
+    private final Gauge<Number> movingAvgBpsGauge;
+    private final Gauge<Number> streamAcquiredGauge;
+    private final Gauge<Number> streamCachedGauge;
+    private final int shard;
+
+    DistributedLogServiceImpl(ServerConfiguration serverConf,
+                              DistributedLogConfiguration dlConf,
+                              DynamicDistributedLogConfiguration dynDlConf,
+                              StreamConfigProvider streamConfigProvider,
+                              URI uri,
+                              StreamPartitionConverter converter,
+                              RoutingService routingService,
+                              StatsLogger statsLogger,
+                              StatsLogger perStreamStatsLogger,
+                              CountDownLatch keepAliveLatch,
+                              LoadAppraiser loadAppraiser)
+            throws IOException {
+        // Configuration.
+        this.serverConfig = serverConf;
+        this.dlConfig = dlConf;
+        this.perStreamStatsLogger = perStreamStatsLogger;
+        this.dlsnVersion = serverConf.getDlsnVersion();
+        this.serverRegionId = serverConf.getRegionId();
+        this.streamPartitionConverter = converter;
+        int serverPort = serverConf.getServerPort();
+        this.shard = serverConf.getServerShardId();
+        int numThreads = serverConf.getServerThreads();
+        this.clientId = DLSocketAddress.toLockId(DLSocketAddress.getSocketAddress(serverPort), shard);
+        String allocatorPoolName = ServerUtils.getLedgerAllocatorPoolName(
+            serverRegionId,
+            shard,
+            serverConf.isUseHostnameAsAllocatorPoolName());
+        dlConf.setLedgerAllocatorPoolName(allocatorPoolName);
+        this.featureProvider = AbstractFeatureProvider.getFeatureProvider("", dlConf, statsLogger.scope("features"));
+        if (this.featureProvider instanceof AbstractFeatureProvider) {
+            ((AbstractFeatureProvider) featureProvider).start();
+        }
+
+        // Build the namespace
+        this.dlNamespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(dlConf)
+                .uri(uri)
+                .statsLogger(statsLogger)
+                .featureProvider(this.featureProvider)
+                .clientId(clientId)
+                .regionId(serverRegionId)
+                .build();
+        this.accessControlManager = this.dlNamespace.createAccessControlManager();
+        this.keepAliveLatch = keepAliveLatch;
+        this.streamConfigProvider = streamConfigProvider;
+
+        // Stats pertaining to stream op execution
+        this.streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+
+        // Executor Service.
+        this.scheduler = OrderedScheduler.newBuilder()
+                .corePoolSize(numThreads)
+                .name("DistributedLogService-Executor")
+                .traceTaskExecution(true)
+                .statsLogger(statsLogger.scope("scheduler"))
+                .build();
+
+        // Timer, kept separate to ensure reliability of timeouts.
+        this.requestTimer = new HashedWheelTimer(
+            new ThreadFactoryBuilder().setNameFormat("DLServiceTimer-%d").build(),
+            dlConf.getTimeoutTimerTickDurationMs(), TimeUnit.MILLISECONDS,
+            dlConf.getTimeoutTimerNumTicks());
+
+        // Creating and managing Streams
+        this.streamFactory = new StreamFactoryImpl(clientId,
+                streamOpStats,
+                serverConf,
+                dlConf,
+                featureProvider,
+                streamConfigProvider,
+                converter,
+                dlNamespace,
+                scheduler,
+                this,
+                requestTimer);
+        this.streamManager = new StreamManagerImpl(
+                clientId,
+                dlConf,
+                scheduler,
+                streamFactory,
+                converter,
+                streamConfigProvider,
+                dlNamespace);
+        this.routingService = routingService;
+        this.regionResolver = new DefaultRegionResolver();
+
+        // Service features
+        this.featureRegionStopAcceptNewStream = this.featureProvider.getFeature(
+                ServerFeatureKeys.REGION_STOP_ACCEPT_NEW_STREAM.name().toLowerCase());
+        this.featureChecksumDisabled = this.featureProvider.getFeature(
+                ServerFeatureKeys.SERVICE_CHECKSUM_DISABLED.name().toLowerCase());
+        this.limiterDisabledFeature = this.featureProvider.getFeature(
+                ServerFeatureKeys.SERVICE_GLOBAL_LIMITER_DISABLED.name().toLowerCase());
+
+        // Resource limiting
+        this.timer = new ScheduledThreadPoolTimer(1, "timer", true);
+        this.movingAvgFactory = new MovingAverageRateFactory(timer);
+        this.windowedRps = movingAvgFactory.create(MOVING_AVERAGE_WINDOW_SECS);
+        this.windowedBps = movingAvgFactory.create(MOVING_AVERAGE_WINDOW_SECS);
+        this.limiter = new ServiceRequestLimiter(
+                dynDlConf,
+                streamOpStats.baseScope("service_limiter"),
+                windowedRps,
+                windowedBps,
+                streamManager,
+                limiterDisabledFeature);
+
+        this.placementPolicy = new LeastLoadPlacementPolicy(
+            loadAppraiser,
+            routingService,
+            dlNamespace,
+            new ZKPlacementStateManager(uri, dlConf, statsLogger),
+            Duration.fromSeconds(serverConf.getResourcePlacementRefreshInterval()),
+            statsLogger);
+        logger.info("placement started");
+
+        // Stats
+        this.statsLogger = statsLogger;
+
+        // Gauges for server status/health
+        this.proxyStatusGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return ServerStatus.DOWN == serverStatus ? -1 : (featureRegionStopAcceptNewStream.isAvailable()
+                    ? 3 : (ServerStatus.WRITE_AND_ACCEPT == serverStatus ? 1 : 2));
+            }
+        };
+        this.movingAvgRpsGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return windowedRps.get();
+            }
+        };
+        this.movingAvgBpsGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return windowedBps.get();
+            }
+        };
+        // Gauges for streams
+        this.streamAcquiredGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return streamManager.numAcquired();
+            }
+        };
+        this.streamCachedGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return streamManager.numCached();
+            }
+        };
+
+        // Stats on server
+        statsLogger.registerGauge("proxy_status", proxyStatusGauge);
+        // Global moving average rps
+        statsLogger.registerGauge("moving_avg_rps", movingAvgRpsGauge);
+        // Global moving average bps
+        statsLogger.registerGauge("moving_avg_bps", movingAvgBpsGauge);
+        // Stats on requests
+        this.bulkWritePendingStat = streamOpStats.requestPendingCounter("bulkWritePending");
+        this.writePendingStat = streamOpStats.requestPendingCounter("writePending");
+        this.redirects = streamOpStats.requestCounter("redirect");
+        this.statusCodeStatLogger = streamOpStats.requestScope("statuscode");
+        this.statusCodeTotal = streamOpStats.requestCounter("statuscode_count");
+        this.receivedRecordCounter = streamOpStats.recordsCounter("received");
+
+        // Stats for streams
+        StatsLogger streamsStatsLogger = statsLogger.scope("streams");
+        streamsStatsLogger.registerGauge("acquired", this.streamAcquiredGauge);
+        streamsStatsLogger.registerGauge("cached", this.streamCachedGauge);
+
+        // Setup complete
+        logger.info("Running distributedlog server : client id {}, allocator pool {}, perstream stat {},"
+            + " dlsn version {}.",
+            new Object[] { clientId, allocatorPoolName, serverConf.isPerStreamStatEnabled(), dlsnVersion });
+    }
+
+    private void countStatusCode(StatusCode code) {
+        Counter counter = statusCodeCounters.get(code);
+        if (null == counter) {
+            counter = statusCodeStatLogger.getCounter(code.name());
+            Counter oldCounter = statusCodeCounters.putIfAbsent(code, counter);
+            if (null != oldCounter) {
+                counter = oldCounter;
+            }
+        }
+        counter.inc();
+        statusCodeTotal.inc();
+    }
+
+    @Override
+    public Future<ServerInfo> handshake() {
+        return handshakeWithClientInfo(new ClientInfo());
+    }
+
+    @Override
+    public Future<ServerInfo> handshakeWithClientInfo(ClientInfo clientInfo) {
+        ServerInfo serverInfo = new ServerInfo();
+        closeLock.readLock().lock();
+        try {
+            serverInfo.setServerStatus(serverStatus);
+        } finally {
+            closeLock.readLock().unlock();
+        }
+
+        if (clientInfo.isSetGetOwnerships() && !clientInfo.isGetOwnerships()) {
+            return Future.value(serverInfo);
+        }
+
+        Optional<String> regex = Optional.absent();
+        if (clientInfo.isSetStreamNameRegex()) {
+            regex = Optional.of(clientInfo.getStreamNameRegex());
+        }
+
+        Map<String, String> ownershipMap = streamManager.getStreamOwnershipMap(regex);
+        serverInfo.setOwnerships(ownershipMap);
+        return Future.value(serverInfo);
+    }
+
+    @VisibleForTesting
+    Stream getLogWriter(String stream) throws IOException {
+        Stream writer = streamManager.getStream(stream);
+        if (null == writer) {
+            closeLock.readLock().lock();
+            try {
+                if (featureRegionStopAcceptNewStream.isAvailable()) {
+                    // accept new stream is disabled in current dc
+                    throw new RegionUnavailableException("Region is unavailable right now.");
+                } else if (!(ServerStatus.WRITE_AND_ACCEPT == serverStatus)) {
+                    // if it is closed, we would not acquire stream again.
+                    return null;
+                }
+                writer = streamManager.getOrCreateStream(stream, true);
+            } finally {
+                closeLock.readLock().unlock();
+            }
+        }
+        return writer;
+    }
+
+    // Service interface methods
+
+    @Override
+    public Future<WriteResponse> write(final String stream, ByteBuffer data) {
+        receivedRecordCounter.inc();
+        return doWrite(stream, data, null /* checksum */, false);
+    }
+
+    @Override
+    public Future<BulkWriteResponse> writeBulkWithContext(final String stream,
+                                                          List<ByteBuffer> data,
+                                                          WriteContext ctx) {
+        bulkWritePendingStat.inc();
+        receivedRecordCounter.add(data.size());
+        BulkWriteOp op = new BulkWriteOp(stream, data, statsLogger, perStreamStatsLogger, streamPartitionConverter,
+            getChecksum(ctx), featureChecksumDisabled, accessControlManager);
+        executeStreamOp(op);
+        return op.result().ensure(new Function0<BoxedUnit>() {
+            public BoxedUnit apply() {
+                bulkWritePendingStat.dec();
+                return null;
+            }
+        });
+    }
+
+    @Override
+    public Future<WriteResponse> writeWithContext(final String stream, ByteBuffer data, WriteContext ctx) {
+        return doWrite(stream, data, getChecksum(ctx), ctx.isIsRecordSet());
+    }
+
+    @Override
+    public Future<WriteResponse> heartbeat(String stream, WriteContext ctx) {
+        HeartbeatOp op = new HeartbeatOp(stream, statsLogger, perStreamStatsLogger, dlsnVersion, getChecksum(ctx),
+            featureChecksumDisabled, accessControlManager);
+        executeStreamOp(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<WriteResponse> heartbeatWithOptions(String stream, WriteContext ctx, HeartbeatOptions options) {
+        HeartbeatOp op = new HeartbeatOp(stream, statsLogger, perStreamStatsLogger, dlsnVersion, getChecksum(ctx),
+            featureChecksumDisabled, accessControlManager);
+        if (options.isSendHeartBeatToReader()) {
+            op.setWriteControlRecord(true);
+        }
+        executeStreamOp(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<WriteResponse> truncate(String stream, String dlsn, WriteContext ctx) {
+        TruncateOp op = new TruncateOp(
+            stream,
+            DLSN.deserialize(dlsn),
+            statsLogger,
+            perStreamStatsLogger,
+            getChecksum(ctx),
+            featureChecksumDisabled,
+            accessControlManager);
+        executeStreamOp(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<WriteResponse> delete(String stream, WriteContext ctx) {
+        DeleteOp op = new DeleteOp(stream, statsLogger, perStreamStatsLogger, streamManager, getChecksum(ctx),
+            featureChecksumDisabled, accessControlManager);
+        executeStreamOp(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<WriteResponse> release(String stream, WriteContext ctx) {
+        ReleaseOp op = new ReleaseOp(stream, statsLogger, perStreamStatsLogger, streamManager, getChecksum(ctx),
+            featureChecksumDisabled, accessControlManager);
+        executeStreamOp(op);
+        return op.result();
+    }
+
+    @Override
+    public Future<WriteResponse> create(String stream, WriteContext ctx) {
+        CreateOp op = new CreateOp(stream, statsLogger, streamManager, getChecksum(ctx), featureChecksumDisabled);
+        return executeStreamAdminOp(op);
+    }
+
+    //
+    // Ownership RPC
+    //
+
+    @Override
+    public Future<WriteResponse> getOwner(String streamName, WriteContext ctx) {
+        if (streamManager.isAcquired(streamName)) {
+            // the stream is already acquired
+            return Future.value(new WriteResponse(ResponseUtils.ownerToHeader(clientId)));
+        }
+
+        return placementPolicy.placeStream(streamName).map(new Function<String, WriteResponse>() {
+            @Override
+            public WriteResponse apply(String server) {
+                String host = DLSocketAddress.toLockId(InetSocketAddressHelper.parse(server), -1);
+                return new WriteResponse(ResponseUtils.ownerToHeader(host));
+            }
+        });
+    }
+
+
+    //
+    // Admin RPCs
+    //
+
+    @Override
+    public Future<Void> setAcceptNewStream(boolean enabled) {
+        closeLock.writeLock().lock();
+        try {
+            logger.info("Set AcceptNewStream = {}", enabled);
+            if (ServerStatus.DOWN != serverStatus) {
+                if (enabled) {
+                    serverStatus = ServerStatus.WRITE_AND_ACCEPT;
+                } else {
+                    serverStatus = ServerStatus.WRITE_ONLY;
+                }
+            }
+        } finally {
+            closeLock.writeLock().unlock();
+        }
+        return Future.Void();
+    }
+
+    private Future<WriteResponse> doWrite(final String name,
+                                          ByteBuffer data,
+                                          Long checksum,
+                                          boolean isRecordSet) {
+        writePendingStat.inc();
+        receivedRecordCounter.inc();
+        WriteOp op = newWriteOp(name, data, checksum, isRecordSet);
+        executeStreamOp(op);
+        return op.result().ensure(new Function0<BoxedUnit>() {
+            public BoxedUnit apply() {
+                writePendingStat.dec();
+                return null;
+            }
+        });
+    }
+
+    private Long getChecksum(WriteContext ctx) {
+        return ctx.isSetCrc32() ? ctx.getCrc32() : null;
+    }
+
+    private Future<WriteResponse> executeStreamAdminOp(final StreamAdminOp op) {
+        try {
+            op.preExecute();
+        } catch (DLException dle) {
+            return Future.exception(dle);
+        }
+        return op.execute();
+    }
+
+    private void executeStreamOp(final StreamOp op) {
+
+        // Must attach this as early as possible--returning before this point will cause us to
+        // lose the status code.
+        op.responseHeader().addEventListener(new FutureEventListener<ResponseHeader>() {
+            @Override
+            public void onSuccess(ResponseHeader header) {
+                if (header.getLocation() != null || header.getCode() == StatusCode.FOUND) {
+                    redirects.inc();
+                }
+                countStatusCode(header.getCode());
+            }
+            @Override
+            public void onFailure(Throwable cause) {
+            }
+        });
+
+        try {
+            // Apply the request limiter
+            limiter.apply(op);
+
+            // Execute per-op pre-exec code
+            op.preExecute();
+
+        } catch (TooManyStreamsException e) {
+            // Translate to StreamUnavailableException to ensure that the client will redirect
+            // to a different host. Ideally we would be able to return TooManyStreamsException,
+            // but the way exception handling works right now we can't control the handling in
+            // the client because client changes deploy very slowly.
+            op.fail(new StreamUnavailableException(e.getMessage()));
+            return;
+        } catch (Exception e) {
+            op.fail(e);
+            return;
+        }
+
+        Stream stream;
+        try {
+            stream = getLogWriter(op.streamName());
+        } catch (RegionUnavailableException rue) {
+            // redirect the requests to other region
+            op.fail(new RegionUnavailableException("Region " + serverRegionId + " is unavailable."));
+            return;
+        } catch (IOException e) {
+            op.fail(e);
+            return;
+        }
+        if (null == stream) {
+            // redirect the requests when stream is unavailable.
+            op.fail(new ServiceUnavailableException("Server " + clientId + " is closed."));
+            return;
+        }
+
+        if (op instanceof WriteOpWithPayload) {
+            WriteOpWithPayload writeOp = (WriteOpWithPayload) op;
+            windowedBps.add(writeOp.getPayloadSize());
+            windowedRps.inc();
+        }
+
+        stream.submit(op);
+    }
+
+    void shutdown() {
+        try {
+            closeLock.writeLock().lock();
+            try {
+                if (ServerStatus.DOWN == serverStatus) {
+                    return;
+                }
+                serverStatus = ServerStatus.DOWN;
+            } finally {
+                closeLock.writeLock().unlock();
+            }
+
+            streamManager.close();
+            movingAvgFactory.close();
+            limiter.close();
+
+            Stopwatch closeStreamsStopwatch = Stopwatch.createStarted();
+
+            Future<List<Void>> closeResult = streamManager.closeStreams();
+            logger.info("Waiting for closing all streams ...");
+            try {
+                Await.result(closeResult, Duration.fromTimeUnit(5, TimeUnit.MINUTES));
+                logger.info("Closed all streams in {} millis.",
+                        closeStreamsStopwatch.elapsed(TimeUnit.MILLISECONDS));
+            } catch (InterruptedException e) {
+                logger.warn("Interrupted on waiting for closing all streams : ", e);
+                Thread.currentThread().interrupt();
+            } catch (Exception e) {
+                logger.warn("Sorry, we didn't close all streams gracefully in 5 minutes : ", e);
+            }
+
+            // shutdown the dl namespace
+            logger.info("Closing distributedlog namespace ...");
+            dlNamespace.close();
+            logger.info("Closed distributedlog namespace .");
+
+            // Stop the feature provider
+            if (this.featureProvider instanceof AbstractFeatureProvider) {
+                ((AbstractFeatureProvider) featureProvider).stop();
+            }
+
+            // Stop the timer.
+            timer.stop();
+            placementPolicy.close();
+
+            // clean up gauge
+            unregisterGauge();
+
+            // shutdown the executor after requesting closing streams.
+            SchedulerUtils.shutdownScheduler(scheduler, 60, TimeUnit.SECONDS);
+        } catch (Exception ex) {
+            logger.info("Exception while shutting down distributedlog service.");
+        } finally {
+            // release the keepAliveLatch in case shutdown is called from a shutdown hook.
+            keepAliveLatch.countDown();
+            logger.info("Finished shutting down distributedlog service.");
+        }
+    }
+
+    protected void startPlacementPolicy() {
+        this.placementPolicy.start(shard == 0);
+    }
+
+    @Override
+    public void notifyFatalError() {
+        triggerShutdown();
+    }
+
+    private void triggerShutdown() {
+        // release the keepAliveLatch to let the main thread shutdown the whole service.
+        logger.info("Releasing KeepAlive Latch to trigger shutdown ...");
+        keepAliveLatch.countDown();
+        logger.info("Released KeepAlive Latch. Main thread will shut the service down.");
+    }
+
+    // Test methods.
+
+    private DynamicDistributedLogConfiguration getDynConf(String streamName) {
+        Optional<DynamicDistributedLogConfiguration> dynDlConf =
+                streamConfigProvider.getDynamicStreamConfig(streamName);
+        if (dynDlConf.isPresent()) {
+            return dynDlConf.get();
+        } else {
+            return ConfUtils.getConstDynConf(dlConfig);
+        }
+    }
+
+    /**
+     * clean up the gauge before we close to help GC.
+     */
+    private void unregisterGauge(){
+        this.statsLogger.unregisterGauge("proxy_status", this.proxyStatusGauge);
+        this.statsLogger.unregisterGauge("moving_avg_rps", this.movingAvgRpsGauge);
+        this.statsLogger.unregisterGauge("moving_avg_bps", this.movingAvgBpsGauge);
+        this.statsLogger.unregisterGauge("acquired", this.streamAcquiredGauge);
+        this.statsLogger.unregisterGauge("cached", this.streamCachedGauge);
+    }
+
+    @VisibleForTesting
+    Stream newStream(String name) throws IOException {
+        return streamManager.getOrCreateStream(name, false);
+    }
+
+    @VisibleForTesting
+    WriteOp newWriteOp(String stream, ByteBuffer data, Long checksum) {
+        return newWriteOp(stream, data, checksum, false);
+    }
+
+    @VisibleForTesting
+    RoutingService getRoutingService() {
+        return this.routingService;
+    }
+
+    @VisibleForTesting
+    DLSocketAddress getServiceAddress() throws IOException {
+        return DLSocketAddress.deserialize(clientId);
+    }
+
+    WriteOp newWriteOp(String stream,
+                       ByteBuffer data,
+                       Long checksum,
+                       boolean isRecordSet) {
+        return new WriteOp(stream, data, statsLogger, perStreamStatsLogger, streamPartitionConverter,
+            serverConfig, dlsnVersion, checksum, isRecordSet, featureChecksumDisabled,
+            accessControlManager);
+    }
+
+    @VisibleForTesting
+    Future<List<Void>> closeStreams() {
+        return streamManager.closeStreams();
+    }
+
+    @VisibleForTesting
+    public DistributedLogNamespace getDistributedLogNamespace() {
+        return dlNamespace;
+    }
+
+    @VisibleForTesting
+    StreamManager getStreamManager() {
+        return streamManager;
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/FatalErrorHandler.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/FatalErrorHandler.java
new file mode 100644
index 0000000..17b5ab3
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/FatalErrorHandler.java
@@ -0,0 +1,30 @@
+/**
+ * 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.service;
+
+/**
+ * Implement handling for an unrecoverable error.
+ */
+public interface FatalErrorHandler {
+
+    /**
+     * This method is invoked when an unrecoverable error has occurred
+     * and no progress can be made. It should implement a shutdown routine.
+     */
+    void notifyFatalError();
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorService.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorService.java
new file mode 100644
index 0000000..b1e2879
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorService.java
@@ -0,0 +1,469 @@
+/**
+ * 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.service;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Stopwatch;
+import com.google.common.collect.Sets;
+import com.google.common.hash.HashFunction;
+import com.google.common.hash.Hashing;
+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.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 com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.stats.Stat;
+import com.twitter.finagle.stats.StatsReceiver;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Duration;
+import com.twitter.util.FutureEventListener;
+import java.io.File;
+import java.io.IOException;
+import java.net.MalformedURLException;
+import java.net.URI;
+import java.util.ArrayList;
+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.CountDownLatch;
+import java.util.concurrent.Executors;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.lang.StringUtils;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Monitor Service.
+ */
+public class MonitorService implements NamespaceListener {
+
+    private static final Logger logger = LoggerFactory.getLogger(MonitorService.class);
+
+    private DistributedLogNamespace dlNamespace = null;
+    private MonitorServiceClient dlClient = null;
+    private DLZkServerSet[] zkServerSets = null;
+    private final ScheduledExecutorService executorService =
+            Executors.newScheduledThreadPool(Runtime.getRuntime().availableProcessors());
+    private final CountDownLatch keepAliveLatch = new CountDownLatch(1);
+    private final Map<String, StreamChecker> knownStreams = new HashMap<String, StreamChecker>();
+
+    // Settings
+    private int regionId = DistributedLogConstants.LOCAL_REGION_ID;
+    private int interval = 100;
+    private String streamRegex = null;
+    private boolean watchNamespaceChanges = false;
+    private boolean handshakeWithClientInfo = false;
+    private int heartbeatEveryChecks = 0;
+    private int instanceId = -1;
+    private int totalInstances = -1;
+    private boolean isThriftMux = false;
+
+    // Options
+    private final Optional<String> uriArg;
+    private final Optional<String> confFileArg;
+    private final Optional<String> serverSetArg;
+    private final Optional<Integer> intervalArg;
+    private final Optional<Integer> regionIdArg;
+    private final Optional<String> streamRegexArg;
+    private final Optional<Integer> instanceIdArg;
+    private final Optional<Integer> totalInstancesArg;
+    private final Optional<Integer> heartbeatEveryChecksArg;
+    private final Optional<Boolean> handshakeWithClientInfoArg;
+    private final Optional<Boolean> watchNamespaceChangesArg;
+    private final Optional<Boolean> isThriftMuxArg;
+
+    // Stats
+    private final StatsProvider statsProvider;
+    private final StatsReceiver statsReceiver;
+    private final StatsReceiver monitorReceiver;
+    private final Stat successStat;
+    private final Stat failureStat;
+    private final Gauge<Number> numOfStreamsGauge;
+    // Hash Function
+    private final HashFunction hashFunction = Hashing.md5();
+
+    class StreamChecker implements Runnable, FutureEventListener<Void>, LogSegmentListener {
+        private final String name;
+        private volatile boolean closed = false;
+        private volatile boolean checking = false;
+        private final Stopwatch stopwatch = Stopwatch.createUnstarted();
+        private DistributedLogManager dlm = null;
+        private int numChecks = 0;
+
+        StreamChecker(String name) {
+            this.name = name;
+        }
+
+        @Override
+        public void run() {
+            if (null == dlm) {
+                try {
+                    dlm = dlNamespace.openLog(name);
+                    dlm.registerListener(this);
+                } catch (IOException e) {
+                    if (null != dlm) {
+                        try {
+                            dlm.close();
+                        } catch (IOException e1) {
+                            logger.error("Failed to close dlm for {} : ", name, e1);
+                        }
+                        dlm = null;
+                    }
+                    executorService.schedule(this, interval, TimeUnit.MILLISECONDS);
+                }
+            } else {
+                stopwatch.reset().start();
+                boolean sendHeartBeat;
+                if (heartbeatEveryChecks > 0) {
+                    synchronized (this) {
+                        ++numChecks;
+                        if (numChecks >= Integer.MAX_VALUE) {
+                            numChecks = 0;
+                        }
+                        sendHeartBeat = (numChecks % heartbeatEveryChecks) == 0;
+                    }
+                } else {
+                    sendHeartBeat = false;
+                }
+                if (sendHeartBeat) {
+                    dlClient.heartbeat(name).addEventListener(this);
+                } else {
+                    dlClient.check(name).addEventListener(this);
+                }
+            }
+        }
+
+        @Override
+        public void onSegmentsUpdated(List<LogSegmentMetadata> segments) {
+            if (segments.size() > 0 && segments.get(0).getRegionId() == regionId) {
+                if (!checking) {
+                    logger.info("Start checking stream {}.", name);
+                    checking = true;
+                    run();
+                }
+            } else {
+                if (checking) {
+                    logger.info("Stop checking stream {}.", name);
+                }
+            }
+        }
+
+        @Override
+        public void onLogStreamDeleted() {
+            logger.info("Stream {} is deleted", name);
+        }
+
+        @Override
+        public void onSuccess(Void value) {
+            successStat.add(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            scheduleCheck();
+        }
+
+        @Override
+        public void onFailure(Throwable cause) {
+            failureStat.add(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            scheduleCheck();
+        }
+
+        private void scheduleCheck() {
+            if (closed) {
+                return;
+            }
+            if (!checking) {
+                return;
+            }
+            try {
+                executorService.schedule(this, interval, TimeUnit.MILLISECONDS);
+            } catch (RejectedExecutionException ree) {
+                logger.error("Failed to schedule checking stream {} in {} ms : ",
+                        new Object[] { name, interval, ree });
+            }
+        }
+
+        private void close() {
+            closed = true;
+            if (null != dlm) {
+                try {
+                    dlm.close();
+                } catch (IOException e) {
+                    logger.error("Failed to close dlm for {} : ", name, e);
+                }
+            }
+        }
+    }
+
+    MonitorService(Optional<String> uriArg,
+                   Optional<String> confFileArg,
+                   Optional<String> serverSetArg,
+                   Optional<Integer> intervalArg,
+                   Optional<Integer> regionIdArg,
+                   Optional<String> streamRegexArg,
+                   Optional<Integer> instanceIdArg,
+                   Optional<Integer> totalInstancesArg,
+                   Optional<Integer> heartbeatEveryChecksArg,
+                   Optional<Boolean> handshakeWithClientInfoArg,
+                   Optional<Boolean> watchNamespaceChangesArg,
+                   Optional<Boolean> isThriftMuxArg,
+                   StatsReceiver statsReceiver,
+                   StatsProvider statsProvider) {
+        // options
+        this.uriArg = uriArg;
+        this.confFileArg = confFileArg;
+        this.serverSetArg = serverSetArg;
+        this.intervalArg = intervalArg;
+        this.regionIdArg = regionIdArg;
+        this.streamRegexArg = streamRegexArg;
+        this.instanceIdArg = instanceIdArg;
+        this.totalInstancesArg = totalInstancesArg;
+        this.heartbeatEveryChecksArg = heartbeatEveryChecksArg;
+        this.handshakeWithClientInfoArg = handshakeWithClientInfoArg;
+        this.watchNamespaceChangesArg = watchNamespaceChangesArg;
+        this.isThriftMuxArg = isThriftMuxArg;
+
+        // Stats
+        this.statsReceiver = statsReceiver;
+        this.monitorReceiver = statsReceiver.scope("monitor");
+        this.successStat = monitorReceiver.stat0("success");
+        this.failureStat = monitorReceiver.stat0("failure");
+        this.statsProvider = statsProvider;
+        this.numOfStreamsGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                return knownStreams.size();
+            }
+        };
+    }
+
+    public void runServer() throws IllegalArgumentException, IOException {
+        checkArgument(uriArg.isPresent(),
+                "No distributedlog uri provided.");
+        checkArgument(serverSetArg.isPresent(),
+                "No proxy server set provided.");
+        if (intervalArg.isPresent()) {
+            interval = intervalArg.get();
+        }
+        if (regionIdArg.isPresent()) {
+            regionId = regionIdArg.get();
+        }
+        if (streamRegexArg.isPresent()) {
+            streamRegex = streamRegexArg.get();
+        }
+        if (instanceIdArg.isPresent()) {
+            instanceId = instanceIdArg.get();
+        }
+        if (totalInstancesArg.isPresent()) {
+            totalInstances = totalInstancesArg.get();
+        }
+        if (heartbeatEveryChecksArg.isPresent()) {
+            heartbeatEveryChecks = heartbeatEveryChecksArg.get();
+        }
+        if (instanceId < 0 || totalInstances <= 0 || instanceId >= totalInstances) {
+            throw new IllegalArgumentException("Invalid instance id or total instances number.");
+        }
+        handshakeWithClientInfo = handshakeWithClientInfoArg.isPresent();
+        watchNamespaceChanges = watchNamespaceChangesArg.isPresent();
+        isThriftMux = isThriftMuxArg.isPresent();
+        URI uri = URI.create(uriArg.get());
+        DistributedLogConfiguration dlConf = new DistributedLogConfiguration();
+        if (confFileArg.isPresent()) {
+            String configFile = confFileArg.get();
+            try {
+                dlConf.loadConf(new File(configFile).toURI().toURL());
+            } catch (ConfigurationException e) {
+                throw new IOException("Failed to load distributedlog configuration from " + configFile + ".");
+            } catch (MalformedURLException e) {
+                throw new IOException("Failed to load distributedlog configuration from malformed "
+                        + configFile + ".");
+            }
+        }
+        logger.info("Starting stats provider : {}.", statsProvider.getClass());
+        statsProvider.start(dlConf);
+        String[] serverSetPaths = StringUtils.split(serverSetArg.get(), ",");
+        if (serverSetPaths.length == 0) {
+            throw new IllegalArgumentException("Invalid serverset paths provided : " + serverSetArg.get());
+        }
+
+        ServerSet[] serverSets = createServerSets(serverSetPaths);
+        ServerSet local = serverSets[0];
+        ServerSet[] remotes  = new ServerSet[serverSets.length - 1];
+        System.arraycopy(serverSets, 1, remotes, 0, remotes.length);
+
+        ClientBuilder finagleClientBuilder = ClientBuilder.get()
+            .connectTimeout(Duration.fromSeconds(1))
+            .tcpConnectTimeout(Duration.fromSeconds(1))
+            .requestTimeout(Duration.fromSeconds(2))
+            .keepAlive(true)
+            .failFast(false);
+
+        if (!isThriftMux) {
+            finagleClientBuilder = finagleClientBuilder
+                .hostConnectionLimit(2)
+                .hostConnectionCoresize(2);
+        }
+
+        dlClient = DistributedLogClientBuilder.newBuilder()
+                .name("monitor")
+                .thriftmux(isThriftMux)
+                .clientId(ClientId$.MODULE$.apply("monitor"))
+                .redirectBackoffMaxMs(50)
+                .redirectBackoffStartMs(100)
+                .requestTimeoutMs(2000)
+                .maxRedirects(2)
+                .serverSets(local, remotes)
+                .streamNameRegex(streamRegex)
+                .handshakeWithClientInfo(handshakeWithClientInfo)
+                .clientBuilder(finagleClientBuilder)
+                .statsReceiver(monitorReceiver.scope("client"))
+                .buildMonitorClient();
+        runMonitor(dlConf, uri);
+    }
+
+    ServerSet[] createServerSets(String[] serverSetPaths) {
+        ServerSet[] serverSets = new ServerSet[serverSetPaths.length];
+        zkServerSets = new DLZkServerSet[serverSetPaths.length];
+        for (int i = 0; i < serverSetPaths.length; i++) {
+            String serverSetPath = serverSetPaths[i];
+            zkServerSets[i] = parseServerSet(serverSetPath);
+            serverSets[i] = zkServerSets[i].getServerSet();
+        }
+        return serverSets;
+    }
+
+    protected DLZkServerSet parseServerSet(String serverSetPath) {
+        return DLZkServerSet.of(URI.create(serverSetPath), 60000);
+    }
+
+    @Override
+    public void onStreamsChanged(Iterator<String> streams) {
+        Set<String> newSet = new HashSet<String>();
+        while (streams.hasNext()) {
+            String s = streams.next();
+            if (null == streamRegex || s.matches(streamRegex)) {
+                if (Math.abs(hashFunction.hashUnencodedChars(s).asInt()) % totalInstances == instanceId) {
+                    newSet.add(s);
+                }
+            }
+        }
+        List<StreamChecker> tasksToCancel = new ArrayList<StreamChecker>();
+        synchronized (knownStreams) {
+            Set<String> knownStreamSet = new HashSet<String>(knownStreams.keySet());
+            Set<String> removedStreams = Sets.difference(knownStreamSet, newSet).immutableCopy();
+            Set<String> addedStreams = Sets.difference(newSet, knownStreamSet).immutableCopy();
+            for (String s : removedStreams) {
+                StreamChecker task = knownStreams.remove(s);
+                if (null != task) {
+                    logger.info("Removed stream {}", s);
+                    tasksToCancel.add(task);
+                }
+            }
+            for (String s : addedStreams) {
+                if (!knownStreams.containsKey(s)) {
+                    logger.info("Added stream {}", s);
+                    StreamChecker sc = new StreamChecker(s);
+                    knownStreams.put(s, sc);
+                    sc.run();
+                }
+            }
+        }
+        for (StreamChecker sc : tasksToCancel) {
+            sc.close();
+        }
+    }
+
+    void runMonitor(DistributedLogConfiguration conf, URI dlUri) throws IOException {
+        // stats
+        statsProvider.getStatsLogger("monitor").registerGauge("num_streams", numOfStreamsGauge);
+        logger.info("Construct dl namespace @ {}", dlUri);
+        dlNamespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(dlUri)
+                .build();
+        if (watchNamespaceChanges) {
+            dlNamespace.registerNamespaceListener(this);
+        } else {
+            onStreamsChanged(dlNamespace.getLogs());
+        }
+    }
+
+    /**
+     * Close the server.
+     */
+    public void close() {
+        logger.info("Closing monitor service.");
+        if (null != dlClient) {
+            dlClient.close();
+        }
+        if (null != zkServerSets) {
+            for (DLZkServerSet zkServerSet : zkServerSets) {
+                zkServerSet.close();
+            }
+        }
+        if (null != dlNamespace) {
+            dlNamespace.close();
+        }
+        executorService.shutdown();
+        try {
+            if (!executorService.awaitTermination(1, TimeUnit.MINUTES)) {
+                executorService.shutdownNow();
+            }
+        } catch (InterruptedException e) {
+            logger.error("Interrupted on waiting shutting down monitor executor service : ", e);
+        }
+        if (null != statsProvider) {
+            // clean up the gauges
+            unregisterGauge();
+            statsProvider.stop();
+        }
+        keepAliveLatch.countDown();
+        logger.info("Closed monitor service.");
+    }
+
+    public void join() throws InterruptedException {
+        keepAliveLatch.await();
+    }
+
+    /**
+     * clean up the gauge before we close to help GC.
+     */
+    private void unregisterGauge(){
+        statsProvider.getStatsLogger("monitor").unregisterGauge("num_streams", numOfStreamsGauge);
+    }
+
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorServiceApp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorServiceApp.java
new file mode 100644
index 0000000..1f45b13
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/MonitorServiceApp.java
@@ -0,0 +1,133 @@
+/**
+ * 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.service;
+
+import static org.apache.distributedlog.util.CommandLineUtils.getOptionalBooleanArg;
+import static org.apache.distributedlog.util.CommandLineUtils.getOptionalIntegerArg;
+import static org.apache.distributedlog.util.CommandLineUtils.getOptionalStringArg;
+
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.io.IOException;
+import org.apache.bookkeeper.stats.NullStatsProvider;
+import org.apache.bookkeeper.stats.StatsProvider;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.cli.BasicParser;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.HelpFormatter;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+
+/**
+ * The launcher to run monitor service.
+ */
+public class MonitorServiceApp {
+
+    private static final Logger logger = LoggerFactory.getLogger(MonitorServiceApp.class);
+
+    static final String USAGE = "MonitorService [-u <uri>] [-c <conf>] [-s serverset]";
+
+    final String[] args;
+    final Options options = new Options();
+
+    private MonitorServiceApp(String[] args) {
+        this.args = args;
+        // prepare options
+        options.addOption("u", "uri", true, "DistributedLog URI");
+        options.addOption("c", "conf", true, "DistributedLog Configuration File");
+        options.addOption("s", "serverset", true, "Proxy Server Set");
+        options.addOption("i", "interval", true, "Check interval");
+        options.addOption("d", "region", true, "Region ID");
+        options.addOption("p", "provider", true, "DistributedLog Stats Provider");
+        options.addOption("f", "filter", true, "Filter streams by regex");
+        options.addOption("w", "watch", false, "Watch stream changes under a given namespace");
+        options.addOption("n", "instance_id", true, "Instance ID");
+        options.addOption("t", "total_instances", true, "Total instances");
+        options.addOption("hck", "heartbeat-num-checks", true, "Send a heartbeat after num checks");
+        options.addOption("hsci", "handshake-with-client-info", false, "Enable handshaking with client info");
+    }
+
+    void printUsage() {
+        HelpFormatter helpFormatter = new HelpFormatter();
+        helpFormatter.printHelp(USAGE, options);
+    }
+
+    private void run() {
+        try {
+            logger.info("Running monitor service.");
+            BasicParser parser = new BasicParser();
+            CommandLine cmdline = parser.parse(options, args);
+            runCmd(cmdline);
+        } catch (ParseException pe) {
+            printUsage();
+            Runtime.getRuntime().exit(-1);
+        } catch (IOException ie) {
+            logger.error("Failed to start monitor service : ", ie);
+            Runtime.getRuntime().exit(-1);
+        }
+    }
+
+    void runCmd(CommandLine cmdline) throws IOException {
+        StatsProvider statsProvider = new NullStatsProvider();
+        if (cmdline.hasOption("p")) {
+            String providerClass = cmdline.getOptionValue("p");
+            statsProvider = ReflectionUtils.newInstance(providerClass, StatsProvider.class);
+        }
+        StatsReceiver statsReceiver = NullStatsReceiver.get();
+
+        final MonitorService monitorService = new MonitorService(
+                getOptionalStringArg(cmdline, "u"),
+                getOptionalStringArg(cmdline, "c"),
+                getOptionalStringArg(cmdline, "s"),
+                getOptionalIntegerArg(cmdline, "i"),
+                getOptionalIntegerArg(cmdline, "d"),
+                getOptionalStringArg(cmdline, "f"),
+                getOptionalIntegerArg(cmdline, "n"),
+                getOptionalIntegerArg(cmdline, "t"),
+                getOptionalIntegerArg(cmdline, "hck"),
+                getOptionalBooleanArg(cmdline, "hsci"),
+                getOptionalBooleanArg(cmdline, "w"),
+                getOptionalBooleanArg(cmdline, "mx"),
+                statsReceiver,
+                statsProvider);
+
+        monitorService.runServer();
+
+        Runtime.getRuntime().addShutdownHook(new Thread() {
+            @Override
+            public void run() {
+                logger.info("Closing monitor service.");
+                monitorService.close();
+                logger.info("Closed monitor service.");
+            }
+        });
+        try {
+            monitorService.join();
+        } catch (InterruptedException ie) {
+            logger.warn("Interrupted when waiting monitor service to be finished : ", ie);
+        }
+    }
+
+    public static void main(String[] args) {
+        final MonitorServiceApp launcher = new MonitorServiceApp(args);
+        launcher.run();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/ResponseUtils.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/ResponseUtils.java
new file mode 100644
index 0000000..a2691d3
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/ResponseUtils.java
@@ -0,0 +1,86 @@
+/**
+ * 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.service;
+
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.thrift.service.BulkWriteResponse;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.thrift.service.StatusCode;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+
+/**
+ * Utility methods for building write proxy service responses.
+ */
+public class ResponseUtils {
+    public static ResponseHeader deniedHeader() {
+        return new ResponseHeader(StatusCode.REQUEST_DENIED);
+    }
+
+    public static ResponseHeader streamUnavailableHeader() {
+        return new ResponseHeader(StatusCode.STREAM_UNAVAILABLE);
+    }
+
+    public static ResponseHeader successHeader() {
+        return new ResponseHeader(StatusCode.SUCCESS);
+    }
+
+    public static ResponseHeader ownerToHeader(String owner) {
+        return new ResponseHeader(StatusCode.FOUND).setLocation(owner);
+    }
+
+    public static ResponseHeader exceptionToHeader(Throwable t) {
+        ResponseHeader response = new ResponseHeader();
+        if (t instanceof DLException) {
+            DLException dle = (DLException) t;
+            if (dle instanceof OwnershipAcquireFailedException) {
+                response.setLocation(((OwnershipAcquireFailedException) dle).getCurrentOwner());
+            }
+            response.setCode(dle.getCode());
+            response.setErrMsg(dle.getMessage());
+        } else {
+            response.setCode(StatusCode.INTERNAL_SERVER_ERROR);
+            response.setErrMsg("Internal server error : " + t.getMessage());
+        }
+        return response;
+    }
+
+    public static WriteResponse write(ResponseHeader responseHeader) {
+        return new WriteResponse(responseHeader);
+    }
+
+    public static WriteResponse writeSuccess() {
+        return new WriteResponse(successHeader());
+    }
+
+    public static WriteResponse writeDenied() {
+        return new WriteResponse(deniedHeader());
+    }
+
+    public static BulkWriteResponse bulkWrite(ResponseHeader responseHeader) {
+        return new BulkWriteResponse(responseHeader);
+    }
+
+    public static BulkWriteResponse bulkWriteSuccess() {
+        return new BulkWriteResponse(successHeader());
+    }
+
+    public static BulkWriteResponse bulkWriteDenied() {
+        return new BulkWriteResponse(deniedHeader());
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/ServerFeatureKeys.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/ServerFeatureKeys.java
new file mode 100644
index 0000000..436145d
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/ServerFeatureKeys.java
@@ -0,0 +1,29 @@
+/**
+ * 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.service;
+
+/**
+ * List of feature keys used by distributedlog server.
+ */
+public enum ServerFeatureKeys {
+
+    REGION_STOP_ACCEPT_NEW_STREAM,
+    SERVICE_RATE_LIMIT_DISABLED,
+    SERVICE_CHECKSUM_DISABLED,
+    SERVICE_GLOBAL_LIMITER_DISABLED
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/StatsFilter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/StatsFilter.java
new file mode 100644
index 0000000..ee64580
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/StatsFilter.java
@@ -0,0 +1,60 @@
+/**
+ * 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.service;
+
+import com.google.common.base.Stopwatch;
+import com.twitter.finagle.Service;
+import com.twitter.finagle.SimpleFilter;
+import com.twitter.util.Future;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * Track distributedlog server finagle-service stats.
+ */
+class StatsFilter<Req, Rep> extends SimpleFilter<Req, Rep> {
+
+    private final StatsLogger stats;
+    private final Counter outstandingAsync;
+    private final OpStatsLogger serviceExec;
+
+    @Override
+    public Future<Rep> apply(Req req, Service<Req, Rep> service) {
+        Future<Rep> result = null;
+        outstandingAsync.inc();
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+        try {
+            result = service.apply(req);
+            serviceExec.registerSuccessfulEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+        } finally {
+            outstandingAsync.dec();
+            if (null == result) {
+                serviceExec.registerFailedEvent(stopwatch.stop().elapsed(TimeUnit.MICROSECONDS));
+            }
+        }
+        return result;
+    }
+
+    public StatsFilter(StatsLogger stats) {
+        this.stats = stats;
+        this.outstandingAsync = stats.getCounter("outstandingAsync");
+        this.serviceExec = stats.getOpStatsLogger("serviceExec");
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/Announcer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/Announcer.java
new file mode 100644
index 0000000..ee64fc7
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/Announcer.java
@@ -0,0 +1,41 @@
+/**
+ * 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.service.announcer;
+
+import java.io.IOException;
+
+/**
+ * Announce service information.
+ */
+public interface Announcer {
+
+    /**
+     * Announce service info.
+     */
+    void announce() throws IOException;
+
+    /**
+     * Unannounce the service info.
+     */
+    void unannounce() throws IOException;
+
+    /**
+     * Close the announcer.
+     */
+    void close();
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/NOPAnnouncer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/NOPAnnouncer.java
new file mode 100644
index 0000000..5a1277a
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/NOPAnnouncer.java
@@ -0,0 +1,40 @@
+/**
+ * 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.service.announcer;
+
+import java.io.IOException;
+
+/**
+ * A no-op implementation of {@link Announcer}.
+ */
+public class NOPAnnouncer implements Announcer {
+    @Override
+    public void announce() throws IOException {
+        // nop
+    }
+
+    @Override
+    public void unannounce() throws IOException {
+        // nop
+    }
+
+    @Override
+    public void close() {
+        // nop
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/ServerSetAnnouncer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/ServerSetAnnouncer.java
new file mode 100644
index 0000000..df4a8e2
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/ServerSetAnnouncer.java
@@ -0,0 +1,111 @@
+/**
+ * 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.service.announcer;
+
+import com.twitter.common.zookeeper.Group;
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.client.serverset.DLZkServerSet;
+import java.io.IOException;
+import java.net.InetAddress;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.net.UnknownHostException;
+import java.util.HashMap;
+import java.util.Map;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * ServerSet based announcer.
+ */
+public class ServerSetAnnouncer implements Announcer {
+
+    private static final Logger logger = LoggerFactory.getLogger(ServerSetAnnouncer.class);
+
+    final String localAddr;
+    final InetSocketAddress serviceEndpoint;
+    final Map<String, InetSocketAddress> additionalEndpoints;
+    final int shardId;
+
+    // ServerSet
+    DLZkServerSet zkServerSet;
+
+    // Service Status
+    ServerSet.EndpointStatus serviceStatus = null;
+
+    /**
+     * Announce server infos.
+     *
+     * @param servicePort
+     *          service port
+     * @param statsPort
+     *          stats port
+     * @param shardId
+     *          shard id
+     */
+    public ServerSetAnnouncer(URI uri,
+                              int servicePort,
+                              int statsPort,
+                              int shardId) throws UnknownHostException {
+        this.shardId = shardId;
+        this.localAddr = InetAddress.getLocalHost().getHostAddress();
+        // service endpoint
+        this.serviceEndpoint = new InetSocketAddress(localAddr, servicePort);
+        // stats endpoint
+        InetSocketAddress statsEndpoint = new InetSocketAddress(localAddr, statsPort);
+        this.additionalEndpoints = new HashMap<String, InetSocketAddress>();
+        this.additionalEndpoints.put("aurora", statsEndpoint);
+        this.additionalEndpoints.put("stats", statsEndpoint);
+        this.additionalEndpoints.put("service", serviceEndpoint);
+        this.additionalEndpoints.put("thrift", serviceEndpoint);
+
+        // Create zookeeper and server set
+        this.zkServerSet = DLZkServerSet.of(uri, 60000);
+    }
+
+    @Override
+    public synchronized void announce() throws IOException {
+        try {
+            serviceStatus =
+                    zkServerSet.getServerSet().join(serviceEndpoint, additionalEndpoints, shardId);
+        } catch (Group.JoinException e) {
+            throw new IOException("Failed to announce service : ", e);
+        } catch (InterruptedException e) {
+            logger.warn("Interrupted on announcing service : ", e);
+            Thread.currentThread().interrupt();
+        }
+    }
+
+    @Override
+    public synchronized void unannounce() throws IOException {
+        if (null == serviceStatus) {
+            logger.warn("No service to unannounce.");
+            return;
+        }
+        try {
+            serviceStatus.leave();
+        } catch (ServerSet.UpdateException e) {
+            throw new IOException("Failed to unannounce service : ", e);
+        }
+    }
+
+    @Override
+    public void close() {
+        zkServerSet.close();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/package-info.java
new file mode 100644
index 0000000..6559bb3
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/announcer/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Announcers to announce servers to server set.
+ */
+package org.apache.distributedlog.service.announcer;
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/Balancer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/Balancer.java
new file mode 100644
index 0000000..cdffaa3
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/Balancer.java
@@ -0,0 +1,68 @@
+/**
+ * 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.service.balancer;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+
+/**
+ * Balancer Interface.
+ *
+ * <p>A balancer is used for balance the streams across the proxy cluster.
+ */
+public interface Balancer {
+
+    /**
+     * Rebalance all the streams from <i>source</i> to others.
+     *
+     * @param source
+     *          source target name.
+     * @param rebalanceConcurrency
+     *          the concurrency to move streams for re-balance.
+     * @param rebalanceRateLimiter
+     *          the rate limiting to move streams for re-balance.
+     */
+    void balanceAll(String source,
+                    int rebalanceConcurrency,
+                    Optional<RateLimiter> rebalanceRateLimiter);
+
+    /**
+     * Balance the streams across all targets.
+     *
+     * @param rebalanceWaterMark
+     *          rebalance water mark. if number of streams of a given target is less than
+     *          the water mark, no streams will be re-balanced from this target.
+     * @param rebalanceTolerancePercentage
+     *          tolerance percentage for the balancer. if number of streams of a given target is
+     *          less than average + average * <i>tolerancePercentage</i> / 100.0, no streams will
+     *          be re-balanced from that target.
+     * @param rebalanceConcurrency
+     *          the concurrency to move streams for re-balance.
+     * @param rebalanceRateLimiter
+     *          the rate limiting to move streams for re-balance.
+     */
+    void balance(int rebalanceWaterMark,
+                 double rebalanceTolerancePercentage,
+                 int rebalanceConcurrency,
+                 Optional<RateLimiter> rebalanceRateLimiter);
+
+    /**
+     * Close the balancer.
+     */
+    void close();
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerTool.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerTool.java
new file mode 100644
index 0000000..964c1cc
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerTool.java
@@ -0,0 +1,327 @@
+/**
+ * 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.service.balancer;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+import com.twitter.common.zookeeper.ServerSet;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.client.serverset.DLZkServerSet;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.distributedlog.service.ClientUtils;
+import org.apache.distributedlog.service.DLSocketAddress;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import org.apache.distributedlog.tools.Tool;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Await;
+import com.twitter.util.Duration;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tool to rebalance cluster.
+ */
+public class BalancerTool extends Tool {
+
+    private static final Logger logger = LoggerFactory.getLogger(BalancerTool.class);
+
+    static DistributedLogClientBuilder createDistributedLogClientBuilder(ServerSet serverSet) {
+        return DistributedLogClientBuilder.newBuilder()
+                        .name("rebalancer_tool")
+                        .clientId(ClientId$.MODULE$.apply("rebalancer_tool"))
+                        .maxRedirects(2)
+                        .serverSet(serverSet)
+                        .clientBuilder(ClientBuilder.get()
+                                .connectionTimeout(Duration.fromSeconds(2))
+                                .tcpConnectTimeout(Duration.fromSeconds(2))
+                                .requestTimeout(Duration.fromSeconds(10))
+                                .hostConnectionLimit(1)
+                                .hostConnectionCoresize(1)
+                                .keepAlive(true)
+                                .failFast(false));
+    }
+
+    /**
+     * Base Command to run balancer.
+     */
+    protected abstract static class BalancerCommand extends OptsCommand {
+
+        protected Options options = new Options();
+        protected int rebalanceWaterMark = 0;
+        protected double rebalanceTolerancePercentage = 0.0f;
+        protected int rebalanceConcurrency = 1;
+        protected Double rate = null;
+        protected Optional<RateLimiter> rateLimiter;
+
+        BalancerCommand(String name, String description) {
+            super(name, description);
+            options.addOption("rwm", "rebalance-water-mark", true, "Rebalance water mark per proxy");
+            options.addOption("rtp", "rebalance-tolerance-percentage", true,
+                "Rebalance tolerance percentage per proxy");
+            options.addOption("rc", "rebalance-concurrency", true, "Concurrency to rebalance stream distribution");
+            options.addOption("r", "rate", true, "Rebalance rate");
+        }
+
+        Optional<RateLimiter> getRateLimiter() {
+            return rateLimiter;
+        }
+
+        @Override
+        protected Options getOptions() {
+            return options;
+        }
+
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            if (cmdline.hasOption("rwm")) {
+                this.rebalanceWaterMark = Integer.parseInt(cmdline.getOptionValue("rwm"));
+            }
+            if (cmdline.hasOption("rtp")) {
+                this.rebalanceTolerancePercentage = Double.parseDouble(cmdline.getOptionValue("rtp"));
+            }
+            if (cmdline.hasOption("rc")) {
+                this.rebalanceConcurrency = Integer.parseInt(cmdline.getOptionValue("rc"));
+            }
+            if (cmdline.hasOption("r")) {
+                this.rate = Double.parseDouble(cmdline.getOptionValue("r"));
+            }
+            checkArgument(rebalanceWaterMark >= 0,
+                    "Rebalance Water Mark should be a non-negative number");
+            checkArgument(rebalanceTolerancePercentage >= 0.0f,
+                    "Rebalance Tolerance Percentage should be a non-negative number");
+            checkArgument(rebalanceConcurrency > 0,
+                    "Rebalance Concurrency should be a positive number");
+            if (null == rate || rate <= 0.0f) {
+                rateLimiter = Optional.absent();
+            } else {
+                rateLimiter = Optional.of(RateLimiter.create(rate));
+            }
+        }
+
+        @Override
+        protected int runCmd(CommandLine cmdline) throws Exception {
+            try {
+                parseCommandLine(cmdline);
+            } catch (ParseException pe) {
+                println("ERROR: fail to parse commandline : '" + pe.getMessage() + "'");
+                printUsage();
+                return -1;
+            }
+            return executeCommand(cmdline);
+        }
+
+        protected abstract int executeCommand(CommandLine cmdline) throws Exception;
+    }
+
+    /**
+     * Command to balance streams within a cluster.
+     */
+    protected static class ClusterBalancerCommand extends BalancerCommand {
+
+        protected URI uri;
+        protected String source = null;
+
+        protected ClusterBalancerCommand() {
+            super("clusterbalancer", "Balance streams inside a cluster");
+            options.addOption("u", "uri", true, "DistributedLog URI");
+            options.addOption("sp", "source-proxy", true, "Source proxy to balance");
+        }
+
+        @Override
+        protected String getUsage() {
+            return "clusterbalancer [options]";
+        }
+
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("u")) {
+                throw new ParseException("No proxy serverset provided.");
+            }
+            uri = URI.create(cmdline.getOptionValue("u"));
+            if (cmdline.hasOption("sp")) {
+                String sourceProxyStr = cmdline.getOptionValue("sp");
+                try {
+                    DLSocketAddress.parseSocketAddress(sourceProxyStr);
+                } catch (IllegalArgumentException iae) {
+                    throw new ParseException("Invalid source proxy " + sourceProxyStr + " : " + iae.getMessage());
+                }
+                this.source = sourceProxyStr;
+            }
+        }
+
+        @Override
+        protected int executeCommand(CommandLine cmdline) throws Exception {
+            DLZkServerSet serverSet = DLZkServerSet.of(uri, 60000);
+            logger.info("Created serverset for {}", uri);
+            try {
+                DistributedLogClientBuilder clientBuilder =
+                        createDistributedLogClientBuilder(serverSet.getServerSet());
+                ClusterBalancer balancer = new ClusterBalancer(clientBuilder);
+                try {
+                    return runBalancer(clientBuilder, balancer);
+                } finally {
+                    balancer.close();
+                }
+            } finally {
+                serverSet.close();
+            }
+        }
+
+        protected int runBalancer(DistributedLogClientBuilder clientBuilder,
+                                  ClusterBalancer balancer)
+                throws Exception {
+            if (null == source) {
+                balancer.balance(
+                    rebalanceWaterMark,
+                    rebalanceTolerancePercentage,
+                    rebalanceConcurrency,
+                    getRateLimiter());
+            } else {
+                balanceFromSource(clientBuilder, balancer, source, getRateLimiter());
+            }
+            return 0;
+        }
+
+        protected void balanceFromSource(DistributedLogClientBuilder clientBuilder,
+                                         ClusterBalancer balancer,
+                                         String source,
+                                         Optional<RateLimiter> rateLimiter)
+                throws Exception {
+            InetSocketAddress sourceAddr = DLSocketAddress.parseSocketAddress(source);
+            DistributedLogClientBuilder sourceClientBuilder =
+                    DistributedLogClientBuilder.newBuilder(clientBuilder)
+                            .host(sourceAddr);
+
+            Pair<DistributedLogClient, MonitorServiceClient> clientPair =
+                    ClientUtils.buildClient(sourceClientBuilder);
+            try {
+                Await.result(clientPair.getRight().setAcceptNewStream(false));
+                logger.info("Disable accepting new stream on proxy {}.", source);
+                balancer.balanceAll(source, rebalanceConcurrency, rateLimiter);
+            } finally {
+                clientPair.getLeft().close();
+            }
+        }
+    }
+
+    /**
+     * Command to balance streams between regions.
+     */
+    protected static class RegionBalancerCommand extends BalancerCommand {
+
+        protected URI region1;
+        protected URI region2;
+        protected String source = null;
+
+        protected RegionBalancerCommand() {
+            super("regionbalancer", "Balance streams between regions");
+            options.addOption("rs", "regions", true, "DistributedLog Region URI: uri1[,uri2]");
+            options.addOption("s", "source", true, "DistributedLog Source Region to balance");
+        }
+
+        @Override
+        protected String getUsage() {
+            return "regionbalancer [options]";
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("rs")) {
+                throw new ParseException("No regions provided.");
+            }
+            String regionsStr = cmdline.getOptionValue("rs");
+            String[] regions = regionsStr.split(",");
+            if (regions.length != 2) {
+                throw new ParseException("Invalid regions provided. Expected : serverset1[,serverset2]");
+            }
+            region1 = URI.create(regions[0]);
+            region2 = URI.create(regions[1]);
+            if (cmdline.hasOption("s")) {
+                source = cmdline.getOptionValue("s");
+            }
+        }
+
+        @Override
+        protected int executeCommand(CommandLine cmdline) throws Exception {
+            DLZkServerSet serverSet1 = DLZkServerSet.of(region1, 60000);
+            logger.info("Created serverset for {}", region1);
+            DLZkServerSet serverSet2 = DLZkServerSet.of(region2, 60000);
+            logger.info("Created serverset for {}", region2);
+            try {
+                DistributedLogClientBuilder builder1 =
+                        createDistributedLogClientBuilder(serverSet1.getServerSet());
+                Pair<DistributedLogClient, MonitorServiceClient> pair1 =
+                        ClientUtils.buildClient(builder1);
+                DistributedLogClientBuilder builder2 =
+                        createDistributedLogClientBuilder(serverSet2.getServerSet());
+                Pair<DistributedLogClient, MonitorServiceClient> pair2 =
+                        ClientUtils.buildClient(builder2);
+                try {
+                    SimpleBalancer balancer = new SimpleBalancer(
+                            BKNamespaceDriver.getZKServersFromDLUri(region1), pair1.getLeft(), pair1.getRight(),
+                            BKNamespaceDriver.getZKServersFromDLUri(region2), pair2.getLeft(), pair2.getRight());
+                    try {
+                        return runBalancer(balancer);
+                    } finally {
+                        balancer.close();
+                    }
+                } finally {
+                    pair1.getLeft().close();
+                    pair2.getLeft().close();
+                }
+            } finally {
+                serverSet1.close();
+                serverSet2.close();
+            }
+        }
+
+        protected int runBalancer(SimpleBalancer balancer) throws Exception {
+            if (null == source) {
+                balancer.balance(
+                    rebalanceWaterMark,
+                    rebalanceTolerancePercentage,
+                    rebalanceConcurrency,
+                    getRateLimiter());
+            } else {
+                balancer.balanceAll(source, rebalanceConcurrency, getRateLimiter());
+            }
+            return 0;
+        }
+    }
+
+    public BalancerTool() {
+        super();
+        addCommand(new ClusterBalancerCommand());
+        addCommand(new RegionBalancerCommand());
+    }
+
+    @Override
+    protected String getName() {
+        return "balancer";
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerUtils.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerUtils.java
new file mode 100644
index 0000000..4c9e075
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/BalancerUtils.java
@@ -0,0 +1,74 @@
+/**
+ * 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.service.balancer;
+
+import java.util.Map;
+
+/**
+ * Utils for balancer.
+ */
+public class BalancerUtils {
+
+    /**
+     * Util function to calculate how many streams to balance for <i>nodeToRebalance</i>,
+     * based on the load distribution <i>loadDistribution</i>.
+     *
+     * @param nodeToRebalance
+     *          node to rebalance
+     * @param loadDistribution
+     *          load distribution map
+     * @param rebalanceWaterMark
+     *          if number of streams of <i>nodeToRebalance</i>
+     *          is less than <i>rebalanceWaterMark</i>, no streams will be re-balanced.
+     * @param tolerancePercentage
+     *          tolerance percentage for the balancer. if number of streams of <i>nodeToRebalance</i>
+     *          is less than average + average * <i>tolerancePercentage</i> / 100.0, no streams will
+     *          be re-balanced.
+     * @param <K>
+     * @return number of streams to rebalance
+     */
+    public static <K> int calculateNumStreamsToRebalance(K nodeToRebalance,
+                                                         Map<K, Integer> loadDistribution,
+                                                         int rebalanceWaterMark,
+                                                         double tolerancePercentage) {
+        Integer myLoad = loadDistribution.get(nodeToRebalance);
+        if (null == myLoad || myLoad <= rebalanceWaterMark) {
+            return 0;
+        }
+
+        long totalLoad = 0L;
+        int numNodes = loadDistribution.size();
+
+        for (Map.Entry<K, Integer> entry : loadDistribution.entrySet()) {
+            if (null == entry.getKey() || null == entry.getValue()) {
+                continue;
+            }
+            totalLoad += entry.getValue();
+        }
+
+        double averageLoad = ((double) totalLoad) / numNodes;
+        long permissibleLoad =
+                Math.max(1L, (long) Math.ceil(averageLoad + averageLoad * tolerancePercentage / 100.0f));
+
+        if (myLoad <= permissibleLoad) {
+            return 0;
+        }
+
+        return Math.max(0, myLoad - (int) Math.ceil(averageLoad));
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/ClusterBalancer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/ClusterBalancer.java
new file mode 100644
index 0000000..5add339
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/ClusterBalancer.java
@@ -0,0 +1,378 @@
+/**
+ * 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.service.balancer;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.service.ClientUtils;
+import org.apache.distributedlog.service.DLSocketAddress;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import com.twitter.util.Await;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import java.io.Serializable;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A balancer balances ownerships with a cluster of targets.
+ */
+public class ClusterBalancer implements Balancer {
+
+    private static final Logger logger = LoggerFactory.getLogger(ClusterBalancer.class);
+
+    /**
+     * Represent a single host. Ordered by number of streams in desc order.
+     */
+    static class Host {
+
+        final SocketAddress address;
+        final Set<String> streams;
+        final DistributedLogClientBuilder clientBuilder;
+        DistributedLogClient client = null;
+        MonitorServiceClient monitor = null;
+
+        Host(SocketAddress address, Set<String> streams,
+             DistributedLogClientBuilder clientBuilder) {
+            this.address = address;
+            this.streams = streams;
+            this.clientBuilder = clientBuilder;
+        }
+
+        private void initializeClientsIfNeeded() {
+            if (null == client) {
+                Pair<DistributedLogClient, MonitorServiceClient> clientPair =
+                        createDistributedLogClient(address, clientBuilder);
+                client = clientPair.getLeft();
+                monitor = clientPair.getRight();
+            }
+        }
+
+        synchronized DistributedLogClient getClient() {
+            initializeClientsIfNeeded();
+            return client;
+        }
+
+        synchronized MonitorServiceClient getMonitor() {
+            initializeClientsIfNeeded();
+            return monitor;
+        }
+
+        synchronized void close() {
+            if (null != client) {
+                client.close();
+            }
+        }
+
+        @Override
+        public String toString() {
+            StringBuilder sb = new StringBuilder();
+            sb.append("Host(").append(address).append(")");
+            return sb.toString();
+        }
+    }
+
+    static class HostComparator implements Comparator<Host>, Serializable {
+        private static final long serialVersionUID = 7984973796525102538L;
+
+        @Override
+        public int compare(Host h1, Host h2) {
+            return h2.streams.size() - h1.streams.size();
+        }
+    }
+
+    protected final DistributedLogClientBuilder clientBuilder;
+    protected final DistributedLogClient client;
+    protected final MonitorServiceClient monitor;
+
+    public ClusterBalancer(DistributedLogClientBuilder clientBuilder) {
+        this(clientBuilder, ClientUtils.buildClient(clientBuilder));
+    }
+
+    ClusterBalancer(DistributedLogClientBuilder clientBuilder,
+                    Pair<DistributedLogClient, MonitorServiceClient> clientPair) {
+        this.clientBuilder = clientBuilder;
+        this.client = clientPair.getLeft();
+        this.monitor = clientPair.getRight();
+    }
+
+    /**
+     * Build a new distributedlog client to a single host <i>host</i>.
+     *
+     * @param host
+     *          host to access
+     * @return distributedlog clients
+     */
+    static Pair<DistributedLogClient, MonitorServiceClient> createDistributedLogClient(
+            SocketAddress host, DistributedLogClientBuilder clientBuilder) {
+        DistributedLogClientBuilder newBuilder =
+                DistributedLogClientBuilder.newBuilder(clientBuilder).host(host);
+        return ClientUtils.buildClient(newBuilder);
+    }
+
+    @Override
+    public void balanceAll(String source,
+                           int rebalanceConcurrency, /* unused */
+                           Optional<RateLimiter> rebalanceRateLimiter) {
+        balance(0, 0.0f, rebalanceConcurrency, Optional.of(source), rebalanceRateLimiter);
+    }
+
+    @Override
+    public void balance(int rebalanceWaterMark,
+                        double rebalanceTolerancePercentage,
+                        int rebalanceConcurrency, /* unused */
+                        Optional<RateLimiter> rebalanceRateLimiter) {
+        Optional<String> source = Optional.absent();
+        balance(rebalanceWaterMark, rebalanceTolerancePercentage, rebalanceConcurrency, source, rebalanceRateLimiter);
+    }
+
+    public void balance(int rebalanceWaterMark,
+                        double rebalanceTolerancePercentage,
+                        int rebalanceConcurrency,
+                        Optional<String> source,
+                        Optional<RateLimiter> rebalanceRateLimiter) {
+        Map<SocketAddress, Set<String>> distribution = monitor.getStreamOwnershipDistribution();
+        if (distribution.size() <= 1) {
+            return;
+        }
+        SocketAddress sourceAddr = null;
+        if (source.isPresent()) {
+            sourceAddr = DLSocketAddress.parseSocketAddress(source.get());
+            logger.info("Balancer source is {}", sourceAddr);
+            if (!distribution.containsKey(sourceAddr)) {
+                return;
+            }
+        }
+        // Get the list of hosts ordered by number of streams in DESC order
+        List<Host> hosts = new ArrayList<Host>(distribution.size());
+        for (Map.Entry<SocketAddress, Set<String>> entry : distribution.entrySet()) {
+            Host host = new Host(entry.getKey(), entry.getValue(), clientBuilder);
+            hosts.add(host);
+        }
+        Collections.sort(hosts, new HostComparator());
+        try {
+
+            // find the host to move streams from.
+            int hostIdxMoveFrom = -1;
+            if (null != sourceAddr) {
+                for (Host host : hosts) {
+                    ++hostIdxMoveFrom;
+                    if (sourceAddr.equals(host.address)) {
+                        break;
+                    }
+                }
+            }
+
+            // compute the average load.
+            int totalStream = 0;
+            for (Host host : hosts) {
+                totalStream += host.streams.size();
+            }
+            double averageLoad;
+            if (hostIdxMoveFrom >= 0) {
+                averageLoad = ((double) totalStream / (hosts.size() - 1));
+            } else {
+                averageLoad = ((double) totalStream / hosts.size());
+            }
+
+            int moveFromLowWaterMark;
+            int moveToHighWaterMark =
+                Math.max(1, (int) (averageLoad + averageLoad * rebalanceTolerancePercentage / 100.0f));
+
+            if (hostIdxMoveFrom >= 0) {
+                moveFromLowWaterMark = Math.max(0, rebalanceWaterMark);
+                moveStreams(
+                        hosts,
+                        new AtomicInteger(hostIdxMoveFrom), moveFromLowWaterMark,
+                        new AtomicInteger(hosts.size() - 1), moveToHighWaterMark,
+                        rebalanceRateLimiter);
+                moveRemainingStreamsFromSource(hosts.get(hostIdxMoveFrom), hosts, rebalanceRateLimiter);
+            } else {
+                moveFromLowWaterMark = Math.max((int) Math.ceil(averageLoad), rebalanceWaterMark);
+                AtomicInteger moveFrom = new AtomicInteger(0);
+                AtomicInteger moveTo = new AtomicInteger(hosts.size() - 1);
+                while (moveFrom.get() < moveTo.get()) {
+                    moveStreams(hosts, moveFrom, moveFromLowWaterMark,
+                        moveTo, moveToHighWaterMark, rebalanceRateLimiter);
+                    moveFrom.incrementAndGet();
+                }
+            }
+        } finally {
+            for (Host host : hosts) {
+                host.close();
+            }
+        }
+    }
+
+    void moveStreams(List<Host> hosts,
+                     AtomicInteger hostIdxMoveFrom,
+                     int moveFromLowWaterMark,
+                     AtomicInteger hostIdxMoveTo,
+                     int moveToHighWaterMark,
+                     Optional<RateLimiter> rateLimiter) {
+        if (hostIdxMoveFrom.get() < 0 || hostIdxMoveFrom.get() >= hosts.size()
+                || hostIdxMoveTo.get() < 0 || hostIdxMoveTo.get() >= hosts.size()
+                || hostIdxMoveFrom.get() >= hostIdxMoveTo.get()) {
+            return;
+        }
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Moving streams : hosts = {}, from = {}, to = {} :"
+                + " from_low_water_mark = {}, to_high_water_mark = {}",
+                new Object[] {
+                    hosts,
+                    hostIdxMoveFrom.get(),
+                    hostIdxMoveTo.get(),
+                    moveFromLowWaterMark,
+                    moveToHighWaterMark });
+        }
+
+        Host hostMoveFrom = hosts.get(hostIdxMoveFrom.get());
+        int numStreamsOnFromHost = hostMoveFrom.streams.size();
+        if (numStreamsOnFromHost <= moveFromLowWaterMark) {
+            // do nothing
+            return;
+        }
+
+        int numStreamsToMove = numStreamsOnFromHost - moveFromLowWaterMark;
+        LinkedList<String> streamsToMove = new LinkedList<String>(hostMoveFrom.streams);
+        Collections.shuffle(streamsToMove);
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Try to move {} streams from host {} : streams = {}",
+                         new Object[] { numStreamsToMove, hostMoveFrom.address, streamsToMove });
+        }
+
+        while (numStreamsToMove-- > 0 && !streamsToMove.isEmpty()) {
+            if (rateLimiter.isPresent()) {
+                rateLimiter.get().acquire();
+            }
+
+            // pick a host to move
+            Host hostMoveTo = hosts.get(hostIdxMoveTo.get());
+            while (hostMoveTo.streams.size() >= moveToHighWaterMark) {
+                int hostIdx = hostIdxMoveTo.decrementAndGet();
+                logger.info("move to host : {}, from {}", hostIdx, hostIdxMoveFrom.get());
+                if (hostIdx <= hostIdxMoveFrom.get()) {
+                    return;
+                } else {
+                    hostMoveTo = hosts.get(hostIdx);
+                    if (logger.isDebugEnabled()) {
+                        logger.debug("Target host to move moved to host {} @ {}",
+                                hostIdx, hostMoveTo);
+                    }
+                }
+            }
+
+            // pick a stream
+            String stream = streamsToMove.remove();
+
+            // move the stream
+            if (moveStream(stream, hostMoveFrom, hostMoveTo)) {
+                hostMoveFrom.streams.remove(stream);
+                hostMoveTo.streams.add(stream);
+            }
+        }
+
+    }
+
+    void moveRemainingStreamsFromSource(Host source,
+                                        List<Host> hosts,
+                                        Optional<RateLimiter> rateLimiter) {
+        LinkedList<String> streamsToMove = new LinkedList<String>(source.streams);
+        Collections.shuffle(streamsToMove);
+
+        if (logger.isDebugEnabled()) {
+            logger.debug("Try to move remaining streams from {} : {}", source, streamsToMove);
+        }
+
+        int hostIdx = hosts.size() - 1;
+
+        while (!streamsToMove.isEmpty()) {
+            if (rateLimiter.isPresent()) {
+                rateLimiter.get().acquire();
+            }
+
+            Host target = hosts.get(hostIdx);
+            if (!target.address.equals(source.address)) {
+                String stream = streamsToMove.remove();
+                // move the stream
+                if (moveStream(stream, source, target)) {
+                    source.streams.remove(stream);
+                    target.streams.add(stream);
+                }
+            }
+            --hostIdx;
+            if (hostIdx < 0) {
+                hostIdx = hosts.size() - 1;
+            }
+        }
+    }
+
+    private boolean moveStream(String stream, Host from, Host to) {
+        try {
+            doMoveStream(stream, from, to);
+            return true;
+        } catch (Exception e) {
+            return false;
+        }
+    }
+
+    private void doMoveStream(final String stream, final Host from, final Host to) throws Exception {
+        logger.info("Moving stream {} from {} to {}.",
+                    new Object[] { stream, from.address, to.address });
+        Await.result(from.getClient().release(stream).flatMap(new Function<Void, Future<Void>>() {
+            @Override
+            public Future<Void> apply(Void result) {
+                logger.info("Released stream {} from {}.", stream, from.address);
+                return to.getMonitor().check(stream).addEventListener(new FutureEventListener<Void>() {
+
+                    @Override
+                    public void onSuccess(Void value) {
+                        logger.info("Moved stream {} from {} to {}.",
+                                    new Object[] { stream, from.address, to.address });
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        logger.info("Failed to move stream {} from {} to {} : ",
+                                    new Object[] { stream, from.address, to.address, cause });
+                    }
+                });
+            }
+        }));
+    }
+
+    @Override
+    public void close() {
+        client.close();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/CountBasedStreamChooser.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/CountBasedStreamChooser.java
new file mode 100644
index 0000000..6a43179
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/CountBasedStreamChooser.java
@@ -0,0 +1,109 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.distributedlog.service.balancer;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import java.io.Serializable;
+import java.net.SocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.apache.commons.lang3.tuple.Pair;
+
+/**
+ * A stream chooser based on number of streams.
+ */
+class CountBasedStreamChooser implements StreamChooser, Serializable,
+        Comparator<Pair<SocketAddress, LinkedList<String>>> {
+
+    private static final long serialVersionUID = 4664153397369979203L;
+
+    final List<Pair<SocketAddress, LinkedList<String>>> streamsDistribution;
+
+    // pivot index in the list of hosts. the chooser will remove streams from the hosts before
+    // pivot, which will reduce their stream counts to make them equal to the stream count of the pivot.
+    int pivot;
+    int pivotCount;
+
+    // next index in the list of hosts to choose stream from.
+    int next;
+
+    CountBasedStreamChooser(Map<SocketAddress, Set<String>> streams) {
+        checkArgument(streams.size() > 0, "Only support no-empty streams distribution");
+        streamsDistribution = new ArrayList<Pair<SocketAddress, LinkedList<String>>>(streams.size());
+        for (Map.Entry<SocketAddress, Set<String>> entry : streams.entrySet()) {
+            LinkedList<String> randomizedStreams = new LinkedList<String>(entry.getValue());
+            Collections.shuffle(randomizedStreams);
+            streamsDistribution.add(Pair.of(entry.getKey(), randomizedStreams));
+        }
+        // sort the hosts by the number of streams in descending order
+        Collections.sort(streamsDistribution, this);
+        pivot = 0;
+        pivotCount = streamsDistribution.get(0).getValue().size();
+        findNextPivot();
+        next = 0;
+    }
+
+    private void findNextPivot() {
+        int prevPivotCount = pivotCount;
+        while (++pivot < streamsDistribution.size()) {
+            pivotCount = streamsDistribution.get(pivot).getValue().size();
+            if (pivotCount < prevPivotCount) {
+                return;
+            }
+        }
+        pivot = streamsDistribution.size();
+        pivotCount = 0;
+    }
+
+    @Override
+    public synchronized String choose() {
+        // reach the pivot
+        if (next == pivot) {
+            if (streamsDistribution.get(next - 1).getRight().size() > pivotCount) {
+                next = 0;
+            } else if (pivotCount == 0) { // the streams are empty now
+                return null;
+            } else {
+                findNextPivot();
+                next = 0;
+            }
+        }
+
+        // get stream count that next host to choose from
+        LinkedList<String> nextStreams = streamsDistribution.get(next).getRight();
+        if (nextStreams.size() == 0) {
+            return null;
+        }
+
+        String chosenStream = nextStreams.remove();
+        ++next;
+        return chosenStream;
+    }
+
+    @Override
+    public int compare(Pair<SocketAddress, LinkedList<String>> o1,
+                       Pair<SocketAddress, LinkedList<String>> o2) {
+        return o2.getValue().size() - o1.getValue().size();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/LimitedStreamChooser.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/LimitedStreamChooser.java
new file mode 100644
index 0000000..4aefc5e
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/LimitedStreamChooser.java
@@ -0,0 +1,57 @@
+/**
+ * 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.service.balancer;
+
+/**
+ * A stream chooser that can only choose limited number of streams.
+ */
+public class LimitedStreamChooser implements StreamChooser {
+
+  /**
+   * Create a limited stream chooser by {@code limit}.
+   *
+   * @param underlying the underlying stream chooser.
+   * @param limit the limit of number of streams to choose.
+   * @return the limited stream chooser.
+   */
+    public static LimitedStreamChooser of(StreamChooser underlying, int limit) {
+        return new LimitedStreamChooser(underlying, limit);
+    }
+
+    final StreamChooser underlying;
+    int limit;
+
+    LimitedStreamChooser(StreamChooser underlying, int limit) {
+        this.underlying = underlying;
+        this.limit = limit;
+    }
+
+    @Override
+    public synchronized String choose() {
+        if (limit <= 0) {
+            return null;
+        }
+        String s = underlying.choose();
+        if (s == null) {
+            limit = 0;
+            return null;
+        }
+        --limit;
+        return s;
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/SimpleBalancer.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/SimpleBalancer.java
new file mode 100644
index 0000000..3c53ccf
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/SimpleBalancer.java
@@ -0,0 +1,246 @@
+/**
+ * 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.service.balancer;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.service.DistributedLogClient;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * A balancer balances ownerships between two targets.
+ */
+public class SimpleBalancer implements Balancer {
+
+    private static final Logger logger = LoggerFactory.getLogger(SimpleBalancer.class);
+
+    protected final String target1;
+    protected final String target2;
+    protected final DistributedLogClient targetClient1;
+    protected final DistributedLogClient targetClient2;
+    protected final MonitorServiceClient targetMonitor1;
+    protected final MonitorServiceClient targetMonitor2;
+
+    public SimpleBalancer(String name1,
+                          DistributedLogClient client1,
+                          MonitorServiceClient monitor1,
+                          String name2,
+                          DistributedLogClient client2,
+                          MonitorServiceClient monitor2) {
+        this.target1 = name1;
+        this.targetClient1 = client1;
+        this.targetMonitor1 = monitor1;
+        this.target2 = name2;
+        this.targetClient2 = client2;
+        this.targetMonitor2 = monitor2;
+    }
+
+    protected static int countNumberStreams(Map<SocketAddress, Set<String>> distribution) {
+        int count = 0;
+        for (Set<String> streams : distribution.values()) {
+            count += streams.size();
+        }
+        return count;
+    }
+
+    @Override
+    public void balance(int rebalanceWaterMark,
+                        double rebalanceTolerancePercentage,
+                        int rebalanceConcurrency,
+                        Optional<RateLimiter> rebalanceRateLimiter) {
+        // get the ownership distributions from individual targets
+        Map<SocketAddress, Set<String>> distribution1 = targetMonitor1.getStreamOwnershipDistribution();
+        Map<SocketAddress, Set<String>> distribution2 = targetMonitor2.getStreamOwnershipDistribution();
+
+        // get stream counts
+        int proxyCount1 = distribution1.size();
+        int streamCount1 = countNumberStreams(distribution1);
+        int proxyCount2 = distribution2.size();
+        int streamCount2 = countNumberStreams(distribution2);
+
+        logger.info("'{}' has {} streams by {} proxies; while '{}' has {} streams by {} proxies.",
+                    new Object[] {target1, streamCount1, proxyCount1, target2, streamCount2, proxyCount2 });
+
+        String source, target;
+        Map<SocketAddress, Set<String>> srcDistribution;
+        DistributedLogClient srcClient, targetClient;
+        MonitorServiceClient srcMonitor, targetMonitor;
+        int srcStreamCount, targetStreamCount;
+        if (streamCount1 > streamCount2) {
+            source = target1;
+            srcStreamCount = streamCount1;
+            srcClient = targetClient1;
+            srcMonitor = targetMonitor1;
+            srcDistribution = distribution1;
+
+            target = target2;
+            targetStreamCount = streamCount2;
+            targetClient = targetClient2;
+            targetMonitor = targetMonitor2;
+        } else {
+            source = target2;
+            srcStreamCount = streamCount2;
+            srcClient = targetClient2;
+            srcMonitor = targetMonitor2;
+            srcDistribution = distribution2;
+
+            target = target1;
+            targetStreamCount = streamCount1;
+            targetClient = targetClient1;
+            targetMonitor = targetMonitor1;
+        }
+
+        Map<String, Integer> loadDistribution = new HashMap<String, Integer>();
+        loadDistribution.put(source, srcStreamCount);
+        loadDistribution.put(target, targetStreamCount);
+
+        // Calculate how many streams to be rebalanced from src region to target region
+        int numStreamsToRebalance = BalancerUtils.calculateNumStreamsToRebalance(
+            source, loadDistribution, rebalanceWaterMark, rebalanceTolerancePercentage);
+
+        if (numStreamsToRebalance <= 0) {
+            logger.info("No streams need to be rebalanced from '{}' to '{}'.", source, target);
+            return;
+        }
+
+        StreamChooser streamChooser =
+                LimitedStreamChooser.of(new CountBasedStreamChooser(srcDistribution), numStreamsToRebalance);
+        StreamMover streamMover =
+            new StreamMoverImpl(source, srcClient, srcMonitor, target, targetClient, targetMonitor);
+
+        moveStreams(streamChooser, streamMover, rebalanceConcurrency, rebalanceRateLimiter);
+    }
+
+    @Override
+    public void balanceAll(String source,
+                           int rebalanceConcurrency,
+                           Optional<RateLimiter> rebalanceRateLimiter) {
+        String target;
+        DistributedLogClient sourceClient, targetClient;
+        MonitorServiceClient sourceMonitor, targetMonitor;
+        if (target1.equals(source)) {
+            sourceClient = targetClient1;
+            sourceMonitor = targetMonitor1;
+            target = target2;
+            targetClient = targetClient2;
+            targetMonitor = targetMonitor2;
+        } else if (target2.equals(source)) {
+            sourceClient = targetClient2;
+            sourceMonitor = targetMonitor2;
+            target = target1;
+            targetClient = targetClient1;
+            targetMonitor = targetMonitor1;
+        } else {
+            throw new IllegalArgumentException("Unknown target " + source);
+        }
+
+        // get the ownership distributions from individual targets
+        Map<SocketAddress, Set<String>> distribution = sourceMonitor.getStreamOwnershipDistribution();
+
+        if (distribution.isEmpty()) {
+            return;
+        }
+
+        StreamChooser streamChooser = new CountBasedStreamChooser(distribution);
+        StreamMover streamMover =
+            new StreamMoverImpl(source, sourceClient, sourceMonitor, target, targetClient, targetMonitor);
+
+        moveStreams(streamChooser, streamMover, rebalanceConcurrency, rebalanceRateLimiter);
+    }
+
+    private void moveStreams(StreamChooser streamChooser,
+                             StreamMover streamMover,
+                             int concurrency,
+                             Optional<RateLimiter> rateLimiter) {
+        CountDownLatch doneLatch = new CountDownLatch(concurrency);
+        RegionMover regionMover = new RegionMover(streamChooser, streamMover, rateLimiter, doneLatch);
+        ExecutorService executorService = Executors.newFixedThreadPool(concurrency);
+        try {
+            for (int i = 0; i < concurrency; i++) {
+                executorService.submit(regionMover);
+            }
+
+            try {
+                doneLatch.await();
+            } catch (InterruptedException e) {
+                logger.info("{} is interrupted. Stopping it ...", streamMover);
+                regionMover.shutdown();
+            }
+        } finally {
+            executorService.shutdown();
+        }
+
+    }
+
+    /**
+     * Move streams from <i>src</i> region to <i>target</i> region.
+     */
+    static class RegionMover implements Runnable {
+
+        final StreamChooser streamChooser;
+        final StreamMover streamMover;
+        final Optional<RateLimiter> rateLimiter;
+        final CountDownLatch doneLatch;
+        volatile boolean running = true;
+
+        RegionMover(StreamChooser streamChooser,
+                    StreamMover streamMover,
+                    Optional<RateLimiter> rateLimiter,
+                    CountDownLatch doneLatch) {
+            this.streamChooser = streamChooser;
+            this.streamMover = streamMover;
+            this.rateLimiter = rateLimiter;
+            this.doneLatch = doneLatch;
+        }
+
+        @Override
+        public void run() {
+            while (running) {
+                if (rateLimiter.isPresent()) {
+                    rateLimiter.get().acquire();
+                }
+
+                String stream = streamChooser.choose();
+                if (null == stream) {
+                    break;
+                }
+
+                streamMover.moveStream(stream);
+            }
+            doneLatch.countDown();
+        }
+
+        void shutdown() {
+            running = false;
+        }
+    }
+
+    @Override
+    public void close() {
+        // no-op
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamChooser.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamChooser.java
new file mode 100644
index 0000000..1d7b6f7
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamChooser.java
@@ -0,0 +1,30 @@
+/**
+ * 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.service.balancer;
+
+/**
+ * Choose a stream to rebalance.
+ */
+public interface StreamChooser {
+    /**
+     * Choose a stream to rebalance.
+     *
+     * @return stream chose
+     */
+    String choose();
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMover.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMover.java
new file mode 100644
index 0000000..4a04530
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMover.java
@@ -0,0 +1,34 @@
+/**
+ * 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.service.balancer;
+
+/**
+ * A stream mover to move streams between proxies.
+ */
+public interface StreamMover {
+
+    /**
+     * Move given stream <i>streamName</i>.
+     *
+     * @param streamName
+     *          stream name to move
+     * @return <i>true</i> if successfully moved the stream, <i>false</i> when failure happens.
+     * @throws Exception
+     */
+    boolean moveStream(final String streamName);
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMoverImpl.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMoverImpl.java
new file mode 100644
index 0000000..68d934b
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/StreamMoverImpl.java
@@ -0,0 +1,94 @@
+/**
+ * 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.service.balancer;
+
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.service.DistributedLogClient;
+import com.twitter.util.Await;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Move Streams from <i>src</i> to <i>target</i>.
+ */
+public class StreamMoverImpl implements StreamMover {
+
+    private static final Logger logger = LoggerFactory.getLogger(StreamMoverImpl.class);
+
+    final String source, target;
+    final DistributedLogClient srcClient, targetClient;
+    final MonitorServiceClient srcMonitor, targetMonitor;
+
+    public StreamMoverImpl(String source, DistributedLogClient srcClient, MonitorServiceClient srcMonitor,
+                           String target, DistributedLogClient targetClient, MonitorServiceClient targetMonitor) {
+        this.source = source;
+        this.srcClient = srcClient;
+        this.srcMonitor = srcMonitor;
+        this.target = target;
+        this.targetClient = targetClient;
+        this.targetMonitor = targetMonitor;
+    }
+
+    /**
+     * Move given stream <i>streamName</i>.
+     *
+     * @param streamName
+     *          stream name to move
+     * @return <i>true</i> if successfully moved the stream, <i>false</i> when failure happens.
+     * @throws Exception
+     */
+    public boolean moveStream(final String streamName) {
+        try {
+            doMoveStream(streamName);
+            return true;
+        } catch (Exception e) {
+            return false;
+        }
+    }
+
+    private void doMoveStream(final String streamName) throws Exception {
+        Await.result(srcClient.release(streamName).flatMap(new Function<Void, Future<Void>>() {
+            @Override
+            public Future<Void> apply(Void result) {
+                return targetMonitor.check(streamName).addEventListener(new FutureEventListener<Void>() {
+                    @Override
+                    public void onSuccess(Void value) {
+                        logger.info("Moved stream {} from {} to {}.",
+                                new Object[]{streamName, source, target});
+                    }
+
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        logger.info("Failed to move stream {} from region {} to {} : ",
+                                new Object[]{streamName, source, target, cause});
+                    }
+                });
+            }
+        }));
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("StreamMover('").append(source).append("' -> '").append(target).append("')");
+        return sb.toString();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/package-info.java
new file mode 100644
index 0000000..9eb8950
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/balancer/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Balancer to move streams around to balance the traffic.
+ */
+package org.apache.distributedlog.service.balancer;
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java
new file mode 100644
index 0000000..7d72093
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/DefaultStreamConfigProvider.java
@@ -0,0 +1,73 @@
+/**
+ * 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.service.config;
+
+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.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.config.FileConfigurationBuilder;
+import org.apache.distributedlog.config.PropertiesConfigurationBuilder;
+import java.io.File;
+import java.net.MalformedURLException;
+import java.util.List;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.configuration.ConfigurationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * For all streams return the same dynamic config based on configFile.
+ */
+public class DefaultStreamConfigProvider implements StreamConfigProvider {
+
+    private static final Logger LOG = LoggerFactory.getLogger(DefaultStreamConfigProvider.class);
+
+    private final Optional<DynamicDistributedLogConfiguration> dynConf;
+    private final ConfigurationSubscription confSub;
+
+    public DefaultStreamConfigProvider(String configFilePath,
+                                       ScheduledExecutorService executorService,
+                                       int reloadPeriod,
+                                       TimeUnit reloadUnit)
+        throws ConfigurationException {
+        try {
+            File configFile = new File(configFilePath);
+            FileConfigurationBuilder properties =
+                new PropertiesConfigurationBuilder(configFile.toURI().toURL());
+            ConcurrentConstConfiguration defaultConf =
+                new ConcurrentConstConfiguration(new DistributedLogConfiguration());
+            DynamicDistributedLogConfiguration conf =
+                new DynamicDistributedLogConfiguration(defaultConf);
+            List<FileConfigurationBuilder> fileConfigBuilders = Lists.newArrayList(properties);
+            confSub = new ConfigurationSubscription(
+                conf, fileConfigBuilders, executorService, reloadPeriod, reloadUnit);
+            this.dynConf = Optional.of(conf);
+        } catch (MalformedURLException ex) {
+            throw new ConfigurationException(ex);
+        }
+    }
+
+    @Override
+    public Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName) {
+        return dynConf;
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/NullStreamConfigProvider.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/NullStreamConfigProvider.java
new file mode 100644
index 0000000..195f29d
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/NullStreamConfigProvider.java
@@ -0,0 +1,40 @@
+/**
+ * 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.service.config;
+
+import com.google.common.base.Optional;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * For all streams return an absent configuration.
+ */
+public class NullStreamConfigProvider implements StreamConfigProvider {
+    static final Logger LOG = LoggerFactory.getLogger(NullStreamConfigProvider.class);
+
+    private static final Optional<DynamicDistributedLogConfiguration> nullConf =
+            Optional.<DynamicDistributedLogConfiguration>absent();
+
+    @Override
+    public Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName) {
+        return nullConf;
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java
new file mode 100644
index 0000000..257b4be
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServerConfiguration.java
@@ -0,0 +1,443 @@
+/**
+ * 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.service.config;
+
+import static com.google.common.base.Preconditions.checkArgument;
+
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogConstants;
+import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.configuration.CompositeConfiguration;
+import org.apache.commons.configuration.ConfigurationException;
+import org.apache.commons.configuration.SystemConfiguration;
+
+/**
+ * Configuration for DistributedLog Server.
+ */
+public class ServerConfiguration extends CompositeConfiguration {
+
+    private static ClassLoader defaultLoader;
+
+    static {
+        defaultLoader = Thread.currentThread().getContextClassLoader();
+        if (null == defaultLoader) {
+            defaultLoader = DistributedLogConfiguration.class.getClassLoader();
+        }
+    }
+
+    // Server DLSN version
+    protected static final String SERVER_DLSN_VERSION = "server_dlsn_version";
+    protected static final byte SERVER_DLSN_VERSION_DEFAULT = DLSN.VERSION1;
+
+    // Server Durable Write Enable/Disable Flag
+    protected static final String SERVER_DURABLE_WRITE_ENABLED = "server_durable_write_enabled";
+    protected static final boolean SERVER_DURABLE_WRITE_ENABLED_DEFAULT = true;
+
+    // Server Region Id
+    protected static final String SERVER_REGION_ID = "server_region_id";
+    protected static final int SERVER_REGION_ID_DEFAULT = DistributedLogConstants.LOCAL_REGION_ID;
+
+    // Server Port
+    protected static final String SERVER_PORT = "server_port";
+    protected static final int SERVER_PORT_DEFAULT = 0;
+
+    // Server Shard Id
+    protected static final String SERVER_SHARD_ID = "server_shard";
+    protected static final int SERVER_SHARD_ID_DEFAULT = -1;
+
+    // Server Threads
+    protected static final String SERVER_NUM_THREADS = "server_threads";
+    protected static final int SERVER_NUM_THREADS_DEFAULT = Runtime.getRuntime().availableProcessors();
+
+    // Server enable per stream stat
+    protected static final String SERVER_ENABLE_PERSTREAM_STAT = "server_enable_perstream_stat";
+    protected static final boolean SERVER_ENABLE_PERSTREAM_STAT_DEFAULT = true;
+
+    // Server graceful shutdown period (in millis)
+    protected static final String SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS = "server_graceful_shutdown_period_ms";
+    protected static final long SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS_DEFAULT = 0L;
+
+    // Server service timeout
+    public static final String SERVER_SERVICE_TIMEOUT_MS = "server_service_timeout_ms";
+    public static final String SERVER_SERVICE_TIMEOUT_MS_OLD = "serviceTimeoutMs";
+    public static final long SERVER_SERVICE_TIMEOUT_MS_DEFAULT = 0;
+
+    // Server close writer timeout
+    public static final String SERVER_WRITER_CLOSE_TIMEOUT_MS = "server_writer_close_timeout_ms";
+    public static final long SERVER_WRITER_CLOSE_TIMEOUT_MS_DEFAULT = 1000;
+
+    // Server stream probation timeout
+    public static final String SERVER_STREAM_PROBATION_TIMEOUT_MS = "server_stream_probation_timeout_ms";
+    public static final String SERVER_STREAM_PROBATION_TIMEOUT_MS_OLD = "streamProbationTimeoutMs";
+    public static final long SERVER_STREAM_PROBATION_TIMEOUT_MS_DEFAULT = 60 * 1000 * 5;
+
+    // Server stream to partition converter
+    protected static final String SERVER_STREAM_PARTITION_CONVERTER_CLASS = "stream_partition_converter_class";
+
+    // Use hostname as the allocator pool name
+    protected static final String SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME =
+        "server_use_hostname_as_allocator_pool_name";
+    protected static final boolean SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME_DEFAULT = false;
+    //Configure refresh interval for calculating resource placement in seconds
+    public static final String SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_S =
+        "server_resource_placement_refresh_interval_sec";
+    public static final int  SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_DEFAULT = 120;
+
+    public ServerConfiguration() {
+        super();
+        addConfiguration(new SystemConfiguration());
+    }
+
+    /**
+     * Load configurations from {@link DistributedLogConfiguration}.
+     *
+     * @param dlConf
+     *          distributedlog configuration
+     */
+    public void loadConf(DistributedLogConfiguration dlConf) {
+        addConfiguration(dlConf);
+    }
+
+    /**
+     * Set the version to encode dlsn.
+     *
+     * @param version
+     *          dlsn version
+     * @return server configuration
+     */
+    public ServerConfiguration setDlsnVersion(byte version) {
+        setProperty(SERVER_DLSN_VERSION, version);
+        return this;
+    }
+
+    /**
+     * Get the version to encode dlsn.
+     *
+     * @see DLSN
+     * @return version to encode dlsn.
+     */
+    public byte getDlsnVersion() {
+        return getByte(SERVER_DLSN_VERSION, SERVER_DLSN_VERSION_DEFAULT);
+    }
+
+    /**
+     * Set the flag to enable/disable durable write.
+     *
+     * @param enabled
+     *          flag to enable/disable durable write
+     * @return server configuration
+     */
+    public ServerConfiguration enableDurableWrite(boolean enabled) {
+        setProperty(SERVER_DURABLE_WRITE_ENABLED, enabled);
+        return this;
+    }
+
+    /**
+     * Is durable write enabled.
+     *
+     * @return true if waiting writes to be durable. otherwise false.
+     */
+    public boolean isDurableWriteEnabled() {
+        return getBoolean(SERVER_DURABLE_WRITE_ENABLED, SERVER_DURABLE_WRITE_ENABLED_DEFAULT);
+    }
+
+    /**
+     * Set the region id used to instantiate DistributedLogNamespace.
+     *
+     * @param regionId
+     *          region id
+     * @return server configuration
+     */
+    public ServerConfiguration setRegionId(int regionId) {
+        setProperty(SERVER_REGION_ID, regionId);
+        return this;
+    }
+
+    /**
+     * Get the region id used to instantiate {@link org.apache.distributedlog.namespace.DistributedLogNamespace}.
+     *
+     * @return region id used to instantiate DistributedLogNamespace
+     */
+    public int getRegionId() {
+        return getInt(SERVER_REGION_ID, SERVER_REGION_ID_DEFAULT);
+    }
+
+    /**
+     * Set the server port running for this service.
+     *
+     * @param port
+     *          server port
+     * @return server configuration
+     */
+    public ServerConfiguration setServerPort(int port) {
+        setProperty(SERVER_PORT, port);
+        return this;
+    }
+
+    /**
+     * Get the server port running for this service.
+     *
+     * @return server port
+     */
+    public int getServerPort() {
+        return getInt(SERVER_PORT, SERVER_PORT_DEFAULT);
+    }
+
+    /**
+     * Set the shard id of this server.
+     *
+     * @param shardId
+     *          shard id
+     * @return shard id of this server
+     */
+    public ServerConfiguration setServerShardId(int shardId) {
+        setProperty(SERVER_SHARD_ID, shardId);
+        return this;
+    }
+
+    /**
+     * Get the shard id of this server.
+     *
+     * <p>It would be used to instantiate the client id used for DistributedLogNamespace.
+     *
+     * @return shard id of this server.
+     */
+    public int getServerShardId() {
+        return getInt(SERVER_SHARD_ID, SERVER_SHARD_ID_DEFAULT);
+    }
+
+    /**
+     * Get the number of threads for the executor of this server.
+     *
+     * @return number of threads for the executor running in this server.
+     */
+    public int getServerThreads() {
+        return getInt(SERVER_NUM_THREADS, SERVER_NUM_THREADS_DEFAULT);
+    }
+
+    /**
+     * Set the number of threads for the executor of this server.
+     *
+     * @param numThreads
+     *          number of threads for the executor running in this server.
+     * @return server configuration
+     */
+    public ServerConfiguration setServerThreads(int numThreads) {
+        setProperty(SERVER_NUM_THREADS, numThreads);
+        return this;
+    }
+
+    /**
+     * Enable/Disable per stream stat.
+     *
+     * @param enabled
+     *          flag to enable/disable per stream stat
+     * @return server configuration
+     */
+    public ServerConfiguration setPerStreamStatEnabled(boolean enabled) {
+        setProperty(SERVER_ENABLE_PERSTREAM_STAT, enabled);
+        return this;
+    }
+
+    /**
+     * Whether the per stream stat enabled for not in this server.
+     *
+     * @return true if per stream stat enable, otherwise false.
+     */
+    public boolean isPerStreamStatEnabled() {
+        return getBoolean(SERVER_ENABLE_PERSTREAM_STAT, SERVER_ENABLE_PERSTREAM_STAT_DEFAULT);
+    }
+
+    /**
+     * Set the graceful shutdown period in millis.
+     *
+     * @param periodMs
+     *          graceful shutdown period in millis.
+     * @return server configuration
+     */
+    public ServerConfiguration setGracefulShutdownPeriodMs(long periodMs) {
+        setProperty(SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS, periodMs);
+        return this;
+    }
+
+    /**
+     * Get the graceful shutdown period in millis.
+     *
+     * @return graceful shutdown period in millis.
+     */
+    public long getGracefulShutdownPeriodMs() {
+        return getLong(SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS, SERVER_GRACEFUL_SHUTDOWN_PERIOD_MS_DEFAULT);
+    }
+
+    /**
+     * Get timeout for stream op execution in proxy layer.
+     *
+     * <p>0 disables timeout.
+     *
+     * @return timeout for stream operation in proxy layer.
+     */
+    public long getServiceTimeoutMs() {
+        return getLong(SERVER_SERVICE_TIMEOUT_MS,
+                getLong(SERVER_SERVICE_TIMEOUT_MS_OLD, SERVER_SERVICE_TIMEOUT_MS_DEFAULT));
+    }
+
+    /**
+     * Set timeout for stream op execution in proxy layer.
+     *
+     * <p>0 disables timeout.
+     *
+     * @param timeoutMs
+     *          timeout for stream operation in proxy layer.
+     * @return dl configuration.
+     */
+    public ServerConfiguration setServiceTimeoutMs(long timeoutMs) {
+        setProperty(SERVER_SERVICE_TIMEOUT_MS, timeoutMs);
+        return this;
+    }
+
+    /**
+     * Get timeout for closing writer in proxy layer.
+     *
+     * <p>0 disables timeout.
+     *
+     * @return timeout for closing writer in proxy layer.
+     */
+    public long getWriterCloseTimeoutMs() {
+        return getLong(SERVER_WRITER_CLOSE_TIMEOUT_MS, SERVER_WRITER_CLOSE_TIMEOUT_MS_DEFAULT);
+    }
+
+    /**
+     * Set timeout for closing writer in proxy layer.
+     *
+     * <p>0 disables timeout.
+     *
+     * @param timeoutMs
+     *          timeout for closing writer in proxy layer.
+     * @return dl configuration.
+     */
+    public ServerConfiguration setWriterCloseTimeoutMs(long timeoutMs) {
+        setProperty(SERVER_WRITER_CLOSE_TIMEOUT_MS, timeoutMs);
+        return this;
+    }
+
+    /**
+     * How long should stream be kept in cache in probationary state after service timeout.
+     *
+     * <p>The setting is to prevent reacquire. The unit of this setting is milliseconds.
+     *
+     * @return stream probation timeout in ms.
+     */
+    public long getStreamProbationTimeoutMs() {
+        return getLong(SERVER_STREAM_PROBATION_TIMEOUT_MS,
+                getLong(SERVER_STREAM_PROBATION_TIMEOUT_MS_OLD, SERVER_STREAM_PROBATION_TIMEOUT_MS_DEFAULT));
+    }
+
+    /**
+     * How long should stream be kept in cache in probationary state after service timeout.
+     *
+     * <p>The setting is to prevent reacquire. The unit of this setting is milliseconds.
+     *
+     * @param timeoutMs probation timeout in ms.
+     * @return server configuration
+     */
+    public ServerConfiguration setStreamProbationTimeoutMs(long timeoutMs) {
+        setProperty(SERVER_STREAM_PROBATION_TIMEOUT_MS, timeoutMs);
+        return this;
+    }
+
+    /**
+     * Set the stream partition converter class.
+     *
+     * @param converterClass
+     *          stream partition converter class
+     * @return server configuration
+     */
+    public ServerConfiguration setStreamPartitionConverterClass(
+        Class<? extends StreamPartitionConverter> converterClass) {
+        setProperty(SERVER_STREAM_PARTITION_CONVERTER_CLASS, converterClass.getName());
+        return this;
+    }
+
+    /**
+     * Get the stream partition converter class.
+     *
+     * @return the stream partition converter class.
+     * @throws ConfigurationException
+     */
+    public Class<? extends StreamPartitionConverter> getStreamPartitionConverterClass()
+            throws ConfigurationException {
+        return ReflectionUtils.getClass(
+                this,
+                SERVER_STREAM_PARTITION_CONVERTER_CLASS,
+                IdentityStreamPartitionConverter.class,
+                StreamPartitionConverter.class,
+                defaultLoader);
+    }
+
+     /**
+      * Set if use hostname as the allocator pool name.
+      *
+      * @param useHostname whether to use hostname as the allocator pool name.
+      * @return server configuration
+      * @see #isUseHostnameAsAllocatorPoolName()
+      */
+    public ServerConfiguration setUseHostnameAsAllocatorPoolName(boolean useHostname) {
+        setProperty(SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME, useHostname);
+        return this;
+    }
+
+    /**
+     * Get if use hostname as the allocator pool name.
+     *
+     * @return true if use hostname as the allocator pool name. otherwise, use
+     * {@link #getServerShardId()} as the allocator pool name.
+     * @see #getServerShardId()
+     */
+    public boolean isUseHostnameAsAllocatorPoolName() {
+        return getBoolean(SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME,
+            SERVER_USE_HOSTNAME_AS_ALLOCATOR_POOL_NAME_DEFAULT);
+    }
+
+    public ServerConfiguration setResourcePlacementRefreshInterval(int refreshIntervalSecs) {
+        setProperty(SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_S, refreshIntervalSecs);
+        return this;
+    }
+
+    public int getResourcePlacementRefreshInterval() {
+        return getInt(SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_S, SERVER_RESOURCE_PLACEMENT_REFRESH_INTERVAL_DEFAULT);
+    }
+
+    /**
+     * Validate the configuration.
+     *
+     * @throws IllegalStateException when there are any invalid settings.
+     */
+    public void validate() {
+        byte dlsnVersion = getDlsnVersion();
+        checkArgument(dlsnVersion >= DLSN.VERSION0 && dlsnVersion <= DLSN.VERSION1,
+                "Unknown dlsn version " + dlsnVersion);
+        checkArgument(getServerThreads() > 0,
+                "Invalid number of server threads : " + getServerThreads());
+        checkArgument(getServerShardId() >= 0,
+                "Invalid server shard id : " + getServerShardId());
+    }
+
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServiceStreamConfigProvider.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServiceStreamConfigProvider.java
new file mode 100644
index 0000000..29052f9
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/ServiceStreamConfigProvider.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.service.config;
+
+import com.google.common.base.Optional;
+import org.apache.distributedlog.config.DynamicConfigurationFactory;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import java.io.File;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.apache.commons.configuration.ConfigurationException;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Provide per stream configuration to DistributedLog service layer.
+ */
+public class ServiceStreamConfigProvider implements StreamConfigProvider {
+
+    private static final Logger LOG = LoggerFactory.getLogger(ServiceStreamConfigProvider.class);
+
+    private static final String CONFIG_EXTENSION = "conf";
+
+    private final File configBaseDir;
+    private final File defaultConfigFile;
+    private final StreamPartitionConverter partitionConverter;
+    private final DynamicConfigurationFactory configFactory;
+    private final DynamicDistributedLogConfiguration defaultDynConf;
+
+    public ServiceStreamConfigProvider(String configPath,
+                                       String defaultConfigPath,
+                                       StreamPartitionConverter partitionConverter,
+                                       ScheduledExecutorService executorService,
+                                       int reloadPeriod,
+                                       TimeUnit reloadUnit)
+                                       throws ConfigurationException {
+        this.configBaseDir = new File(configPath);
+        if (!configBaseDir.exists()) {
+            throw new ConfigurationException("Stream configuration base directory "
+                + configPath + " does not exist");
+        }
+        this.defaultConfigFile = new File(configPath);
+        if (!defaultConfigFile.exists()) {
+            throw new ConfigurationException("Stream configuration default config "
+                + defaultConfigPath + " does not exist");
+        }
+
+        // Construct reloading default configuration
+        this.partitionConverter = partitionConverter;
+        this.configFactory = new DynamicConfigurationFactory(executorService, reloadPeriod, reloadUnit);
+        // We know it exists from the check above.
+        this.defaultDynConf = configFactory.getDynamicConfiguration(defaultConfigPath).get();
+    }
+
+    @Override
+    public Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName) {
+        String configName = partitionConverter.convert(streamName).getStream();
+        String configPath = getConfigPath(configName);
+        Optional<DynamicDistributedLogConfiguration> dynConf = Optional.<DynamicDistributedLogConfiguration>absent();
+        try {
+            dynConf = configFactory.getDynamicConfiguration(configPath, defaultDynConf);
+        } catch (ConfigurationException ex) {
+            LOG.warn("Configuration exception for stream {} ({}) at {}",
+                    new Object[] {streamName, configName, configPath, ex});
+        }
+        return dynConf;
+    }
+
+    private String getConfigPath(String configName) {
+        return new File(configBaseDir, String.format("%s.%s", configName, CONFIG_EXTENSION)).getPath();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/StreamConfigProvider.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/StreamConfigProvider.java
new file mode 100644
index 0000000..c704f70
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/StreamConfigProvider.java
@@ -0,0 +1,34 @@
+/**
+ * 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.service.config;
+
+import com.google.common.base.Optional;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+
+/**
+ * Expose per-stream configs to dl proxy.
+ */
+public interface StreamConfigProvider {
+    /**
+     * Get dynamic per stream config overrides for a given stream.
+     *
+     * @param streamName stream name to return config for
+     * @return Optional dynamic configuration instance
+     */
+    Optional<DynamicDistributedLogConfiguration> getDynamicStreamConfig(String streamName);
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/package-info.java
new file mode 100644
index 0000000..b07605e
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/config/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * DistributedLog Server Configurations.
+ */
+package org.apache.distributedlog.service.config;
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/package-info.java
new file mode 100644
index 0000000..3fcfeda
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * DistributedLog Proxy Service.
+ */
+package org.apache.distributedlog.service;
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/EqualLoadAppraiser.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/EqualLoadAppraiser.java
new file mode 100644
index 0000000..fa3dd49
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/EqualLoadAppraiser.java
@@ -0,0 +1,39 @@
+/**
+ * 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.service.placement;
+
+import com.twitter.util.Future;
+
+/**
+ * Equal Load Appraiser.
+ *
+ * <p>Created for those who hold these truths to be self-evident, that all streams are created equal,
+ * that they are endowed by their creator with certain unalienable loads, that among these are
+ * Uno, Eins, and One.
+ */
+public class EqualLoadAppraiser implements LoadAppraiser {
+    @Override
+    public Future<StreamLoad> getStreamLoad(String stream) {
+        return Future.value(new StreamLoad(stream, 1));
+    }
+
+    @Override
+    public Future<Void> refreshCache() {
+        return Future.value(null);
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java
new file mode 100644
index 0000000..2e9dd6b
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LeastLoadPlacementPolicy.java
@@ -0,0 +1,200 @@
+/**
+ * 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.service.placement;
+
+import org.apache.distributedlog.client.routing.RoutingService;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import com.twitter.util.Duration;
+import com.twitter.util.Function;
+import com.twitter.util.Future;
+import com.twitter.util.Futures;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.TreeSet;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Least Load Placement Policy.
+ *
+ * <p>A LoadPlacementPolicy that attempts to place streams in such a way that the load is balanced as
+ * evenly as possible across all shards. The LoadAppraiser remains responsible for determining what
+ * the load of a server would be. This placement policy then distributes these streams across the
+ * servers.
+ */
+public class LeastLoadPlacementPolicy extends PlacementPolicy {
+
+    private static final Logger logger = LoggerFactory.getLogger(LeastLoadPlacementPolicy.class);
+
+    private TreeSet<ServerLoad> serverLoads = new TreeSet<ServerLoad>();
+    private Map<String, String> streamToServer = new HashMap<String, String>();
+
+    public LeastLoadPlacementPolicy(LoadAppraiser loadAppraiser, RoutingService routingService,
+                                    DistributedLogNamespace namespace, PlacementStateManager placementStateManager,
+                                    Duration refreshInterval, StatsLogger statsLogger) {
+        super(loadAppraiser, routingService, namespace, placementStateManager, refreshInterval, statsLogger);
+        statsLogger.registerGauge("placement/load.diff", new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return 0;
+            }
+
+            @Override
+            public Number getSample() {
+                if (serverLoads.size() > 0) {
+                    return serverLoads.last().getLoad() - serverLoads.first().getLoad();
+                } else {
+                    return getDefaultValue();
+                }
+            }
+        });
+    }
+
+    private synchronized String getStreamOwner(String stream) {
+        return streamToServer.get(stream);
+    }
+
+    @Override
+    public Future<String> placeStream(String stream) {
+        String streamOwner = getStreamOwner(stream);
+        if (null != streamOwner) {
+            return Future.value(streamOwner);
+        }
+        Future<StreamLoad> streamLoadFuture = loadAppraiser.getStreamLoad(stream);
+        return streamLoadFuture.map(new Function<StreamLoad, String>() {
+            @Override
+            public String apply(StreamLoad streamLoad) {
+                return placeStreamSynchronized(streamLoad);
+            }
+        });
+    }
+
+    private synchronized String placeStreamSynchronized(StreamLoad streamLoad) {
+        ServerLoad serverLoad = serverLoads.pollFirst();
+        serverLoad.addStream(streamLoad);
+        serverLoads.add(serverLoad);
+        return serverLoad.getServer();
+    }
+
+    @Override
+    public void refresh() {
+        logger.info("Refreshing server loads.");
+        Future<Void> refresh = loadAppraiser.refreshCache();
+        final Set<String> servers = getServers();
+        final Set<String> allStreams = getStreams();
+        Future<TreeSet<ServerLoad>> serverLoadsFuture = refresh.flatMap(
+            new Function<Void, Future<TreeSet<ServerLoad>>>() {
+            @Override
+            public Future<TreeSet<ServerLoad>> apply(Void v1) {
+                return calculate(servers, allStreams);
+            }
+        });
+        serverLoadsFuture.map(new Function<TreeSet<ServerLoad>, BoxedUnit>() {
+            @Override
+            public BoxedUnit apply(TreeSet<ServerLoad> serverLoads) {
+                try {
+                    updateServerLoads(serverLoads);
+                } catch (PlacementStateManager.StateManagerSaveException e) {
+                    logger.error("The refreshed mapping could not be persisted and will not be used.", e);
+                }
+                return BoxedUnit.UNIT;
+            }
+        });
+    }
+
+    private synchronized void updateServerLoads(TreeSet<ServerLoad> serverLoads)
+        throws PlacementStateManager.StateManagerSaveException {
+        this.placementStateManager.saveOwnership(serverLoads);
+        this.streamToServer = serverLoadsToMap(serverLoads);
+        this.serverLoads = serverLoads;
+    }
+
+    @Override
+    public synchronized void load(TreeSet<ServerLoad> serverLoads) {
+        this.serverLoads = serverLoads;
+        this.streamToServer = serverLoadsToMap(serverLoads);
+    }
+
+    public Future<TreeSet<ServerLoad>> calculate(final Set<String> servers, Set<String> streams) {
+        logger.info("Calculating server loads");
+        final long startTime = System.currentTimeMillis();
+        ArrayList<Future<StreamLoad>> futures = new ArrayList<Future<StreamLoad>>(streams.size());
+
+        for (String stream : streams) {
+            Future<StreamLoad> streamLoad = loadAppraiser.getStreamLoad(stream);
+            futures.add(streamLoad);
+        }
+
+        return Futures.collect(futures).map(new Function<List<StreamLoad>, TreeSet<ServerLoad>>() {
+            @Override
+            public TreeSet<ServerLoad> apply(List<StreamLoad> streamLoads) {
+        /* Sort streamLoads so largest streams are placed first for better balance */
+                TreeSet<StreamLoad> streamQueue = new TreeSet<StreamLoad>();
+                for (StreamLoad streamLoad : streamLoads) {
+                    streamQueue.add(streamLoad);
+                }
+
+                TreeSet<ServerLoad> serverLoads = new TreeSet<ServerLoad>();
+                for (String server : servers) {
+                    ServerLoad serverLoad = new ServerLoad(server);
+                    if (!streamQueue.isEmpty()) {
+                        serverLoad.addStream(streamQueue.pollFirst());
+                    }
+                    serverLoads.add(serverLoad);
+                }
+
+                while (!streamQueue.isEmpty()) {
+                    ServerLoad serverLoad = serverLoads.pollFirst();
+                    serverLoad.addStream(streamQueue.pollFirst());
+                    serverLoads.add(serverLoad);
+                }
+                return serverLoads;
+            }
+        }).onSuccess(new Function<TreeSet<ServerLoad>, BoxedUnit>() {
+            @Override
+            public BoxedUnit apply(TreeSet<ServerLoad> serverLoads) {
+                placementCalcStats.registerSuccessfulEvent(System.currentTimeMillis() - startTime);
+                return BoxedUnit.UNIT;
+            }
+        }).onFailure(new Function<Throwable, BoxedUnit>() {
+            @Override
+            public BoxedUnit apply(Throwable t) {
+                logger.error("Failure calculating loads", t);
+                placementCalcStats.registerFailedEvent(System.currentTimeMillis() - startTime);
+                return BoxedUnit.UNIT;
+            }
+        });
+    }
+
+    private static Map<String, String> serverLoadsToMap(Collection<ServerLoad> serverLoads) {
+        HashMap<String, String> streamToServer = new HashMap<String, String>(serverLoads.size());
+        for (ServerLoad serverLoad : serverLoads) {
+            for (StreamLoad streamLoad : serverLoad.getStreamLoads()) {
+                streamToServer.put(streamLoad.getStream(), serverLoad.getServer());
+            }
+        }
+        return streamToServer;
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LoadAppraiser.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LoadAppraiser.java
new file mode 100644
index 0000000..5cd8980
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/LoadAppraiser.java
@@ -0,0 +1,39 @@
+/**
+ * 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.service.placement;
+
+import com.twitter.util.Future;
+
+/**
+ * Interface for load appraiser.
+ */
+public interface LoadAppraiser {
+    /**
+     * Retrieve the stream load for a given {@code stream}.
+     *
+     * @param stream name of the stream
+     * @return the stream load of the stream.
+     */
+    Future<StreamLoad> getStreamLoad(String stream);
+
+    /**
+     * Refesch the cache.
+     * @return
+     */
+    Future<Void> refreshCache();
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java
new file mode 100644
index 0000000..ac952aa
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementPolicy.java
@@ -0,0 +1,148 @@
+/**
+ * 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.service.placement;
+
+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;
+import com.twitter.util.Future;
+import com.twitter.util.ScheduledThreadPoolTimer;
+import com.twitter.util.Time;
+import com.twitter.util.Timer;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+import java.util.TreeSet;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.BoxedUnit;
+
+/**
+ * A PlacementPolicy assigns streams to servers given an appraisal of the load that the stream contains.
+ *
+ * <p>The load of a stream is determined by the LoadAppraiser used. The PlacementPolicy will
+ * then distributed these StreamLoads to the available servers in a manner defined by the
+ * implementation creating ServerLoad objects. It then saves this assignment via the
+ * PlacementStateManager.
+ */
+public abstract class PlacementPolicy {
+
+    private static final Logger logger = LoggerFactory.getLogger(PlacementPolicy.class);
+
+    protected final LoadAppraiser loadAppraiser;
+    protected final RoutingService routingService;
+    protected final DistributedLogNamespace 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,
+                           Duration refreshInterval, StatsLogger statsLogger) {
+        this.loadAppraiser = loadAppraiser;
+        this.routingService = routingService;
+        this.namespace = namespace;
+        this.placementStateManager = placementStateManager;
+        this.refreshInterval = refreshInterval;
+        placementCalcStats = statsLogger.getOpStatsLogger("placement");
+    }
+
+    public Set<String> getServers() {
+        Set<SocketAddress> hosts = routingService.getHosts();
+        Set<String> servers = new HashSet<String>(hosts.size());
+        for (SocketAddress address : hosts) {
+            servers.add(DLSocketAddress.toString((InetSocketAddress) address));
+        }
+        return servers;
+    }
+
+    public Set<String> getStreams() {
+        Set<String> streams = new HashSet<String>();
+        try {
+            Iterator<String> logs = namespace.getLogs();
+            while (logs.hasNext()) {
+                streams.add(logs.next());
+            }
+        } catch (IOException e) {
+            logger.error("Could not get streams for placement policy.", e);
+        }
+        return streams;
+    }
+
+    public void start(boolean leader) {
+        logger.info("Starting placement policy");
+
+        TreeSet<ServerLoad> emptyServerLoads = new TreeSet<ServerLoad>();
+        for (String server : getServers()) {
+            emptyServerLoads.add(new ServerLoad(server));
+        }
+        load(emptyServerLoads); //Pre-Load so streams don't NPE
+        if (leader) { //this is the leader shard
+            logger.info("Shard is leader. Scheduling timed refresh.");
+            placementRefreshTimer = new ScheduledThreadPoolTimer(1, "timer", true);
+            placementRefreshTimer.schedule(Time.now(), refreshInterval, new Function0<BoxedUnit>() {
+                @Override
+                public BoxedUnit apply() {
+                    refresh();
+                    return BoxedUnit.UNIT;
+                }
+            });
+        } else {
+            logger.info("Shard is not leader. Watching for server load changes.");
+            placementStateManager.watch(new PlacementStateManager.PlacementCallback() {
+                @Override
+                public void callback(TreeSet<ServerLoad> serverLoads) {
+                    if (!serverLoads.isEmpty()) {
+                        load(serverLoads);
+                    }
+                }
+            });
+        }
+    }
+
+    public void close() {
+        if (placementRefreshTimer != null) {
+            placementRefreshTimer.stop();
+        }
+    }
+
+    /**
+     * Places the stream on a server according to the policy.
+     *
+     * <p>It returns a future containing the host that owns the stream upon completion
+     */
+    public abstract Future<String> placeStream(String stream);
+
+    /**
+     * Recalculates the entire placement mapping and updates stores it using the PlacementStateManager.
+     */
+    public abstract void refresh();
+
+    /**
+     * Loads the placement mapping into the node from a TreeSet of ServerLoads.
+     */
+    public abstract void load(TreeSet<ServerLoad> serverLoads);
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementStateManager.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementStateManager.java
new file mode 100644
index 0000000..0187bed
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/PlacementStateManager.java
@@ -0,0 +1,79 @@
+/**
+ * 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.service.placement;
+
+import java.util.TreeSet;
+
+/**
+ * The PlacementStateManager handles persistence of calculated resource placements.
+ */
+public interface PlacementStateManager {
+
+    /**
+     * Saves the ownership mapping as a TreeSet of ServerLoads to persistent storage.
+     */
+    void saveOwnership(TreeSet<ServerLoad> serverLoads) throws StateManagerSaveException;
+
+    /**
+     * Loads the ownership mapping as TreeSet of ServerLoads from persistent storage.
+     */
+    TreeSet<ServerLoad> loadOwnership() throws StateManagerLoadException;
+
+    /**
+     * Watch the persistent storage for changes to the ownership mapping.
+     *
+     * <p>The placementCallback callbacks will be triggered with the new mapping when a change occurs.
+     */
+    void watch(PlacementCallback placementCallback);
+
+    /**
+     * Placement Callback.
+     *
+     * <p>The callback is triggered when server loads are updated.
+     */
+    interface PlacementCallback {
+        void callback(TreeSet<ServerLoad> serverLoads);
+    }
+
+    /**
+     * The base exception thrown when state manager encounters errors.
+     */
+    abstract class StateManagerException extends Exception {
+        public StateManagerException(String message, Exception e) {
+            super(message, e);
+        }
+    }
+
+    /**
+     * Exception thrown when failed to load the ownership mapping.
+     */
+    class StateManagerLoadException extends StateManagerException {
+        public StateManagerLoadException(Exception e) {
+            super("Load of Ownership failed", e);
+        }
+    }
+
+    /**
+     * Exception thrown when failed to save the ownership mapping.
+     */
+    class StateManagerSaveException extends StateManagerException {
+        public StateManagerSaveException(Exception e) {
+            super("Save of Ownership failed", e);
+        }
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ServerLoad.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ServerLoad.java
new file mode 100644
index 0000000..d65c401
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ServerLoad.java
@@ -0,0 +1,158 @@
+/**
+ * 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.service.placement;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.Set;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TJSONProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.apache.thrift.transport.TMemoryInputTransport;
+
+/**
+ * An object represents the server load.
+ *
+ * <p>A comparable data object containing the identifier of the server, total appraised load on the
+ * server, and all streams assigned to the server by the resource placement mapping. This is
+ * comparable first by load and then by server so that a sorted data structure of these will be
+ * consistent across multiple calculations.
+ */
+public class ServerLoad implements Comparable {
+    private static final int BUFFER_SIZE = 4096000;
+    private final String server;
+    private final HashSet<StreamLoad> streamLoads = new HashSet<StreamLoad>();
+    private long load = 0L;
+
+    public ServerLoad(String server) {
+        this.server = server;
+    }
+
+    public synchronized long addStream(StreamLoad stream) {
+        this.load += stream.getLoad();
+        streamLoads.add(stream);
+        return this.load;
+    }
+
+    public synchronized long removeStream(String stream) {
+        for (StreamLoad streamLoad : streamLoads) {
+            if (streamLoad.stream.equals(stream)) {
+                this.load -= streamLoad.getLoad();
+                streamLoads.remove(streamLoad);
+                return this.load;
+            }
+        }
+        return this.load; //Throwing an exception wouldn't help us as our logic should never reach here
+    }
+
+    public synchronized long getLoad() {
+        return load;
+    }
+
+    public synchronized Set<StreamLoad> getStreamLoads() {
+        return streamLoads;
+    }
+
+    public synchronized String getServer() {
+        return server;
+    }
+
+    protected synchronized org.apache.distributedlog.service.placement.thrift.ServerLoad toThrift() {
+        org.apache.distributedlog.service.placement.thrift.ServerLoad tServerLoad =
+            new org.apache.distributedlog.service.placement.thrift.ServerLoad();
+        tServerLoad.setServer(server);
+        tServerLoad.setLoad(load);
+        ArrayList<org.apache.distributedlog.service.placement.thrift.StreamLoad> tStreamLoads =
+            new ArrayList<org.apache.distributedlog.service.placement.thrift.StreamLoad>();
+        for (StreamLoad streamLoad : streamLoads) {
+            tStreamLoads.add(streamLoad.toThrift());
+        }
+        tServerLoad.setStreams(tStreamLoads);
+        return tServerLoad;
+    }
+
+    public byte[] serialize() throws IOException {
+        TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
+        TJSONProtocol protocol = new TJSONProtocol(transport);
+        try {
+            toThrift().write(protocol);
+            transport.flush();
+            return transport.toString(UTF_8.name()).getBytes(UTF_8);
+        } catch (TException e) {
+            throw new IOException("Failed to serialize server load : ", e);
+        } catch (UnsupportedEncodingException uee) {
+            throw new IOException("Failed to serialize server load : ", uee);
+        }
+    }
+
+    public static ServerLoad deserialize(byte[] data) throws IOException {
+        org.apache.distributedlog.service.placement.thrift.ServerLoad tServerLoad =
+            new org.apache.distributedlog.service.placement.thrift.ServerLoad();
+        TMemoryInputTransport transport = new TMemoryInputTransport(data);
+        TJSONProtocol protocol = new TJSONProtocol(transport);
+        try {
+            tServerLoad.read(protocol);
+            ServerLoad serverLoad = new ServerLoad(tServerLoad.getServer());
+            if (tServerLoad.isSetStreams()) {
+                for (org.apache.distributedlog.service.placement.thrift.StreamLoad tStreamLoad :
+                    tServerLoad.getStreams()) {
+                    serverLoad.addStream(new StreamLoad(tStreamLoad.getStream(), tStreamLoad.getLoad()));
+                }
+            }
+            return serverLoad;
+        } catch (TException e) {
+            throw new IOException("Failed to deserialize server load : ", e);
+        }
+    }
+
+    @Override
+    public synchronized int compareTo(Object o) {
+        ServerLoad other = (ServerLoad) o;
+        if (load == other.getLoad()) {
+            return server.compareTo(other.getServer());
+        } else {
+            return Long.compare(load, other.getLoad());
+        }
+    }
+
+    @Override
+    public synchronized boolean equals(Object o) {
+        if (!(o instanceof ServerLoad)) {
+            return false;
+        }
+        ServerLoad other = (ServerLoad) o;
+        return server.equals(other.getServer())
+            && load == other.getLoad()
+            && streamLoads.equals(other.getStreamLoads());
+    }
+
+    @Override
+    public synchronized String toString() {
+        return String.format("ServerLoad<Server: %s, Load: %d, Streams: %s>", server, load, streamLoads);
+    }
+
+    @Override
+    public synchronized int hashCode() {
+        return new HashCodeBuilder().append(server).append(load).append(streamLoads).build();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/StreamLoad.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/StreamLoad.java
new file mode 100644
index 0000000..f271222
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/StreamLoad.java
@@ -0,0 +1,115 @@
+/**
+ * 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.service.placement;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+import java.io.IOException;
+import java.io.UnsupportedEncodingException;
+import org.apache.commons.lang3.builder.HashCodeBuilder;
+import org.apache.thrift.TException;
+import org.apache.thrift.protocol.TJSONProtocol;
+import org.apache.thrift.transport.TMemoryBuffer;
+import org.apache.thrift.transport.TMemoryInputTransport;
+
+/**
+ * An object represent the load of a stream.
+ *
+ * <p>A comparable data object containing the identifier of the stream and the appraised load produced
+ * by the stream.
+ */
+public class StreamLoad implements Comparable {
+    private static final int BUFFER_SIZE = 4096;
+    public final String stream;
+    private final int load;
+
+    public StreamLoad(String stream, int load) {
+        this.stream = stream;
+        this.load = load;
+    }
+
+    public int getLoad() {
+        return load;
+    }
+
+    public String getStream() {
+        return stream;
+    }
+
+    protected org.apache.distributedlog.service.placement.thrift.StreamLoad toThrift() {
+        org.apache.distributedlog.service.placement.thrift.StreamLoad tStreamLoad =
+            new org.apache.distributedlog.service.placement.thrift.StreamLoad();
+        return tStreamLoad.setStream(stream).setLoad(load);
+    }
+
+    public byte[] serialize() throws IOException {
+        TMemoryBuffer transport = new TMemoryBuffer(BUFFER_SIZE);
+        TJSONProtocol protocol = new TJSONProtocol(transport);
+        try {
+            toThrift().write(protocol);
+            transport.flush();
+            return transport.toString(UTF_8.name()).getBytes(UTF_8);
+        } catch (TException e) {
+            throw new IOException("Failed to serialize stream load : ", e);
+        } catch (UnsupportedEncodingException uee) {
+            throw new IOException("Failed to serialize stream load : ", uee);
+        }
+    }
+
+    public static StreamLoad deserialize(byte[] data) throws IOException {
+        org.apache.distributedlog.service.placement.thrift.StreamLoad tStreamLoad =
+            new org.apache.distributedlog.service.placement.thrift.StreamLoad();
+        TMemoryInputTransport transport = new TMemoryInputTransport(data);
+        TJSONProtocol protocol = new TJSONProtocol(transport);
+        try {
+            tStreamLoad.read(protocol);
+            return new StreamLoad(tStreamLoad.getStream(), tStreamLoad.getLoad());
+        } catch (TException e) {
+            throw new IOException("Failed to deserialize stream load : ", e);
+        }
+    }
+
+    @Override
+    public int compareTo(Object o) {
+        StreamLoad other = (StreamLoad) o;
+        if (load == other.getLoad()) {
+            return stream.compareTo(other.getStream());
+        } else {
+            return Long.compare(load, other.getLoad());
+        }
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (!(o instanceof StreamLoad)) {
+            return false;
+        }
+        StreamLoad other = (StreamLoad) o;
+        return stream.equals(other.getStream()) && load == other.getLoad();
+    }
+
+    @Override
+    public String toString() {
+        return String.format("StreamLoad<Stream: %s, Load: %d>", stream, load);
+    }
+
+    @Override
+    public int hashCode() {
+        return new HashCodeBuilder().append(stream).append(load).build();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java
new file mode 100644
index 0000000..862f05a
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/ZKPlacementStateManager.java
@@ -0,0 +1,173 @@
+/**
+ * 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.service.placement;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.ZooKeeperClient;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.distributedlog.util.Utils;
+import java.io.IOException;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.List;
+import java.util.TreeSet;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.Transaction;
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+import org.apache.zookeeper.data.Stat;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * An implementation of the PlacementStateManager that saves data to and loads from Zookeeper to
+ * avoid necessitating an additional system for the resource placement.
+ */
+public class ZKPlacementStateManager implements PlacementStateManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(ZKPlacementStateManager.class);
+
+    private static final String SERVER_LOAD_DIR = "/.server-load";
+
+    private final String serverLoadPath;
+    private final ZooKeeperClient zkClient;
+
+    private boolean watching = false;
+
+    public ZKPlacementStateManager(URI uri, DistributedLogConfiguration conf, StatsLogger statsLogger) {
+        String zkServers = BKNamespaceDriver.getZKServersFromDLUri(uri);
+        zkClient = BKNamespaceDriver.createZKClientBuilder(
+            String.format("ZKPlacementStateManager-%s", zkServers),
+            conf,
+            zkServers,
+            statsLogger.scope("placement_state_manager")).build();
+        serverLoadPath = uri.getPath() + SERVER_LOAD_DIR;
+    }
+
+    private void createServerLoadPathIfNoExists(byte[] data)
+        throws ZooKeeperClient.ZooKeeperConnectionException, KeeperException, InterruptedException {
+        try {
+            Utils.zkCreateFullPathOptimistic(
+                zkClient, serverLoadPath, data, zkClient.getDefaultACL(), CreateMode.PERSISTENT);
+        } catch (KeeperException.NodeExistsException nee) {
+            logger.debug("the server load path {} is already created by others", serverLoadPath, nee);
+        }
+    }
+
+    @Override
+    public void saveOwnership(TreeSet<ServerLoad> serverLoads) throws StateManagerSaveException {
+        logger.info("saving ownership");
+        try {
+            ZooKeeper zk = zkClient.get();
+            // use timestamp as data so watchers will see any changes
+            byte[] timestamp = ByteBuffer.allocate(8).putLong(System.currentTimeMillis()).array();
+
+            if (zk.exists(serverLoadPath, false) == null) { //create path to rootnode if it does not yet exist
+                createServerLoadPathIfNoExists(timestamp);
+            }
+
+            Transaction tx = zk.transaction();
+            List<String> children = zk.getChildren(serverLoadPath, false);
+            HashSet<String> servers = new HashSet<String>(children);
+            tx.setData(serverLoadPath, timestamp, -1); // trigger the watcher that data has been updated
+            for (ServerLoad serverLoad : serverLoads) {
+                String server = serverToZkFormat(serverLoad.getServer());
+                String serverPath = serverPath(server);
+                if (servers.contains(server)) {
+                    servers.remove(server);
+                    tx.setData(serverPath, serverLoad.serialize(), -1);
+                } else {
+                    tx.create(serverPath, serverLoad.serialize(), zkClient.getDefaultACL(), CreateMode.PERSISTENT);
+                }
+            }
+            for (String server : servers) {
+                tx.delete(serverPath(server), -1);
+            }
+            tx.commit();
+        } catch (InterruptedException | IOException | KeeperException e) {
+            throw new StateManagerSaveException(e);
+        }
+    }
+
+    @Override
+    public TreeSet<ServerLoad> loadOwnership() throws StateManagerLoadException {
+        TreeSet<ServerLoad> ownerships = new TreeSet<ServerLoad>();
+        try {
+            ZooKeeper zk = zkClient.get();
+            List<String> children = zk.getChildren(serverLoadPath, false);
+            for (String server : children) {
+                ownerships.add(ServerLoad.deserialize(zk.getData(serverPath(server), false, new Stat())));
+            }
+            return ownerships;
+        } catch (InterruptedException | IOException | KeeperException e) {
+            throw new StateManagerLoadException(e);
+        }
+    }
+
+    @Override
+    public synchronized void watch(final PlacementCallback callback) {
+        if (watching) {
+            return; // do not double watch
+        }
+        watching = true;
+
+        try {
+            ZooKeeper zk = zkClient.get();
+            try {
+                zk.getData(serverLoadPath, new Watcher() {
+                    @Override
+                    public void process(WatchedEvent watchedEvent) {
+                        try {
+                            callback.callback(loadOwnership());
+                        } catch (StateManagerLoadException e) {
+                            logger.error("Watch of Ownership failed", e);
+                        } finally {
+                            watching = false;
+                            watch(callback);
+                        }
+                    }
+                }, new Stat());
+            } catch (KeeperException.NoNodeException nee) {
+                byte[] timestamp = ByteBuffer.allocate(8).putLong(System.currentTimeMillis()).array();
+                createServerLoadPathIfNoExists(timestamp);
+                watching = false;
+                watch(callback);
+            }
+        } catch (ZooKeeperClient.ZooKeeperConnectionException | InterruptedException | KeeperException e) {
+            logger.error("Watch of Ownership failed", e);
+            watching = false;
+            watch(callback);
+        }
+    }
+
+    public String serverPath(String server) {
+        return String.format("%s/%s", serverLoadPath, server);
+    }
+
+    protected String serverToZkFormat(String server) {
+        return server.replaceAll("/", "--");
+    }
+
+    protected String zkFormatToServer(String zkFormattedServer) {
+        return zkFormattedServer.replaceAll("--", "/");
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/package-info.java
new file mode 100644
index 0000000..ea79251
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/placement/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Placement Policy to place streams across proxy services.
+ */
+package org.apache.distributedlog.service.placement;
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java
new file mode 100644
index 0000000..83ac668
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractStreamOp.java
@@ -0,0 +1,175 @@
+/**
+ * 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.service.stream;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.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 com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Promise;
+import com.twitter.util.Return;
+import com.twitter.util.Try;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.Option;
+
+/**
+ * Abstract Stream Operation.
+ */
+public abstract class AbstractStreamOp<Response> implements StreamOp {
+
+    private static final Logger logger = LoggerFactory.getLogger(AbstractStreamOp.class);
+
+    protected final String stream;
+    protected final OpStatsLogger opStatsLogger;
+    private final Promise<Response> result = new Promise<Response>();
+    protected final Stopwatch stopwatch = Stopwatch.createUnstarted();
+    protected final Long checksum;
+    protected final Feature checksumDisabledFeature;
+
+    public AbstractStreamOp(String stream,
+                            OpStatsLogger statsLogger,
+                            Long checksum,
+                            Feature checksumDisabledFeature) {
+        this.stream = stream;
+        this.opStatsLogger = statsLogger;
+        // start here in case the operation is failed before executing.
+        stopwatch.reset().start();
+        this.checksum = checksum;
+        this.checksumDisabledFeature = checksumDisabledFeature;
+    }
+
+    @Override
+    public String streamName() {
+        return stream;
+    }
+
+    @Override
+    public Stopwatch stopwatch() {
+        return stopwatch;
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!checksumDisabledFeature.isAvailable() && null != checksum) {
+            Long serverChecksum = computeChecksum();
+            if (null != serverChecksum && !checksum.equals(serverChecksum)) {
+                throw new ChecksumFailedException();
+            }
+        }
+    }
+
+    @Override
+    public Long computeChecksum() {
+        return null;
+    }
+
+    @Override
+    public Future<Void> execute(AsyncLogWriter writer, Sequencer sequencer, Object txnLock) {
+        stopwatch.reset().start();
+        return executeOp(writer, sequencer, txnLock)
+                .addEventListener(new FutureEventListener<Response>() {
+            @Override
+            public void onSuccess(Response response) {
+                opStatsLogger.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+                setResponse(response);
+            }
+            @Override
+            public void onFailure(Throwable cause) {
+            }
+        }).voided();
+    }
+
+    /**
+     * Fail with current <i>owner</i> and its reason <i>t</i>.
+     *
+     * @param cause
+     *          failure reason
+     */
+    @Override
+    public void fail(Throwable cause) {
+        if (cause instanceof OwnershipAcquireFailedException) {
+            // Ownership exception is a control exception, not an error, so we don't stat
+            // it with the other errors.
+            OwnershipAcquireFailedException oafe = (OwnershipAcquireFailedException) cause;
+            fail(ResponseUtils.ownerToHeader(oafe.getCurrentOwner()));
+        } else {
+            opStatsLogger.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+            fail(ResponseUtils.exceptionToHeader(cause));
+        }
+    }
+
+    protected void setResponse(Response response) {
+      Return<Response> responseTry = new Return(response);
+      boolean isEmpty = result.updateIfEmpty(responseTry);
+      if (!isEmpty) {
+        Option<Try<Response>> resultTry = result.poll();
+        logger.error("Result set multiple times. Value='{}', New='{}'", resultTry, responseTry);
+      }
+    }
+
+    /**
+     * Return the full response, header and body.
+     *
+     * @return A future containing the response or the exception
+     *      encountered by the op if it failed.
+     */
+    public Future<Response> result() {
+        return result;
+    }
+
+    /**
+     * Execute the operation and return its corresponding response.
+     *
+     * @param writer
+     *          writer to execute the operation.
+     * @param sequencer
+     *          sequencer used for generating transaction id for stream operations
+     * @param txnLock
+     *          transaction lock to guarantee ordering of transaction id
+     * @return future representing the operation.
+     */
+    protected abstract Future<Response> executeOp(AsyncLogWriter writer,
+                                                  Sequencer sequencer,
+                                                  Object txnLock);
+
+    // fail the result with the given response header
+    protected abstract void fail(ResponseHeader header);
+
+    public static OpStatsLogger requestStat(StatsLogger statsLogger, String opName) {
+        return requestLogger(statsLogger).getOpStatsLogger(opName);
+    }
+
+    public static StatsLogger requestLogger(StatsLogger statsLogger) {
+        return statsLogger.scope("request");
+    }
+
+    public static StatsLogger requestScope(StatsLogger statsLogger, String scope) {
+        return requestLogger(statsLogger).scope(scope);
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractWriteOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractWriteOp.java
new file mode 100644
index 0000000..77c7d71
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/AbstractWriteOp.java
@@ -0,0 +1,60 @@
+/**
+ * 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.service.stream;
+
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import org.apache.distributedlog.util.ProtocolUtils;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Abstract Write Operation.
+ */
+public abstract class AbstractWriteOp extends AbstractStreamOp<WriteResponse> {
+
+    protected AbstractWriteOp(String stream,
+                              OpStatsLogger statsLogger,
+                              Long checksum,
+                              Feature checksumDisabledFeature) {
+        super(stream, statsLogger, checksum, checksumDisabledFeature);
+    }
+
+    @Override
+    protected void fail(ResponseHeader header) {
+        setResponse(ResponseUtils.write(header));
+    }
+
+    @Override
+    public Long computeChecksum() {
+        return ProtocolUtils.streamOpCRC32(stream);
+    }
+
+    @Override
+    public Future<ResponseHeader> responseHeader() {
+        return result().map(new AbstractFunction1<WriteResponse, ResponseHeader>() {
+            @Override
+            public ResponseHeader apply(WriteResponse response) {
+                return response.getHeader();
+            }
+        });
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java
new file mode 100644
index 0000000..6c98468
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/BulkWriteOp.java
@@ -0,0 +1,253 @@
+/**
+ * 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.service.stream;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.acl.AccessControlManager;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.LockingException;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.exceptions.RequestDeniedException;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.service.streamset.Partition;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.distributedlog.thrift.service.BulkWriteResponse;
+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 com.twitter.util.ConstFuture;
+import com.twitter.util.Future;
+import com.twitter.util.Future$;
+import com.twitter.util.FutureEventListener;
+import com.twitter.util.Try;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Bulk Write Operation.
+ */
+public class BulkWriteOp extends AbstractStreamOp<BulkWriteResponse> implements WriteOpWithPayload {
+    private final List<ByteBuffer> buffers;
+    private final long payloadSize;
+
+    // Stats
+    private final Counter deniedBulkWriteCounter;
+    private final Counter successRecordCounter;
+    private final Counter failureRecordCounter;
+    private final Counter redirectRecordCounter;
+    private final OpStatsLogger latencyStat;
+    private final Counter bytes;
+    private final Counter bulkWriteBytes;
+
+    private final AccessControlManager accessControlManager;
+
+    // We need to pass these through to preserve ownership change behavior in
+    // client/server. Only include failures which are guaranteed to have failed
+    // all subsequent writes.
+    private boolean isDefiniteFailure(Try<DLSN> result) {
+        boolean def = false;
+        try {
+            result.get();
+        } catch (Exception ex) {
+            if (ex instanceof OwnershipAcquireFailedException
+                || ex instanceof AlreadyClosedException
+                || ex instanceof LockingException) {
+                def = true;
+            }
+        }
+        return def;
+    }
+
+    public BulkWriteOp(String stream,
+                       List<ByteBuffer> buffers,
+                       StatsLogger statsLogger,
+                       StatsLogger perStreamStatsLogger,
+                       StreamPartitionConverter streamPartitionConverter,
+                       Long checksum,
+                       Feature checksumDisabledFeature,
+                       AccessControlManager accessControlManager) {
+        super(stream, requestStat(statsLogger, "bulkWrite"), checksum, checksumDisabledFeature);
+        this.buffers = buffers;
+        long total = 0;
+        // We do this here because the bytebuffers are mutable.
+        for (ByteBuffer bb : buffers) {
+          total += bb.remaining();
+        }
+        this.payloadSize = total;
+
+        final Partition partition = streamPartitionConverter.convert(stream);
+        // Write record stats
+        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+        this.deniedBulkWriteCounter = streamOpStats.requestDeniedCounter("bulkWrite");
+        this.successRecordCounter = streamOpStats.recordsCounter("success");
+        this.failureRecordCounter = streamOpStats.recordsCounter("failure");
+        this.redirectRecordCounter = streamOpStats.recordsCounter("redirect");
+        this.bulkWriteBytes = streamOpStats.scopedRequestCounter("bulkWrite", "bytes");
+        this.latencyStat = streamOpStats.streamRequestLatencyStat(partition, "bulkWrite");
+        this.bytes = streamOpStats.streamRequestCounter(partition, "bulkWrite", "bytes");
+
+        this.accessControlManager = accessControlManager;
+
+        final long size = getPayloadSize();
+        result().addEventListener(new FutureEventListener<BulkWriteResponse>() {
+            @Override
+            public void onSuccess(BulkWriteResponse response) {
+                if (response.getHeader().getCode() == StatusCode.SUCCESS) {
+                    latencyStat.registerSuccessfulEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
+                    bytes.add(size);
+                    bulkWriteBytes.add(size);
+                } else {
+                    latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
+                }
+            }
+            @Override
+            public void onFailure(Throwable cause) {
+                latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
+            }
+        });
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!accessControlManager.allowWrite(stream)) {
+            deniedBulkWriteCounter.inc();
+            throw new RequestDeniedException(stream, "bulkWrite");
+        }
+        super.preExecute();
+    }
+
+    @Override
+    public long getPayloadSize() {
+      return payloadSize;
+    }
+
+    @Override
+    protected Future<BulkWriteResponse> executeOp(AsyncLogWriter writer,
+                                                  Sequencer sequencer,
+                                                  Object txnLock) {
+        // Need to convert input buffers to LogRecords.
+        List<LogRecord> records;
+        Future<List<Future<DLSN>>> futureList;
+        synchronized (txnLock) {
+            records = asRecordList(buffers, sequencer);
+            futureList = writer.writeBulk(records);
+        }
+
+        // Collect into a list of tries to make it easier to extract exception or DLSN.
+        Future<List<Try<DLSN>>> writes = asTryList(futureList);
+
+        Future<BulkWriteResponse> response = writes.flatMap(
+            new AbstractFunction1<List<Try<DLSN>>, Future<BulkWriteResponse>>() {
+                @Override
+                public Future<BulkWriteResponse> apply(List<Try<DLSN>> results) {
+
+                    // Considered a success at batch level even if no individual writes succeeed.
+                    // The reason is that its impossible to make an appropriate decision re retries without
+                    // individual buffer failure reasons.
+                    List<WriteResponse> writeResponses = new ArrayList<WriteResponse>(results.size());
+                    BulkWriteResponse bulkWriteResponse =
+                        ResponseUtils.bulkWriteSuccess().setWriteResponses(writeResponses);
+
+                    // Promote the first result to an op-level failure if we're sure all other writes have
+                    // failed.
+                    if (results.size() > 0) {
+                        Try<DLSN> firstResult = results.get(0);
+                        if (isDefiniteFailure(firstResult)) {
+                            return new ConstFuture(firstResult);
+                        }
+                    }
+
+                    // Translate all futures to write responses.
+                    Iterator<Try<DLSN>> iterator = results.iterator();
+                    while (iterator.hasNext()) {
+                        Try<DLSN> completedFuture = iterator.next();
+                        try {
+                            DLSN dlsn = completedFuture.get();
+                            WriteResponse writeResponse = ResponseUtils.writeSuccess().setDlsn(dlsn.serialize());
+                            writeResponses.add(writeResponse);
+                            successRecordCounter.inc();
+                        } catch (Exception ioe) {
+                            WriteResponse writeResponse = ResponseUtils.write(ResponseUtils.exceptionToHeader(ioe));
+                            writeResponses.add(writeResponse);
+                            if (StatusCode.FOUND == writeResponse.getHeader().getCode()) {
+                                redirectRecordCounter.inc();
+                            } else {
+                                failureRecordCounter.inc();
+                            }
+                        }
+                    }
+
+                    return Future.value(bulkWriteResponse);
+                }
+            }
+        );
+
+        return response;
+    }
+
+    private List<LogRecord> asRecordList(List<ByteBuffer> buffers, Sequencer sequencer) {
+        List<LogRecord> records = new ArrayList<LogRecord>(buffers.size());
+        for (ByteBuffer buffer : buffers) {
+            byte[] payload = new byte[buffer.remaining()];
+            buffer.get(payload);
+            records.add(new LogRecord(sequencer.nextId(), payload));
+        }
+        return records;
+    }
+
+    private Future<List<Try<DLSN>>> asTryList(Future<List<Future<DLSN>>> futureList) {
+        return futureList.flatMap(new AbstractFunction1<List<Future<DLSN>>, Future<List<Try<DLSN>>>>() {
+            @Override
+            public Future<List<Try<DLSN>>> apply(List<Future<DLSN>> results) {
+                return Future$.MODULE$.collectToTry(results);
+            }
+        });
+    }
+
+    @Override
+    protected void fail(ResponseHeader header) {
+        if (StatusCode.FOUND == header.getCode()) {
+            redirectRecordCounter.add(buffers.size());
+        } else {
+            failureRecordCounter.add(buffers.size());
+        }
+        setResponse(ResponseUtils.bulkWrite(header));
+    }
+
+    @Override
+    public Future<ResponseHeader> responseHeader() {
+        return result().map(new AbstractFunction1<BulkWriteResponse, ResponseHeader>() {
+            @Override
+            public ResponseHeader apply(BulkWriteResponse response) {
+                return response.getHeader();
+            }
+        });
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java
new file mode 100644
index 0000000..3ecb46f
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/DeleteOp.java
@@ -0,0 +1,76 @@
+/**
+ * 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.service.stream;
+
+import org.apache.distributedlog.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 com.twitter.util.Future;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Operation to delete a log stream.
+ */
+public class DeleteOp extends AbstractWriteOp {
+    private final StreamManager streamManager;
+    private final Counter deniedDeleteCounter;
+    private final AccessControlManager accessControlManager;
+
+    public DeleteOp(String stream,
+                    StatsLogger statsLogger,
+                    StatsLogger perStreamStatsLogger,
+                    StreamManager streamManager,
+                    Long checksum,
+                    Feature checksumEnabledFeature,
+                    AccessControlManager accessControlManager) {
+        super(stream, requestStat(statsLogger, "delete"), checksum, checksumEnabledFeature);
+        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+        this.deniedDeleteCounter = streamOpStats.requestDeniedCounter("delete");
+        this.accessControlManager = accessControlManager;
+        this.streamManager = streamManager;
+    }
+
+    @Override
+    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
+                                              Sequencer sequencer,
+                                              Object txnLock) {
+        Future<Void> result = streamManager.deleteAndRemoveAsync(streamName());
+        return result.map(new AbstractFunction1<Void, WriteResponse>() {
+            @Override
+            public WriteResponse apply(Void value) {
+                return ResponseUtils.writeSuccess();
+            }
+        });
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!accessControlManager.allowTruncate(stream)) {
+            deniedDeleteCounter.inc();
+            throw new RequestDeniedException(stream, "delete");
+        }
+        super.preExecute();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.java
new file mode 100644
index 0000000..5cda0ca
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/HeartbeatOp.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.service.stream;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.BKAsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecord;
+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 com.twitter.util.Future;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Heartbeat Operation.
+ */
+public class HeartbeatOp extends AbstractWriteOp {
+
+    static final byte[] HEARTBEAT_DATA = "heartbeat".getBytes(UTF_8);
+
+    private final AccessControlManager accessControlManager;
+    private final Counter deniedHeartbeatCounter;
+    private final byte dlsnVersion;
+
+    private boolean writeControlRecord = false;
+
+    public HeartbeatOp(String stream,
+                       StatsLogger statsLogger,
+                       StatsLogger perStreamStatsLogger,
+                       byte dlsnVersion,
+                       Long checksum,
+                       Feature checksumDisabledFeature,
+                       AccessControlManager accessControlManager) {
+        super(stream, requestStat(statsLogger, "heartbeat"), checksum, checksumDisabledFeature);
+        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+        this.deniedHeartbeatCounter = streamOpStats.requestDeniedCounter("heartbeat");
+        this.dlsnVersion = dlsnVersion;
+        this.accessControlManager = accessControlManager;
+    }
+
+    public HeartbeatOp setWriteControlRecord(boolean writeControlRecord) {
+        this.writeControlRecord = writeControlRecord;
+        return this;
+    }
+
+    @Override
+    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
+                                              Sequencer sequencer,
+                                              Object txnLock) {
+        // write a control record if heartbeat is the first request of the recovered log segment.
+        if (writeControlRecord) {
+            long txnId;
+            Future<DLSN> writeResult;
+            synchronized (txnLock) {
+                txnId = sequencer.nextId();
+                writeResult = ((BKAsyncLogWriter) writer).writeControlRecord(new LogRecord(txnId, HEARTBEAT_DATA));
+            }
+            return writeResult.map(new AbstractFunction1<DLSN, WriteResponse>() {
+                @Override
+                public WriteResponse apply(DLSN value) {
+                    return ResponseUtils.writeSuccess().setDlsn(value.serialize(dlsnVersion));
+                }
+            });
+        } else {
+            return Future.value(ResponseUtils.writeSuccess());
+        }
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!accessControlManager.allowAcquire(stream)) {
+            deniedHeartbeatCounter.inc();
+            throw new RequestDeniedException(stream, "heartbeat");
+        }
+        super.preExecute();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java
new file mode 100644
index 0000000..6ec8642
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/ReleaseOp.java
@@ -0,0 +1,76 @@
+/**
+ * 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.service.stream;
+
+import org.apache.distributedlog.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 com.twitter.util.Future;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Operation to release ownership of a log stream.
+ */
+public class ReleaseOp extends AbstractWriteOp {
+    private final StreamManager streamManager;
+    private final Counter deniedReleaseCounter;
+    private final AccessControlManager accessControlManager;
+
+    public ReleaseOp(String stream,
+                     StatsLogger statsLogger,
+                     StatsLogger perStreamStatsLogger,
+                     StreamManager streamManager,
+                     Long checksum,
+                     Feature checksumDisabledFeature,
+                     AccessControlManager accessControlManager) {
+        super(stream, requestStat(statsLogger, "release"), checksum, checksumDisabledFeature);
+        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+        this.deniedReleaseCounter = streamOpStats.requestDeniedCounter("release");
+        this.accessControlManager = accessControlManager;
+        this.streamManager = streamManager;
+    }
+
+    @Override
+    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
+                                              Sequencer sequencer,
+                                              Object txnLock) {
+        Future<Void> result = streamManager.closeAndRemoveAsync(streamName());
+        return result.map(new AbstractFunction1<Void, WriteResponse>() {
+            @Override
+            public WriteResponse apply(Void value) {
+                return ResponseUtils.writeSuccess();
+            }
+        });
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!accessControlManager.allowRelease(stream)) {
+            deniedReleaseCounter.inc();
+            throw new RequestDeniedException(stream, "release");
+        }
+        super.preExecute();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/Stream.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/Stream.java
new file mode 100644
index 0000000..3517a63
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/Stream.java
@@ -0,0 +1,93 @@
+/**
+ * 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.service.stream;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.service.streamset.Partition;
+import com.twitter.util.Future;
+import java.io.IOException;
+
+/**
+ * Stream is the per stream request handler in the DL service layer.
+ *
+ * <p>The collection of Streams in the proxy are managed by StreamManager.
+ */
+public interface Stream {
+
+    /**
+     * Get the stream configuration for this stream.
+     *
+     * @return stream configuration
+     */
+    DynamicDistributedLogConfiguration getStreamConfiguration();
+
+    /**
+     * Get the stream's last recorded current owner (may be out of date). Used
+     * as a hint for the client.
+     * @return last known owner for the stream
+     */
+    String getOwner();
+
+    /**
+     * Get the stream name.
+     * @return stream name
+     */
+    String getStreamName();
+
+    /**
+     * Get the represented partition name.
+     *
+     * @return represented partition name.
+     */
+    Partition getPartition();
+
+    /**
+     * Expensive initialization code run after stream has been allocated in
+     * StreamManager.
+     *
+     * @throws IOException when encountered exception on initialization
+     */
+    void initialize() throws IOException;
+
+    /**
+     * Another initialize method (actually Thread.start). Should probably be
+     * moved to initialize().
+     */
+    void start();
+
+    /**
+     * Asynchronous close method.
+     * @param reason for closing
+     * @return future satisfied once close complete
+     */
+    Future<Void> requestClose(String reason);
+
+    /**
+     * Delete the stream from DL backend.
+     *
+     * @throws IOException when encountered exception on deleting the stream.
+     */
+    void delete() throws IOException;
+
+    /**
+     * Execute the stream operation against this stream.
+     *
+     * @param op operation to execute
+     */
+    void submit(StreamOp op);
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactory.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactory.java
new file mode 100644
index 0000000..845ef21
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactory.java
@@ -0,0 +1,38 @@
+/**
+ * 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.service.stream;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+
+/**
+ * Factory to create a stream with provided stream configuration {@code streamConf}.
+ */
+public interface StreamFactory {
+
+    /**
+     * Create a stream object.
+     *
+     * @param name stream name
+     * @param streamConf stream configuration
+     * @param streamManager manager of streams
+     * @return stream object
+     */
+    Stream create(String name,
+                  DynamicDistributedLogConfiguration streamConf,
+                  StreamManager streamManager);
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.java
new file mode 100644
index 0000000..2b90d55
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamFactoryImpl.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.service.stream;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+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;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.distributedlog.util.OrderedScheduler;
+import com.twitter.util.Timer;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.jboss.netty.util.HashedWheelTimer;
+
+/**
+ * The implementation of {@link StreamFactory}.
+ */
+public class StreamFactoryImpl implements StreamFactory {
+    private final String clientId;
+    private final StreamOpStats streamOpStats;
+    private final ServerConfiguration serverConfig;
+    private final DistributedLogConfiguration dlConfig;
+    private final FeatureProvider featureProvider;
+    private final StreamConfigProvider streamConfigProvider;
+    private final StreamPartitionConverter streamPartitionConverter;
+    private final DistributedLogNamespace dlNamespace;
+    private final OrderedScheduler scheduler;
+    private final FatalErrorHandler fatalErrorHandler;
+    private final HashedWheelTimer requestTimer;
+    private final Timer futureTimer;
+
+    public StreamFactoryImpl(String clientId,
+        StreamOpStats streamOpStats,
+        ServerConfiguration serverConfig,
+        DistributedLogConfiguration dlConfig,
+        FeatureProvider featureProvider,
+        StreamConfigProvider streamConfigProvider,
+        StreamPartitionConverter streamPartitionConverter,
+        DistributedLogNamespace dlNamespace,
+        OrderedScheduler scheduler,
+        FatalErrorHandler fatalErrorHandler,
+        HashedWheelTimer requestTimer) {
+
+        this.clientId = clientId;
+        this.streamOpStats = streamOpStats;
+        this.serverConfig = serverConfig;
+        this.dlConfig = dlConfig;
+        this.featureProvider = featureProvider;
+        this.streamConfigProvider = streamConfigProvider;
+        this.streamPartitionConverter = streamPartitionConverter;
+        this.dlNamespace = dlNamespace;
+        this.scheduler = scheduler;
+        this.fatalErrorHandler = fatalErrorHandler;
+        this.requestTimer = requestTimer;
+        this.futureTimer = new com.twitter.finagle.util.HashedWheelTimer(requestTimer);
+    }
+
+    @Override
+    public Stream create(String name,
+                         DynamicDistributedLogConfiguration streamConf,
+                         StreamManager streamManager) {
+        return new StreamImpl(name,
+            streamPartitionConverter.convert(name),
+            clientId,
+            streamManager,
+            streamOpStats,
+            serverConfig,
+            dlConfig,
+            streamConf,
+            featureProvider,
+            streamConfigProvider,
+            dlNamespace,
+            scheduler,
+            fatalErrorHandler,
+            requestTimer,
+            futureTimer);
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java
new file mode 100644
index 0000000..bf7a1ad
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamImpl.java
@@ -0,0 +1,925 @@
+/**
+ * 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.service.stream;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.AlreadyClosedException;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.exceptions.StreamNotReadyException;
+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.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.util.OrderedScheduler;
+import org.apache.distributedlog.util.TimeSequencer;
+import org.apache.distributedlog.util.Utils;
+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.TimeoutException;
+import com.twitter.util.Timer;
+import java.io.IOException;
+import java.util.ArrayDeque;
+import java.util.Queue;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.Gauge;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.jboss.netty.util.HashedWheelTimer;
+import org.jboss.netty.util.Timeout;
+import org.jboss.netty.util.TimerTask;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+import scala.runtime.AbstractFunction1;
+import scala.runtime.BoxedUnit;
+
+/**
+ * Implementation of {@link Stream}.
+ */
+public class StreamImpl implements Stream {
+
+    private static final Logger logger = LoggerFactory.getLogger(StreamImpl.class);
+
+    /**
+     * The status of the stream.
+     *
+     * <p>The status change of the stream should just go in one direction. If a stream hits
+     * any error, the stream should be put in error state. If a stream is in error state,
+     * it should be removed and not reused anymore.
+     */
+    public enum StreamStatus {
+        UNINITIALIZED(-1),
+        INITIALIZING(0),
+        INITIALIZED(1),
+        CLOSING(-4),
+        CLOSED(-5),
+        // if a stream is in error state, it should be abort during closing.
+        ERROR(-6);
+
+        final int code;
+
+        StreamStatus(int code) {
+            this.code = code;
+        }
+
+        int getCode() {
+            return code;
+        }
+
+        public static boolean isUnavailable(StreamStatus status) {
+            return StreamStatus.ERROR == status || StreamStatus.CLOSING == status || StreamStatus.CLOSED == status;
+        }
+    }
+
+    private final String name;
+    private final Partition partition;
+    private DistributedLogManager manager;
+
+    private volatile AsyncLogWriter writer;
+    private volatile StreamStatus status;
+    private volatile String owner;
+    private volatile Throwable lastException;
+    private volatile Queue<StreamOp> pendingOps = new ArrayDeque<StreamOp>();
+
+    private final Promise<Void> closePromise = new Promise<Void>();
+    private final Object txnLock = new Object();
+    private final TimeSequencer sequencer = new TimeSequencer();
+    private final StreamRequestLimiter limiter;
+    private final DynamicDistributedLogConfiguration dynConf;
+    private final DistributedLogConfiguration dlConfig;
+    private final DistributedLogNamespace dlNamespace;
+    private final String clientId;
+    private final OrderedScheduler scheduler;
+    private final ReentrantReadWriteLock closeLock = new ReentrantReadWriteLock();
+    private final Feature featureRateLimitDisabled;
+    private final StreamManager streamManager;
+    private final StreamConfigProvider streamConfigProvider;
+    private final FatalErrorHandler fatalErrorHandler;
+    private final long streamProbationTimeoutMs;
+    private final long serviceTimeoutMs;
+    private final long writerCloseTimeoutMs;
+    private final boolean failFastOnStreamNotReady;
+    private final HashedWheelTimer requestTimer;
+    private final Timer futureTimer;
+
+    // Stats
+    private final StatsLogger streamLogger;
+    private final StatsLogger streamExceptionStatLogger;
+    private final StatsLogger limiterStatLogger;
+    private final Counter serviceTimeout;
+    private final OpStatsLogger streamAcquireStat;
+    private final OpStatsLogger writerCloseStatLogger;
+    private final Counter pendingOpsCounter;
+    private final Counter unexpectedExceptions;
+    private final Counter writerCloseTimeoutCounter;
+    private final StatsLogger exceptionStatLogger;
+    private final ConcurrentHashMap<String, Counter> exceptionCounters =
+        new ConcurrentHashMap<String, Counter>();
+    private final Gauge<Number> streamStatusGauge;
+
+    // Since we may create and discard streams at initialization if there's a race,
+    // must not do any expensive initialization here (particularly any locking or
+    // significant resource allocation etc.).
+    StreamImpl(final String name,
+               final Partition partition,
+               String clientId,
+               StreamManager streamManager,
+               StreamOpStats streamOpStats,
+               ServerConfiguration serverConfig,
+               DistributedLogConfiguration dlConfig,
+               DynamicDistributedLogConfiguration streamConf,
+               FeatureProvider featureProvider,
+               StreamConfigProvider streamConfigProvider,
+               DistributedLogNamespace dlNamespace,
+               OrderedScheduler scheduler,
+               FatalErrorHandler fatalErrorHandler,
+               HashedWheelTimer requestTimer,
+               Timer futureTimer) {
+        this.clientId = clientId;
+        this.dlConfig = dlConfig;
+        this.streamManager = streamManager;
+        this.name = name;
+        this.partition = partition;
+        this.status = StreamStatus.UNINITIALIZED;
+        this.lastException = new IOException("Fail to write record to stream " + name);
+        this.streamConfigProvider = streamConfigProvider;
+        this.dlNamespace = dlNamespace;
+        this.featureRateLimitDisabled = featureProvider.getFeature(
+            ServerFeatureKeys.SERVICE_RATE_LIMIT_DISABLED.name().toLowerCase());
+        this.scheduler = scheduler;
+        this.serviceTimeoutMs = serverConfig.getServiceTimeoutMs();
+        this.streamProbationTimeoutMs = serverConfig.getStreamProbationTimeoutMs();
+        this.writerCloseTimeoutMs = serverConfig.getWriterCloseTimeoutMs();
+        this.failFastOnStreamNotReady = dlConfig.getFailFastOnStreamNotReady();
+        this.fatalErrorHandler = fatalErrorHandler;
+        this.dynConf = streamConf;
+        StatsLogger limiterStatsLogger = BroadCastStatsLogger.two(
+            streamOpStats.baseScope("stream_limiter"),
+            streamOpStats.streamRequestScope(partition, "limiter"));
+        this.limiter = new StreamRequestLimiter(name, dynConf, limiterStatsLogger, featureRateLimitDisabled);
+        this.requestTimer = requestTimer;
+        this.futureTimer = futureTimer;
+
+        // Stats
+        this.streamLogger = streamOpStats.streamRequestStatsLogger(partition);
+        this.limiterStatLogger = streamOpStats.baseScope("request_limiter");
+        this.streamExceptionStatLogger = streamLogger.scope("exceptions");
+        this.serviceTimeout = streamOpStats.baseCounter("serviceTimeout");
+        StatsLogger streamsStatsLogger = streamOpStats.baseScope("streams");
+        this.streamAcquireStat = streamsStatsLogger.getOpStatsLogger("acquire");
+        this.pendingOpsCounter = streamOpStats.baseCounter("pending_ops");
+        this.unexpectedExceptions = streamOpStats.baseCounter("unexpected_exceptions");
+        this.exceptionStatLogger = streamOpStats.requestScope("exceptions");
+        this.writerCloseStatLogger = streamsStatsLogger.getOpStatsLogger("writer_close");
+        this.writerCloseTimeoutCounter = streamsStatsLogger.getCounter("writer_close_timeouts");
+        // Gauges
+        this.streamStatusGauge = new Gauge<Number>() {
+            @Override
+            public Number getDefaultValue() {
+                return StreamStatus.UNINITIALIZED.getCode();
+            }
+            @Override
+            public Number getSample() {
+                return status.getCode();
+            }
+        };
+    }
+
+    @Override
+    public String getOwner() {
+        return owner;
+    }
+
+    @Override
+    public String getStreamName() {
+        return name;
+    }
+
+    @Override
+    public DynamicDistributedLogConfiguration getStreamConfiguration() {
+        return dynConf;
+    }
+
+    @Override
+    public Partition getPartition() {
+        return partition;
+    }
+
+    private DistributedLogManager openLog(String name) throws IOException {
+        Optional<DistributedLogConfiguration> dlConf = Optional.<DistributedLogConfiguration>absent();
+        Optional<DynamicDistributedLogConfiguration> dynDlConf = Optional.of(dynConf);
+        Optional<StatsLogger> perStreamStatsLogger = Optional.of(streamLogger);
+        return dlNamespace.openLog(name, dlConf, dynDlConf, perStreamStatsLogger);
+    }
+
+    // Expensive initialization, only called once per stream.
+    @Override
+    public void initialize() throws IOException {
+        manager = openLog(name);
+
+        // Better to avoid registering the gauge multiple times, so do this in init
+        // which only gets called once.
+        streamLogger.registerGauge("stream_status", this.streamStatusGauge);
+
+        // Signal initialization is complete, should be last in this method.
+        status = StreamStatus.INITIALIZING;
+    }
+
+    @Override
+    public String toString() {
+        return String.format("Stream:%s, %s, %s Status:%s", name, manager, writer, status);
+    }
+
+    @Override
+    public void start() {
+        // acquire the stream
+        acquireStream().addEventListener(new FutureEventListener<Boolean>() {
+                @Override
+                public void onSuccess(Boolean success) {
+                    if (!success) {
+                        // failed to acquire the stream. set the stream in error status and close it.
+                        setStreamInErrorStatus();
+                        requestClose("Failed to acquire the ownership");
+                    }
+                }
+
+                @Override
+                public void onFailure(Throwable cause) {
+                    // unhandled exceptions
+                    logger.error("Stream {} threw unhandled exception : ", name, cause);
+                    // failed to acquire the stream. set the stream in error status and close it.
+                    setStreamInErrorStatus();
+                    requestClose("Unhandled exception");
+                }
+            });
+    }
+
+    //
+    // Stats Operations
+    //
+
+    void countException(Throwable t, StatsLogger streamExceptionLogger) {
+        String exceptionName = null == t ? "null" : t.getClass().getName();
+        Counter counter = exceptionCounters.get(exceptionName);
+        if (null == counter) {
+            counter = exceptionStatLogger.getCounter(exceptionName);
+            Counter oldCounter = exceptionCounters.putIfAbsent(exceptionName, counter);
+            if (null != oldCounter) {
+                counter = oldCounter;
+            }
+        }
+        counter.inc();
+        streamExceptionLogger.getCounter(exceptionName).inc();
+    }
+
+    boolean isCriticalException(Throwable cause) {
+        return !(cause instanceof OwnershipAcquireFailedException);
+    }
+
+    //
+    // Service Timeout:
+    // - schedule a timeout function to handle operation timeouts: {@link #handleServiceTimeout(String)}
+    // - if the operation is completed within timeout period, cancel the timeout.
+    //
+
+    void scheduleTimeout(final StreamOp op) {
+        final Timeout timeout = requestTimer.newTimeout(new TimerTask() {
+            @Override
+            public void run(Timeout timeout) throws Exception {
+                if (!timeout.isCancelled()) {
+                    serviceTimeout.inc();
+                    handleServiceTimeout("Operation " + op.getClass().getName() + " timeout");
+                }
+            }
+        }, serviceTimeoutMs, TimeUnit.MILLISECONDS);
+        op.responseHeader().ensure(new Function0<BoxedUnit>() {
+            @Override
+            public BoxedUnit apply() {
+                timeout.cancel();
+                return null;
+            }
+        });
+    }
+
+    /**
+     * Close the stream and schedule cache eviction at some point in the future.
+     * We delay this as a way to place the stream in a probationary state--cached
+     * in the proxy but unusable.
+     * This mechanism helps the cluster adapt to situations where a proxy has
+     * persistent connectivity/availability issues, because it keeps an affected
+     * stream off the proxy for a period of time, hopefully long enough for the
+     * issues to be resolved, or for whoop to kick in and kill the shard.
+     */
+    void handleServiceTimeout(String reason) {
+        synchronized (this) {
+            if (StreamStatus.isUnavailable(status)) {
+                return;
+            }
+            // Mark stream in error state
+            setStreamInErrorStatus();
+        }
+
+        // Async close request, and schedule eviction when its done.
+        Future<Void> closeFuture = requestClose(reason, false /* dont remove */);
+        closeFuture.onSuccess(new AbstractFunction1<Void, BoxedUnit>() {
+            @Override
+            public BoxedUnit apply(Void result) {
+                streamManager.scheduleRemoval(StreamImpl.this, streamProbationTimeoutMs);
+                return BoxedUnit.UNIT;
+            }
+        });
+    }
+
+    //
+    // Submit the operation to the stream.
+    //
+
+    /**
+     * Execute the StreamOp. If reacquire is needed, this may initiate reacquire and queue the op for
+     * execution once complete.
+     *
+     * @param op
+     *          stream operation to execute.
+     */
+    @Override
+    public void submit(StreamOp op) {
+        try {
+            limiter.apply(op);
+        } catch (OverCapacityException ex) {
+            op.fail(ex);
+            return;
+        }
+
+        // Timeout stream op if requested.
+        if (serviceTimeoutMs > 0) {
+            scheduleTimeout(op);
+        }
+
+        boolean completeOpNow = false;
+        boolean success = true;
+        if (StreamStatus.isUnavailable(status)) {
+            // Stream is closed, fail the op immediately
+            op.fail(new StreamUnavailableException("Stream " + name + " is closed."));
+            return;
+        } else if (StreamStatus.INITIALIZED == status && writer != null) {
+            completeOpNow = true;
+            success = true;
+        } else {
+            synchronized (this) {
+                if (StreamStatus.isUnavailable(status)) {
+                    // Stream is closed, fail the op immediately
+                    op.fail(new StreamUnavailableException("Stream " + name + " is closed."));
+                    return;
+                } else if (StreamStatus.INITIALIZED == status) {
+                    completeOpNow = true;
+                    success = true;
+                } else if (failFastOnStreamNotReady) {
+                    op.fail(new StreamNotReadyException("Stream " + name + " is not ready; status = " + status));
+                    return;
+                } else { // the stream is still initializing
+                    pendingOps.add(op);
+                    pendingOpsCounter.inc();
+                    if (1 == pendingOps.size()) {
+                        if (op instanceof HeartbeatOp) {
+                            ((HeartbeatOp) op).setWriteControlRecord(true);
+                        }
+                    }
+                }
+            }
+        }
+        if (completeOpNow) {
+            executeOp(op, success);
+        }
+    }
+
+    //
+    // Execute operations and handle exceptions on operations
+    //
+
+    /**
+     * Execute the <i>op</i> immediately.
+     *
+     * @param op
+     *          stream operation to execute.
+     * @param success
+     *          whether the operation is success or not.
+     */
+    void executeOp(final StreamOp op, boolean success) {
+        final AsyncLogWriter writer;
+        final Throwable lastException;
+        synchronized (this) {
+            writer = this.writer;
+            lastException = this.lastException;
+        }
+        if (null != writer && success) {
+            op.execute(writer, sequencer, txnLock)
+                    .addEventListener(new FutureEventListener<Void>() {
+                @Override
+                public void onSuccess(Void value) {
+                    // nop
+                }
+                @Override
+                public void onFailure(Throwable cause) {
+                    boolean countAsException = true;
+                    if (cause instanceof DLException) {
+                        final DLException dle = (DLException) cause;
+                        switch (dle.getCode()) {
+                        case FOUND:
+                            assert(cause instanceof OwnershipAcquireFailedException);
+                            countAsException = false;
+                            handleExceptionOnStreamOp(op, cause);
+                            break;
+                        case ALREADY_CLOSED:
+                            assert(cause instanceof AlreadyClosedException);
+                            op.fail(cause);
+                            handleAlreadyClosedException((AlreadyClosedException) cause);
+                            break;
+                        // exceptions that mostly from client (e.g. too large record)
+                        case NOT_IMPLEMENTED:
+                        case METADATA_EXCEPTION:
+                        case LOG_EMPTY:
+                        case LOG_NOT_FOUND:
+                        case TRUNCATED_TRANSACTION:
+                        case END_OF_STREAM:
+                        case TRANSACTION_OUT_OF_ORDER:
+                        case INVALID_STREAM_NAME:
+                        case TOO_LARGE_RECORD:
+                        case STREAM_NOT_READY:
+                        case OVER_CAPACITY:
+                            op.fail(cause);
+                            break;
+                        // the DL writer hits exception, simple set the stream to error status
+                        // and fail the request
+                        default:
+                            handleExceptionOnStreamOp(op, cause);
+                            break;
+                        }
+                    } else {
+                        handleExceptionOnStreamOp(op, cause);
+                    }
+                    if (countAsException) {
+                        countException(cause, streamExceptionStatLogger);
+                    }
+                }
+            });
+        } else {
+            if (null != lastException) {
+                op.fail(lastException);
+            } else {
+                op.fail(new StreamUnavailableException("Stream " + name + " is closed."));
+            }
+        }
+    }
+
+    /**
+     * Handle exception when executing <i>op</i>.
+     *
+     * @param op
+     *          stream operation executing
+     * @param cause
+     *          exception received when executing <i>op</i>
+     */
+    private void handleExceptionOnStreamOp(StreamOp op, final Throwable cause) {
+        AsyncLogWriter oldWriter = null;
+        boolean statusChanged = false;
+        synchronized (this) {
+            if (StreamStatus.INITIALIZED == status) {
+                oldWriter = setStreamStatus(StreamStatus.ERROR, StreamStatus.INITIALIZED, null, cause);
+                statusChanged = true;
+            }
+        }
+        if (statusChanged) {
+            Abortables.asyncAbort(oldWriter, false);
+            if (isCriticalException(cause)) {
+                logger.error("Failed to write data into stream {} : ", name, cause);
+            } else {
+                logger.warn("Failed to write data into stream {} : {}", name, cause.getMessage());
+            }
+            requestClose("Failed to write data into stream " + name + " : " + cause.getMessage());
+        }
+        op.fail(cause);
+    }
+
+    /**
+     * Handling already closed exception.
+     */
+    private void handleAlreadyClosedException(AlreadyClosedException ace) {
+        unexpectedExceptions.inc();
+        logger.error("Encountered unexpected exception when writing data into stream {} : ", name, ace);
+        fatalErrorHandler.notifyFatalError();
+    }
+
+    //
+    // Acquire streams
+    //
+
+    Future<Boolean> acquireStream() {
+        final Stopwatch stopwatch = Stopwatch.createStarted();
+        final Promise<Boolean> acquirePromise = new Promise<Boolean>();
+        manager.openAsyncLogWriter().addEventListener(
+            FutureUtils.OrderedFutureEventListener.of(new FutureEventListener<AsyncLogWriter>() {
+
+            @Override
+            public void onSuccess(AsyncLogWriter w) {
+                onAcquireStreamSuccess(w, stopwatch, acquirePromise);
+            }
+
+            @Override
+            public void onFailure(Throwable cause) {
+                onAcquireStreamFailure(cause, stopwatch, acquirePromise);
+            }
+
+        }, scheduler, getStreamName()));
+        return acquirePromise;
+    }
+
+    private void onAcquireStreamSuccess(AsyncLogWriter w,
+                                        Stopwatch stopwatch,
+                                        Promise<Boolean> acquirePromise) {
+        synchronized (txnLock) {
+            sequencer.setLastId(w.getLastTxId());
+        }
+        AsyncLogWriter oldWriter;
+        Queue<StreamOp> oldPendingOps;
+        boolean success;
+        synchronized (StreamImpl.this) {
+            oldWriter = setStreamStatus(StreamStatus.INITIALIZED,
+                    StreamStatus.INITIALIZING, w, null);
+            oldPendingOps = pendingOps;
+            pendingOps = new ArrayDeque<StreamOp>();
+            success = true;
+        }
+        // check if the stream is allowed to be acquired
+        if (!streamManager.allowAcquire(StreamImpl.this)) {
+            if (null != oldWriter) {
+                Abortables.asyncAbort(oldWriter, true);
+            }
+            int maxAcquiredPartitions = dynConf.getMaxAcquiredPartitionsPerProxy();
+            StreamUnavailableException sue = new StreamUnavailableException("Stream " + partition.getStream()
+                    + " is not allowed to acquire more than " + maxAcquiredPartitions + " partitions");
+            countException(sue, exceptionStatLogger);
+            logger.error("Failed to acquire stream {} because it is unavailable : {}",
+                    name, sue.getMessage());
+            synchronized (this) {
+                oldWriter = setStreamStatus(StreamStatus.ERROR,
+                        StreamStatus.INITIALIZED, null, sue);
+                // we don't switch the pending ops since they are already switched
+                // when setting the status to initialized
+                success = false;
+            }
+        }
+        processPendingRequestsAfterAcquire(success, oldWriter, oldPendingOps, stopwatch, acquirePromise);
+    }
+
+    private void onAcquireStreamFailure(Throwable cause,
+                                        Stopwatch stopwatch,
+                                        Promise<Boolean> acquirePromise) {
+        AsyncLogWriter oldWriter;
+        Queue<StreamOp> oldPendingOps;
+        boolean success;
+        if (cause instanceof AlreadyClosedException) {
+            countException(cause, streamExceptionStatLogger);
+            handleAlreadyClosedException((AlreadyClosedException) cause);
+            return;
+        } else {
+            if (isCriticalException(cause)) {
+                countException(cause, streamExceptionStatLogger);
+                logger.error("Failed to acquire stream {} : ", name, cause);
+            } else {
+                logger.warn("Failed to acquire stream {} : {}", name, cause.getMessage());
+            }
+            synchronized (StreamImpl.this) {
+                oldWriter = setStreamStatus(StreamStatus.ERROR,
+                        StreamStatus.INITIALIZING, null, cause);
+                oldPendingOps = pendingOps;
+                pendingOps = new ArrayDeque<StreamOp>();
+                success = false;
+            }
+        }
+        processPendingRequestsAfterAcquire(success, oldWriter, oldPendingOps, stopwatch, acquirePromise);
+    }
+
+    /**
+     * Process the pending request after acquired stream.
+     *
+     * @param success whether the acquisition succeed or not
+     * @param oldWriter the old writer to abort
+     * @param oldPendingOps the old pending ops to execute
+     * @param stopwatch stopwatch to measure the time spent on acquisition
+     * @param acquirePromise the promise to complete the acquire operation
+     */
+    void processPendingRequestsAfterAcquire(boolean success,
+                                            AsyncLogWriter oldWriter,
+                                            Queue<StreamOp> oldPendingOps,
+                                            Stopwatch stopwatch,
+                                            Promise<Boolean> acquirePromise) {
+        if (success) {
+            streamAcquireStat.registerSuccessfulEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+        } else {
+            streamAcquireStat.registerFailedEvent(stopwatch.elapsed(TimeUnit.MICROSECONDS));
+        }
+        for (StreamOp op : oldPendingOps) {
+            executeOp(op, success);
+            pendingOpsCounter.dec();
+        }
+        Abortables.asyncAbort(oldWriter, true);
+        FutureUtils.setValue(acquirePromise, success);
+    }
+
+    //
+    // Stream Status Changes
+    //
+
+    synchronized void setStreamInErrorStatus() {
+        if (StreamStatus.CLOSING == status || StreamStatus.CLOSED == status) {
+            return;
+        }
+        this.status = StreamStatus.ERROR;
+    }
+
+    /**
+     * Update the stream status. The changes are only applied when there isn't status changed.
+     *
+     * @param newStatus
+     *          new status
+     * @param oldStatus
+     *          old status
+     * @param writer
+     *          new log writer
+     * @param t
+     *          new exception
+     * @return old writer if it exists
+     */
+    synchronized AsyncLogWriter setStreamStatus(StreamStatus newStatus,
+                                                StreamStatus oldStatus,
+                                                AsyncLogWriter writer,
+                                                Throwable t) {
+        if (oldStatus != this.status) {
+            logger.info("Stream {} status already changed from {} -> {} when trying to change it to {}",
+                    new Object[] { name, oldStatus, this.status, newStatus });
+            return null;
+        }
+
+        String owner = null;
+        if (t instanceof OwnershipAcquireFailedException) {
+            owner = ((OwnershipAcquireFailedException) t).getCurrentOwner();
+        }
+
+        AsyncLogWriter oldWriter = this.writer;
+        this.writer = writer;
+        if (null != owner && owner.equals(clientId)) {
+            unexpectedExceptions.inc();
+            logger.error("I am waiting myself {} to release lock on stream {}, so have to shut myself down :",
+                         new Object[] { owner, name, t });
+            // I lost the ownership but left a lock over zookeeper
+            // I should not ask client to redirect to myself again as I can't handle it :(
+            // shutdown myself
+            fatalErrorHandler.notifyFatalError();
+            this.owner = null;
+        } else {
+            this.owner = owner;
+        }
+        this.lastException = t;
+        this.status = newStatus;
+        if (StreamStatus.INITIALIZED == newStatus) {
+            streamManager.notifyAcquired(this);
+            logger.info("Inserted acquired stream {} -> writer {}", name, this);
+        } else {
+            streamManager.notifyReleased(this);
+            logger.info("Removed acquired stream {} -> writer {}", name, this);
+        }
+        return oldWriter;
+    }
+
+    //
+    // Stream Close Functions
+    //
+
+    void close(DistributedLogManager dlm) {
+        if (null != dlm) {
+            try {
+                dlm.close();
+            } catch (IOException ioe) {
+                logger.warn("Failed to close dlm for {} : ", name, ioe);
+            }
+        }
+    }
+
+    @Override
+    public Future<Void> requestClose(String reason) {
+        return requestClose(reason, true);
+    }
+
+    Future<Void> requestClose(String reason, boolean uncache) {
+        final boolean abort;
+        closeLock.writeLock().lock();
+        try {
+            if (StreamStatus.CLOSING == status
+                || StreamStatus.CLOSED == status) {
+                return closePromise;
+            }
+            logger.info("Request to close stream {} : {}", getStreamName(), reason);
+            // if the stream isn't closed from INITIALIZED state, we abort the stream instead of closing it.
+            abort = StreamStatus.INITIALIZED != status;
+            status = StreamStatus.CLOSING;
+            streamManager.notifyReleased(this);
+        } finally {
+            closeLock.writeLock().unlock();
+        }
+        // we will fail the requests that are coming in between closing and closed only
+        // after the async writer is closed. so we could clear up the lock before redirect
+        // them.
+        close(abort, uncache);
+        return closePromise;
+    }
+
+    @Override
+    public void delete() throws IOException {
+        if (null != writer) {
+            Utils.close(writer);
+            synchronized (this) {
+                writer = null;
+                lastException = new StreamUnavailableException("Stream was deleted");
+            }
+        }
+        if (null == manager) {
+            throw new UnexpectedException("No stream " + name + " to delete");
+        }
+        manager.delete();
+    }
+
+    /**
+     * Post action executed after closing.
+     */
+    private void postClose(boolean uncache) {
+        closeManagerAndErrorOutPendingRequests();
+        unregisterGauge();
+        if (uncache) {
+            if (null != owner) {
+                long probationTimeoutMs = 2 * dlConfig.getZKSessionTimeoutMilliseconds() / 3;
+                streamManager.scheduleRemoval(this, probationTimeoutMs);
+            } else {
+                streamManager.notifyRemoved(this);
+                logger.info("Removed cached stream {}.", getStreamName());
+            }
+        }
+        FutureUtils.setValue(closePromise, null);
+    }
+
+    /**
+     * Shouldn't call close directly. The callers should call #requestClose instead
+     *
+     * @param shouldAbort shall we abort the stream instead of closing
+     */
+    private Future<Void> close(boolean shouldAbort, final boolean uncache) {
+        boolean abort;
+        closeLock.writeLock().lock();
+        try {
+            if (StreamStatus.CLOSED == status) {
+                return closePromise;
+            }
+            abort = shouldAbort || (StreamStatus.INITIALIZED != status && StreamStatus.CLOSING != status);
+            status = StreamStatus.CLOSED;
+            streamManager.notifyReleased(this);
+        } finally {
+            closeLock.writeLock().unlock();
+        }
+        logger.info("Closing stream {} ...", name);
+        // Close the writers to release the locks before failing the requests
+        Future<Void> closeWriterFuture;
+        if (abort) {
+            closeWriterFuture = Abortables.asyncAbort(writer, true);
+        } else {
+            closeWriterFuture = Utils.asyncClose(writer, true);
+        }
+        // close the manager and error out pending requests after close writer
+        Duration closeWaitDuration;
+        if (writerCloseTimeoutMs <= 0) {
+            closeWaitDuration = Duration.Top();
+        } else {
+            closeWaitDuration = Duration.fromMilliseconds(writerCloseTimeoutMs);
+        }
+
+        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);
+                    }
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        if (cause instanceof TimeoutException) {
+                            writerCloseTimeoutCounter.inc();
+                        }
+                        postClose(uncache);
+                    }
+                }, scheduler, name));
+        return closePromise;
+    }
+
+    private void closeManagerAndErrorOutPendingRequests() {
+        close(manager);
+        // Failed the pending requests.
+        Queue<StreamOp> oldPendingOps;
+        synchronized (this) {
+            oldPendingOps = pendingOps;
+            pendingOps = new ArrayDeque<StreamOp>();
+        }
+        StreamUnavailableException closingException =
+                new StreamUnavailableException("Stream " + name + " is closed.");
+        for (StreamOp op : oldPendingOps) {
+            op.fail(closingException);
+            pendingOpsCounter.dec();
+        }
+        limiter.close();
+        logger.info("Closed stream {}.", name);
+    }
+
+    /**
+     * clean up the gauge to help GC.
+     */
+    private void unregisterGauge(){
+        streamLogger.unregisterGauge("stream_status", this.streamStatusGauge);
+    }
+
+    // Test-only apis
+
+    @VisibleForTesting
+    public int numPendingOps() {
+        Queue<StreamOp> queue = pendingOps;
+        return null == queue ? 0 : queue.size();
+    }
+
+    @VisibleForTesting
+    public StreamStatus getStatus() {
+        return status;
+    }
+
+    @VisibleForTesting
+    public void setStatus(StreamStatus status) {
+        this.status = status;
+    }
+
+    @VisibleForTesting
+    public AsyncLogWriter getWriter() {
+        return writer;
+    }
+
+    @VisibleForTesting
+    public DistributedLogManager getManager() {
+        return manager;
+    }
+
+    @VisibleForTesting
+    public Throwable getLastException() {
+        return lastException;
+    }
+
+    @VisibleForTesting
+    public Future<Void> getCloseFuture() {
+        return closePromise;
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManager.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManager.java
new file mode 100644
index 0000000..d86c538
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManager.java
@@ -0,0 +1,142 @@
+/**
+ * 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.service.stream;
+
+import com.google.common.base.Optional;
+import com.twitter.util.Future;
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Manage lifecycle of streams.
+ *
+ * <p>StreamManager is responsible for creating, destroying, and keeping track of Stream objects.
+ *
+ * <p>Stream objects, which are managed by StreamManager and created by StreamFactory, are essentially the
+ * per stream request handlers, responsible fo dispatching ex. write requests to an underlying AsyncLogWriter,
+ * managing stream lock, interpreting exceptions, error conditions, and etc.
+ */
+public interface StreamManager {
+
+    /**
+     * Get a cached stream, returning null if it doesnt exist.
+     * @param stream name
+     * @return the cached stream
+     */
+    Stream getStream(String stream);
+
+    /**
+     * Get a cached stream and create a new one if it doesnt exist.
+     * @param streamName stream name
+     * @param start whether to start the stream after it is created.
+     * @return future satisfied once close complete
+     */
+    Stream getOrCreateStream(String streamName, boolean start) throws IOException;
+
+    /**
+     * Asynchronously create a new stream.
+     * @param stream
+     * @return Future satisfied once the stream is created
+     */
+    Future<Void> createStreamAsync(String stream);
+
+    /**
+     * Is acquiring stream allowed?
+     *
+     * @param stream
+     *          stream instance
+     * @return true if it is allowed to acquire this stream, otherwise false.
+     */
+    boolean allowAcquire(Stream stream);
+
+    /**
+     * Notify the manager that a stream was acquired.
+     * @param stream being acquired
+     */
+    void notifyAcquired(Stream stream);
+
+    /**
+     * Notify the manager that a stream was released.
+     * @param stream being released
+     */
+    void notifyReleased(Stream stream);
+
+    /**
+     * Notify the manager that a stream was completely removed.
+     * @param stream being uncached
+     * @return whether the stream existed or not
+     */
+    boolean notifyRemoved(Stream stream);
+
+    /**
+     * Asynchronous delete method.
+     * @param streamName stream name
+     * @return future satisfied once delete complete
+     */
+    Future<Void> deleteAndRemoveAsync(String streamName);
+
+    /**
+     * Asynchronous close and uncache method.
+     * @param streamName stream name
+     * @return future satisfied once close and uncache complete
+     */
+    Future<Void> closeAndRemoveAsync(String streamName);
+
+    /**
+     * Close and uncache after delayMs.
+     * @param stream to remove
+     */
+    void scheduleRemoval(Stream stream, long delayMs);
+
+    /**
+     * Close all stream.
+     * @return future satisfied all streams closed
+     */
+    Future<List<Void>> closeStreams();
+
+    /**
+     * Return map with stream ownership info.
+     * @param regex for filtering streams
+     * @return map containing ownership info
+     */
+    Map<String, String> getStreamOwnershipMap(Optional<String> regex);
+
+    /**
+     * Number of acquired streams.
+     * @return number of acquired streams
+     */
+    int numAcquired();
+
+    /**
+     * Number of cached streams.
+     * @return number of cached streams
+     */
+    int numCached();
+
+    /**
+     * Is the stream denoted by streamName in the acquired state.
+     * @return true if the stream is in the acquired state
+     */
+    boolean isAcquired(String streamName);
+
+    /**
+     * Close manager and disallow further activity.
+     */
+    void close();
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java
new file mode 100644
index 0000000..5d54738
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamManagerImpl.java
@@ -0,0 +1,413 @@
+/**
+ * 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.service.stream;
+
+import com.google.common.annotations.VisibleForTesting;
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.DistributedLogConfiguration;
+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;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.distributedlog.util.ConfUtils;
+import com.twitter.util.Future;
+import com.twitter.util.Promise;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.locks.ReentrantReadWriteLock;
+
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * StreamManagerImpl is the default implementation responsible for creating, destroying, and keeping track
+ * of Streams.
+ *
+ * <p>StreamFactory, supplied to StreamManagerImpl in the constructor below, is reposible simply for creating
+ * a stream object in isolation from the rest of the system. We pass a StreamFactory in instead of simply
+ * creating StreamImpl's ourselves in order to inject dependencies without bloating the StreamManagerImpl
+ * constructor.
+ */
+public class StreamManagerImpl implements StreamManager {
+
+    private static final Logger logger = LoggerFactory.getLogger(StreamManagerImpl.class);
+
+    private final ConcurrentHashMap<String, Stream> streams =
+        new ConcurrentHashMap<String, Stream>();
+    private final AtomicInteger numCached = new AtomicInteger(0);
+
+    private final ConcurrentHashMap<String, Stream> acquiredStreams =
+        new ConcurrentHashMap<String, Stream>();
+    private final AtomicInteger numAcquired = new AtomicInteger(0);
+
+    //
+    // Partitions
+    //
+    private final StreamPartitionConverter partitionConverter;
+    private final PartitionMap cachedPartitions = new PartitionMap();
+    private final PartitionMap acquiredPartitions = new PartitionMap();
+
+    final ReentrantReadWriteLock closeLock = new ReentrantReadWriteLock();
+    private final ScheduledExecutorService executorService;
+    private final DistributedLogConfiguration dlConfig;
+    private final StreamConfigProvider streamConfigProvider;
+    private final String clientId;
+    private boolean closed = false;
+    private final StreamFactory streamFactory;
+    private final DistributedLogNamespace dlNamespace;
+
+    public StreamManagerImpl(String clientId,
+                             DistributedLogConfiguration dlConfig,
+                             ScheduledExecutorService executorService,
+                             StreamFactory streamFactory,
+                             StreamPartitionConverter partitionConverter,
+                             StreamConfigProvider streamConfigProvider,
+                             DistributedLogNamespace dlNamespace) {
+        this.clientId = clientId;
+        this.executorService = executorService;
+        this.streamFactory = streamFactory;
+        this.partitionConverter = partitionConverter;
+        this.dlConfig = dlConfig;
+        this.streamConfigProvider = streamConfigProvider;
+        this.dlNamespace = dlNamespace;
+    }
+
+    private DynamicDistributedLogConfiguration getDynConf(String streamName) {
+        Optional<DynamicDistributedLogConfiguration> dynDlConf =
+                streamConfigProvider.getDynamicStreamConfig(streamName);
+        if (dynDlConf.isPresent()) {
+            return dynDlConf.get();
+        } else {
+            return ConfUtils.getConstDynConf(dlConfig);
+        }
+    }
+
+    @Override
+    public boolean allowAcquire(Stream stream) {
+        return acquiredPartitions.addPartition(
+                stream.getPartition(),
+                stream.getStreamConfiguration().getMaxAcquiredPartitionsPerProxy());
+    }
+
+    /**
+     * Must be enqueued to an executor to avoid deadlocks (close and execute-op both
+     * try to acquire the same read-write lock).
+     */
+    @Override
+    public Future<Void> deleteAndRemoveAsync(final String stream) {
+        final Promise<Void> result = new Promise<Void>();
+        java.util.concurrent.Future<?> scheduleFuture = schedule(new Runnable() {
+            @Override
+            public void run() {
+                result.become(doDeleteAndRemoveAsync(stream));
+            }
+        }, 0);
+        if (null == scheduleFuture) {
+            return Future.exception(
+                new ServiceUnavailableException("Couldn't schedule a delete task."));
+        }
+        return result;
+    }
+
+    /**
+     * Must be enqueued to an executor to avoid deadlocks (close and execute-op both
+     * try to acquire the same read-write lock).
+     */
+    @Override
+    public Future<Void> closeAndRemoveAsync(final String streamName) {
+        final Promise<Void> releasePromise = new Promise<Void>();
+        java.util.concurrent.Future<?> scheduleFuture = schedule(new Runnable() {
+            @Override
+            public void run() {
+                releasePromise.become(doCloseAndRemoveAsync(streamName));
+            }
+        }, 0);
+        if (null == scheduleFuture) {
+            return Future.exception(
+                new ServiceUnavailableException("Couldn't schedule a release task."));
+        }
+        return releasePromise;
+    }
+
+    @Override
+    public Future<Void> createStreamAsync(final String stream) {
+        final Promise<Void> createPromise = new Promise<Void>();
+        java.util.concurrent.Future<?> scheduleFuture = schedule(new Runnable() {
+            @Override
+            public void run() {
+                try {
+                    dlNamespace.createLog(stream);
+                    createPromise.setValue(null);
+                } catch (Exception e) {
+                    createPromise.setException(e);
+                }
+            }
+        }, 0);
+        if (null == scheduleFuture) {
+            return Future.exception(
+                new ServiceUnavailableException("Couldn't schedule a create task."));
+        }
+        return createPromise;
+    }
+
+    @Override
+    public void notifyReleased(Stream stream) {
+        acquiredPartitions.removePartition(stream.getPartition());
+        if (acquiredStreams.remove(stream.getStreamName(), stream)) {
+            numAcquired.getAndDecrement();
+        }
+    }
+
+    @Override
+    public void notifyAcquired(Stream stream) {
+        if (null == acquiredStreams.put(stream.getStreamName(), stream)) {
+            numAcquired.getAndIncrement();
+        }
+    }
+
+    @Override
+    public boolean notifyRemoved(Stream stream) {
+        cachedPartitions.removePartition(stream.getPartition());
+        if (streams.remove(stream.getStreamName(), stream)) {
+            numCached.getAndDecrement();
+            return true;
+        }
+        return false;
+    }
+
+    @Override
+    public Map<String, String> getStreamOwnershipMap(Optional<String> regex) {
+        Map<String, String> ownershipMap = new HashMap<String, String>();
+        for (Map.Entry<String, Stream> entry : acquiredStreams.entrySet()) {
+            String name = entry.getKey();
+            if (regex.isPresent() && !name.matches(regex.get())) {
+                continue;
+            }
+            Stream stream = entry.getValue();
+            if (null == stream) {
+                continue;
+            }
+            String owner = stream.getOwner();
+            if (null == owner) {
+                ownershipMap.put(name, clientId);
+            }
+        }
+        return ownershipMap;
+    }
+
+    @Override
+    public Stream getStream(String stream) {
+        return streams.get(stream);
+    }
+
+    @Override
+    public Stream getOrCreateStream(String streamName, boolean start) throws IOException {
+        Stream stream = streams.get(streamName);
+        if (null == stream) {
+            closeLock.readLock().lock();
+            try {
+                if (closed) {
+                    return null;
+                }
+                DynamicDistributedLogConfiguration dynConf = getDynConf(streamName);
+                int maxCachedPartitions = dynConf.getMaxCachedPartitionsPerProxy();
+
+                // get partition from the stream name
+                Partition partition = partitionConverter.convert(streamName);
+
+                // add partition to cached map
+                if (!cachedPartitions.addPartition(partition, maxCachedPartitions)) {
+                    throw new StreamUnavailableException("Stream " + streamName
+                            + " is not allowed to cache more than " + maxCachedPartitions + " partitions");
+                }
+
+                stream = newStream(streamName, dynConf);
+                Stream oldWriter = streams.putIfAbsent(streamName, stream);
+                if (null != oldWriter) {
+                    stream = oldWriter;
+                } else {
+                    numCached.getAndIncrement();
+                    logger.info("Inserted mapping stream name {} -> stream {}", streamName, stream);
+                    stream.initialize();
+                    if (start) {
+                        stream.start();
+                    }
+                }
+            } finally {
+                closeLock.readLock().unlock();
+            }
+        }
+        return stream;
+    }
+
+    @Override
+    public Future<List<Void>> closeStreams() {
+        int numAcquired = acquiredStreams.size();
+        int numCached = streams.size();
+        logger.info("Closing all acquired streams : acquired = {}, cached = {}.",
+            numAcquired, numCached);
+        Set<Stream> streamsToClose = new HashSet<Stream>();
+        streamsToClose.addAll(streams.values());
+        return closeStreams(streamsToClose, Optional.<RateLimiter>absent());
+    }
+
+    @Override
+    public void scheduleRemoval(final Stream stream, long delayMs) {
+        if (delayMs > 0) {
+            logger.info("Scheduling removal of stream {} from cache after {} sec.",
+                    stream.getStreamName(), delayMs);
+        }
+        schedule(new Runnable() {
+            @Override
+            public void run() {
+                if (notifyRemoved(stream)) {
+                    logger.info("Removed cached stream {} after probation.", stream.getStreamName());
+                } else {
+                    logger.info("Cached stream {} already removed.", stream.getStreamName());
+                }
+            }
+        }, delayMs);
+    }
+
+    @Override
+    public int numAcquired() {
+        return numAcquired.get();
+    }
+
+    @Override
+    public int numCached() {
+        return numCached.get();
+    }
+
+    @Override
+    public boolean isAcquired(String streamName) {
+        return acquiredStreams.containsKey(streamName);
+    }
+
+    @Override
+    public void close() {
+        closeLock.writeLock().lock();
+        try {
+            if (closed) {
+                return;
+            }
+            closed = true;
+        } finally {
+            closeLock.writeLock().unlock();
+        }
+    }
+
+    private Future<List<Void>> closeStreams(Set<Stream> streamsToClose, Optional<RateLimiter> rateLimiter) {
+        if (streamsToClose.isEmpty()) {
+            logger.info("No streams to close.");
+            List<Void> emptyList = new ArrayList<Void>();
+            return Future.value(emptyList);
+        }
+        List<Future<Void>> futures = new ArrayList<Future<Void>>(streamsToClose.size());
+        for (Stream stream : streamsToClose) {
+            if (rateLimiter.isPresent()) {
+                rateLimiter.get().acquire();
+            }
+            futures.add(stream.requestClose("Close Streams"));
+        }
+        return Future.collect(futures);
+    }
+
+    private Stream newStream(String name, DynamicDistributedLogConfiguration streamConf) {
+        return streamFactory.create(name, streamConf, this);
+    }
+
+    public Future<Void> doCloseAndRemoveAsync(final String streamName) {
+        Stream stream = streams.get(streamName);
+        if (null == stream) {
+            logger.info("No stream {} to release.", streamName);
+            return Future.value(null);
+        } else {
+            return stream.requestClose("release ownership");
+        }
+    }
+
+    /**
+     * Dont schedule if we're closed - closeLock is acquired to close, so if we acquire the
+     * lock and discover we're not closed, we won't schedule.
+     */
+    private java.util.concurrent.Future<?> schedule(Runnable runnable, long delayMs) {
+        closeLock.readLock().lock();
+        try {
+            if (closed) {
+                return null;
+            } else if (delayMs > 0) {
+                return executorService.schedule(runnable, delayMs, TimeUnit.MILLISECONDS);
+            } else {
+                return executorService.submit(runnable);
+            }
+        } catch (RejectedExecutionException ree) {
+            logger.error("Failed to schedule task {} in {} ms : ",
+                    new Object[] { runnable, delayMs, ree });
+            return null;
+        } finally {
+            closeLock.readLock().unlock();
+        }
+    }
+
+    private Future<Void> doDeleteAndRemoveAsync(final String streamName) {
+        Stream stream = streams.get(streamName);
+        if (null == stream) {
+            logger.warn("No stream {} to delete.", streamName);
+            return Future.exception(new UnexpectedException("No stream " + streamName + " to delete."));
+        } else {
+            Future<Void> result;
+            logger.info("Deleting stream {}, {}", streamName, stream);
+            try {
+                stream.delete();
+                result = stream.requestClose("Stream Deleted");
+            } catch (IOException e) {
+                logger.error("Failed on removing stream {} : ", streamName, e);
+                result = Future.exception(e);
+            }
+            return result;
+        }
+    }
+
+    @VisibleForTesting
+    public ConcurrentHashMap<String, Stream> getCachedStreams() {
+        return streams;
+    }
+
+    @VisibleForTesting
+    public ConcurrentHashMap<String, Stream> getAcquiredStreams() {
+        return acquiredStreams;
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOp.java
new file mode 100644
index 0000000..d0b8de4
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOp.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.service.stream;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.thrift.service.ResponseHeader;
+import org.apache.distributedlog.util.Sequencer;
+import com.twitter.util.Future;
+
+/**
+ * An operation applied to a stream.
+ */
+public interface StreamOp {
+    /**
+     * Execute a stream op with the supplied writer.
+     *
+     * @param writer active writer for applying the change
+     * @param sequencer sequencer used for generating transaction id for stream operations
+     * @param txnLock transaction lock to guarantee ordering of transaction id
+     * @return a future satisfied when the operation completes execution
+     */
+    Future<Void> execute(AsyncLogWriter writer,
+                         Sequencer sequencer,
+                         Object txnLock);
+
+    /**
+     * Invoked before the stream op is executed.
+     */
+    void preExecute() throws DLException;
+
+    /**
+     * Return the response header (containing the status code etc.).
+     *
+     * @return A future containing the response header or the exception
+     *      encountered by the op if it failed.
+     */
+    Future<ResponseHeader> responseHeader();
+
+    /**
+     * Abort the operation with the givem exception.
+     */
+    void fail(Throwable t);
+
+    /**
+     * Return the stream name.
+     */
+    String streamName();
+
+    /**
+     * Stopwatch gives the start time of the operation.
+     */
+    Stopwatch stopwatch();
+
+    /**
+     * Compute checksum from arguments.
+     */
+    Long computeChecksum();
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.java
new file mode 100644
index 0000000..f3fc610
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/StreamOpStats.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.service.stream;
+
+import org.apache.distributedlog.service.streamset.Partition;
+import org.apache.distributedlog.stats.BroadCastStatsLogger;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * Encapsulate stream op stats construction to make it easier to access stream
+ * op stats consistently from different scopes.
+ */
+public class StreamOpStats {
+    private final StatsLogger baseStatsLogger;
+    private final StatsLogger requestStatsLogger;
+    private final StatsLogger recordsStatsLogger;
+    private final StatsLogger requestDeniedStatsLogger;
+    private final StatsLogger streamStatsLogger;
+
+    public StreamOpStats(StatsLogger statsLogger,
+                         StatsLogger perStreamStatsLogger) {
+        this.baseStatsLogger = statsLogger;
+        this.requestStatsLogger = statsLogger.scope("request");
+        this.recordsStatsLogger = statsLogger.scope("records");
+        this.requestDeniedStatsLogger = statsLogger.scope("denied");
+        this.streamStatsLogger = perStreamStatsLogger;
+    }
+
+    public StatsLogger baseStatsLogger(String opName) {
+        return baseStatsLogger;
+    }
+
+    public Counter baseCounter(String opName) {
+        return baseStatsLogger.getCounter(opName);
+    }
+
+    public StatsLogger baseScope(String opName) {
+        return baseStatsLogger.scope(opName);
+    }
+
+    public OpStatsLogger requestLatencyStat(String opName) {
+        return requestStatsLogger.getOpStatsLogger(opName);
+    }
+
+    public StatsLogger requestScope(String scopeName) {
+        return requestStatsLogger.scope(scopeName);
+    }
+
+    public Counter scopedRequestCounter(String opName, String counterName) {
+        return requestScope(opName).getCounter(counterName);
+    }
+
+    public Counter requestCounter(String counterName) {
+        return requestStatsLogger.getCounter(counterName);
+    }
+
+    public Counter requestPendingCounter(String counterName) {
+        return requestCounter(counterName);
+    }
+
+    public Counter requestDeniedCounter(String counterName) {
+        return requestDeniedStatsLogger.getCounter(counterName);
+    }
+
+    public Counter recordsCounter(String counterName) {
+        return recordsStatsLogger.getCounter(counterName);
+    }
+
+    public StatsLogger streamRequestStatsLogger(Partition partition) {
+        return BroadCastStatsLogger.masterslave(
+            streamStatsLogger.scope(partition.getStream()).scope("partition")
+                .scope(partition.getPaddedId()), streamStatsLogger.scope(partition.getStream())
+                .scope("aggregate"));
+    }
+
+    public StatsLogger streamRequestScope(Partition partition, String scopeName) {
+        return streamRequestStatsLogger(partition).scope(scopeName);
+    }
+
+    public OpStatsLogger streamRequestLatencyStat(Partition partition, String opName) {
+        return streamRequestStatsLogger(partition).getOpStatsLogger(opName);
+    }
+
+    public Counter streamRequestCounter(Partition partition, String opName, String counterName) {
+        return streamRequestScope(partition, opName).getCounter(counterName);
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.java
new file mode 100644
index 0000000..7a38d14
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/TruncateOp.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.service.stream;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+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.ProtocolUtils;
+import org.apache.distributedlog.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.
+ */
+public class TruncateOp extends AbstractWriteOp {
+
+    private static final Logger logger = LoggerFactory.getLogger(TruncateOp.class);
+
+    private final Counter deniedTruncateCounter;
+    private final DLSN dlsn;
+    private final AccessControlManager accessControlManager;
+
+    public TruncateOp(String stream,
+                      DLSN dlsn,
+                      StatsLogger statsLogger,
+                      StatsLogger perStreamStatsLogger,
+                      Long checksum,
+                      Feature checksumDisabledFeature,
+                      AccessControlManager accessControlManager) {
+        super(stream, requestStat(statsLogger, "truncate"), checksum, checksumDisabledFeature);
+        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+        this.deniedTruncateCounter = streamOpStats.requestDeniedCounter("truncate");
+        this.accessControlManager = accessControlManager;
+        this.dlsn = dlsn;
+    }
+
+    @Override
+    public Long computeChecksum() {
+        return ProtocolUtils.truncateOpCRC32(stream, dlsn);
+    }
+
+    @Override
+    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
+                                              Sequencer sequencer,
+                                              Object txnLock) {
+        if (!stream.equals(writer.getStreamName())) {
+            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();
+            }
+        });
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!accessControlManager.allowTruncate(stream)) {
+            deniedTruncateCounter.inc();
+            throw new RequestDeniedException(stream, "truncate");
+        }
+        super.preExecute();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java
new file mode 100644
index 0000000..c4bdcc2
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOp.java
@@ -0,0 +1,173 @@
+/**
+ * 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.service.stream;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecord;
+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.service.config.ServerConfiguration;
+import org.apache.distributedlog.service.streamset.Partition;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+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.ProtocolUtils;
+import org.apache.distributedlog.util.Sequencer;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import java.nio.ByteBuffer;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.feature.Feature;
+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 scala.runtime.AbstractFunction1;
+
+/**
+ * Operation to write a single record to a log stream.
+ */
+public class WriteOp extends AbstractWriteOp implements WriteOpWithPayload {
+
+    private static final Logger logger = LoggerFactory.getLogger(WriteOp.class);
+
+    private final byte[] payload;
+    private final boolean isRecordSet;
+
+    // Stats
+    private final Counter deniedWriteCounter;
+    private final Counter successRecordCounter;
+    private final Counter failureRecordCounter;
+    private final Counter redirectRecordCounter;
+    private final OpStatsLogger latencyStat;
+    private final Counter bytes;
+    private final Counter writeBytes;
+
+    private final byte dlsnVersion;
+    private final AccessControlManager accessControlManager;
+
+    public WriteOp(String stream,
+                   ByteBuffer data,
+                   StatsLogger statsLogger,
+                   StatsLogger perStreamStatsLogger,
+                   StreamPartitionConverter streamPartitionConverter,
+                   ServerConfiguration conf,
+                   byte dlsnVersion,
+                   Long checksum,
+                   boolean isRecordSet,
+                   Feature checksumDisabledFeature,
+                   AccessControlManager accessControlManager) {
+        super(stream, requestStat(statsLogger, "write"), checksum, checksumDisabledFeature);
+        payload = new byte[data.remaining()];
+        data.get(payload);
+        this.isRecordSet = isRecordSet;
+
+        final Partition partition = streamPartitionConverter.convert(stream);
+        StreamOpStats streamOpStats = new StreamOpStats(statsLogger, perStreamStatsLogger);
+        this.successRecordCounter = streamOpStats.recordsCounter("success");
+        this.failureRecordCounter = streamOpStats.recordsCounter("failure");
+        this.redirectRecordCounter = streamOpStats.recordsCounter("redirect");
+        this.deniedWriteCounter = streamOpStats.requestDeniedCounter("write");
+        this.writeBytes = streamOpStats.scopedRequestCounter("write", "bytes");
+        this.latencyStat = streamOpStats.streamRequestLatencyStat(partition, "write");
+        this.bytes = streamOpStats.streamRequestCounter(partition, "write", "bytes");
+
+        this.dlsnVersion = dlsnVersion;
+        this.accessControlManager = accessControlManager;
+
+        final long size = getPayloadSize();
+        result().addEventListener(new FutureEventListener<WriteResponse>() {
+            @Override
+            public void onSuccess(WriteResponse response) {
+                if (response.getHeader().getCode() == StatusCode.SUCCESS) {
+                    latencyStat.registerSuccessfulEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
+                    bytes.add(size);
+                    writeBytes.add(size);
+                } else {
+                    latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
+                }
+            }
+            @Override
+            public void onFailure(Throwable cause) {
+                latencyStat.registerFailedEvent(stopwatch().elapsed(TimeUnit.MICROSECONDS));
+            }
+        });
+    }
+
+    @Override
+    public long getPayloadSize() {
+      return payload.length;
+    }
+
+    @Override
+    public Long computeChecksum() {
+        return ProtocolUtils.writeOpCRC32(stream, payload);
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!accessControlManager.allowWrite(stream)) {
+            deniedWriteCounter.inc();
+            throw new RequestDeniedException(stream, "write");
+        }
+        super.preExecute();
+    }
+
+    @Override
+    protected Future<WriteResponse> executeOp(AsyncLogWriter writer,
+                                              Sequencer sequencer,
+                                              Object txnLock) {
+        if (!stream.equals(writer.getStreamName())) {
+            logger.error("Write: Stream Name Mismatch in the Stream Map {}, {}", stream, writer.getStreamName());
+            return Future.exception(new IllegalStateException("The stream mapping is incorrect, fail the request"));
+        }
+
+        long txnId;
+        Future<DLSN> writeResult;
+        synchronized (txnLock) {
+            txnId = sequencer.nextId();
+            LogRecord record = new LogRecord(txnId, payload);
+            if (isRecordSet) {
+                record.setRecordSet();
+            }
+            writeResult = writer.write(record);
+        }
+        return writeResult.map(new AbstractFunction1<DLSN, WriteResponse>() {
+            @Override
+            public WriteResponse apply(DLSN value) {
+                successRecordCounter.inc();
+                return ResponseUtils.writeSuccess().setDlsn(value.serialize(dlsnVersion));
+            }
+        });
+    }
+
+    @Override
+    protected void fail(ResponseHeader header) {
+        if (StatusCode.FOUND == header.getCode()) {
+            redirectRecordCounter.inc();
+        } else {
+            failureRecordCounter.inc();
+        }
+        super.fail(header);
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOpWithPayload.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOpWithPayload.java
new file mode 100644
index 0000000..e411b420
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/WriteOpWithPayload.java
@@ -0,0 +1,27 @@
+/**
+ * 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.service.stream;
+
+/**
+ * A write operation with payload.
+ */
+public interface WriteOpWithPayload {
+
+    // Return the payload size in bytes
+    long getPayloadSize();
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/AdminOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/AdminOp.java
new file mode 100644
index 0000000..fcaee35
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/AdminOp.java
@@ -0,0 +1,40 @@
+/**
+ * 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.service.stream.admin;
+
+import org.apache.distributedlog.exceptions.DLException;
+import com.twitter.util.Future;
+
+/**
+ * Admin operation interface.
+ */
+public interface AdminOp<RespT> {
+
+    /**
+     * Invoked before the stream op is executed.
+     */
+    void preExecute() throws DLException;
+
+    /**
+     * Execute the operation.
+     *
+     * @return the future represents the response of the operation
+     */
+    Future<RespT> execute();
+
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/CreateOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/CreateOp.java
new file mode 100644
index 0000000..89a2566
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/CreateOp.java
@@ -0,0 +1,57 @@
+/**
+ * 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.service.stream.admin;
+
+import static org.apache.distributedlog.service.stream.AbstractStreamOp.requestStat;
+
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.service.stream.StreamManager;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.StatsLogger;
+import scala.runtime.AbstractFunction1;
+
+/**
+ * Operation to create log stream.
+ */
+public class CreateOp extends StreamAdminOp {
+
+  public CreateOp(String stream,
+                  StatsLogger statsLogger,
+                  StreamManager streamManager,
+                  Long checksum,
+                  Feature checksumEnabledFeature) {
+    super(stream,
+            streamManager,
+            requestStat(statsLogger, "create"),
+            checksum,
+            checksumEnabledFeature);
+  }
+
+  @Override
+  protected Future<WriteResponse> executeOp() {
+    Future<Void> result = streamManager.createStreamAsync(stream);
+    return result.map(new AbstractFunction1<Void, WriteResponse>() {
+      @Override
+      public WriteResponse apply(Void value) {
+        return ResponseUtils.writeSuccess();
+      }
+    });
+  }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/StreamAdminOp.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/StreamAdminOp.java
new file mode 100644
index 0000000..3f28c42
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/StreamAdminOp.java
@@ -0,0 +1,100 @@
+/**
+ * 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.service.stream.admin;
+
+import com.google.common.base.Stopwatch;
+import org.apache.distributedlog.exceptions.ChecksumFailedException;
+import org.apache.distributedlog.exceptions.DLException;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.service.stream.StreamManager;
+import org.apache.distributedlog.thrift.service.WriteResponse;
+import org.apache.distributedlog.util.ProtocolUtils;
+import com.twitter.util.Future;
+import com.twitter.util.FutureTransformer;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.OpStatsLogger;
+
+/**
+ * Stream admin op.
+ */
+public abstract class StreamAdminOp implements AdminOp<WriteResponse> {
+
+    protected final String stream;
+    protected final StreamManager streamManager;
+    protected final OpStatsLogger opStatsLogger;
+    protected final Stopwatch stopwatch = Stopwatch.createUnstarted();
+    protected final Long checksum;
+    protected final Feature checksumDisabledFeature;
+
+    protected StreamAdminOp(String stream,
+                            StreamManager streamManager,
+                            OpStatsLogger statsLogger,
+                            Long checksum,
+                            Feature checksumDisabledFeature) {
+        this.stream = stream;
+        this.streamManager = streamManager;
+        this.opStatsLogger = statsLogger;
+        // start here in case the operation is failed before executing.
+        stopwatch.reset().start();
+        this.checksum = checksum;
+        this.checksumDisabledFeature = checksumDisabledFeature;
+    }
+
+    protected Long computeChecksum() {
+        return ProtocolUtils.streamOpCRC32(stream);
+    }
+
+    @Override
+    public void preExecute() throws DLException {
+        if (!checksumDisabledFeature.isAvailable() && null != checksum) {
+            Long serverChecksum = computeChecksum();
+            if (null != serverChecksum && !checksum.equals(serverChecksum)) {
+                throw new ChecksumFailedException();
+            }
+        }
+    }
+
+    /**
+     * Execute the operation.
+     *
+     * @return execute operation
+     */
+    protected abstract Future<WriteResponse> executeOp();
+
+    @Override
+    public Future<WriteResponse> execute() {
+        return executeOp().transformedBy(new FutureTransformer<WriteResponse, WriteResponse>() {
+
+            @Override
+            public WriteResponse map(WriteResponse response) {
+                opStatsLogger.registerSuccessfulEvent(
+                        stopwatch.elapsed(TimeUnit.MICROSECONDS));
+                return response;
+            }
+
+            @Override
+            public WriteResponse handle(Throwable cause) {
+                opStatsLogger.registerFailedEvent(
+                        stopwatch.elapsed(TimeUnit.MICROSECONDS));
+                return ResponseUtils.write(ResponseUtils.exceptionToHeader(cause));
+            }
+
+        });
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/package-info.java
new file mode 100644
index 0000000..5b583e1
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/admin/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Stream Related Admin Operations.
+ */
+package org.apache.distributedlog.service.stream.admin;
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/DynamicRequestLimiter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/DynamicRequestLimiter.java
new file mode 100644
index 0000000..5db2037
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/DynamicRequestLimiter.java
@@ -0,0 +1,94 @@
+/**
+ * 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.service.stream.limiter;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.limiter.RequestLimiter;
+import java.io.Closeable;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.apache.commons.configuration.event.ConfigurationEvent;
+import org.apache.commons.configuration.event.ConfigurationListener;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Dynamically rebuild a rate limiter when the supplied dynamic config changes.
+ *
+ * <p>Subclasses implement build() to build the limiter. DynamicRequestLimiter must be closed to deregister
+ * the config listener.
+ */
+public abstract class DynamicRequestLimiter<Req> implements RequestLimiter<Req>, Closeable {
+    private static final Logger LOG = LoggerFactory.getLogger(DynamicRequestLimiter.class);
+
+    private final ConfigurationListener listener;
+    private final Feature rateLimitDisabledFeature;
+    volatile RequestLimiter<Req> limiter;
+    final DynamicDistributedLogConfiguration dynConf;
+
+    public DynamicRequestLimiter(DynamicDistributedLogConfiguration dynConf,
+                                 StatsLogger statsLogger,
+                                 Feature rateLimitDisabledFeature) {
+        final StatsLogger limiterStatsLogger = statsLogger.scope("dynamic");
+        this.dynConf = dynConf;
+        this.rateLimitDisabledFeature = rateLimitDisabledFeature;
+        this.listener = new ConfigurationListener() {
+            @Override
+            public void configurationChanged(ConfigurationEvent event) {
+                // Note that this method may be called several times if several config options
+                // are changed. The effect is harmless except that we create and discard more
+                // objects than we need to.
+                LOG.debug("Config changed callback invoked with event {} {} {} {}", new Object[] {
+                        event.getPropertyName(), event.getPropertyValue(), event.getType(),
+                        event.isBeforeUpdate()});
+                if (!event.isBeforeUpdate()) {
+                    limiterStatsLogger.getCounter("config_changed").inc();
+                    LOG.debug("Rebuilding limiter");
+                    limiter = build();
+                }
+            }
+        };
+        LOG.debug("Registering config changed callback");
+        dynConf.addConfigurationListener(listener);
+    }
+
+    public void initialize() {
+        this.limiter = build();
+    }
+
+    @Override
+    public void apply(Req request) throws OverCapacityException {
+        if (rateLimitDisabledFeature.isAvailable()) {
+            return;
+        }
+        limiter.apply(request);
+    }
+
+    @Override
+    public void close() {
+        boolean success = dynConf.removeConfigurationListener(listener);
+        LOG.debug("Deregistering config changed callback success={}", success);
+    }
+
+   /**
+    * Build the underlying limiter. Called when DynamicRequestLimiter detects config has changed.
+    * This may be called multiple times so the method should be cheap.
+    */
+    protected abstract RequestLimiter<Req> build();
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/RequestLimiterBuilder.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/RequestLimiterBuilder.java
new file mode 100644
index 0000000..fc30599
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/RequestLimiterBuilder.java
@@ -0,0 +1,116 @@
+/**
+ * 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.service.stream.limiter;
+
+import static com.google.common.base.Preconditions.checkNotNull;
+
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter.CostFunction;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
+import org.apache.distributedlog.limiter.GuavaRateLimiter;
+import org.apache.distributedlog.limiter.RateLimiter;
+import org.apache.distributedlog.limiter.RequestLimiter;
+import org.apache.distributedlog.service.stream.StreamOp;
+import org.apache.distributedlog.service.stream.WriteOpWithPayload;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * Request limiter builder.
+ */
+public class RequestLimiterBuilder {
+    private OverlimitFunction<StreamOp> overlimitFunction = NOP_OVERLIMIT_FUNCTION;
+    private RateLimiter limiter;
+    private CostFunction<StreamOp> costFunction;
+    private StatsLogger statsLogger = NullStatsLogger.INSTANCE;
+
+    /**
+     * Function to calculate the `RPS` (Request per second) cost of a given stream operation.
+     */
+    public static final CostFunction<StreamOp> RPS_COST_FUNCTION = new CostFunction<StreamOp>() {
+        @Override
+        public int apply(StreamOp op) {
+            if (op instanceof WriteOpWithPayload) {
+                return 1;
+            } else {
+                return 0;
+            }
+        }
+    };
+
+    /**
+     * Function to calculate the `BPS` (Bytes per second) cost of a given stream operation.
+     */
+    public static final CostFunction<StreamOp> BPS_COST_FUNCTION = new CostFunction<StreamOp>() {
+        @Override
+        public int apply(StreamOp op) {
+            if (op instanceof WriteOpWithPayload) {
+                WriteOpWithPayload writeOp = (WriteOpWithPayload) op;
+                return (int) Math.min(writeOp.getPayloadSize(), Integer.MAX_VALUE);
+            } else {
+                return 0;
+            }
+        }
+    };
+
+    /**
+     * Function to check if a stream operation will cause {@link OverCapacityException}.
+     */
+    public static final OverlimitFunction<StreamOp> NOP_OVERLIMIT_FUNCTION = new OverlimitFunction<StreamOp>() {
+        @Override
+        public void apply(StreamOp op) throws OverCapacityException {
+            return;
+        }
+    };
+
+    public RequestLimiterBuilder limit(int limit) {
+        this.limiter = GuavaRateLimiter.of(limit);
+        return this;
+    }
+
+    public RequestLimiterBuilder overlimit(OverlimitFunction<StreamOp> overlimitFunction) {
+        this.overlimitFunction = overlimitFunction;
+        return this;
+    }
+
+    public RequestLimiterBuilder cost(CostFunction<StreamOp> costFunction) {
+        this.costFunction = costFunction;
+        return this;
+    }
+
+    public RequestLimiterBuilder statsLogger(StatsLogger statsLogger) {
+        this.statsLogger = statsLogger;
+        return this;
+    }
+
+    public static RequestLimiterBuilder newRpsLimiterBuilder() {
+        return new RequestLimiterBuilder().cost(RPS_COST_FUNCTION);
+    }
+
+    public static RequestLimiterBuilder newBpsLimiterBuilder() {
+        return new RequestLimiterBuilder().cost(BPS_COST_FUNCTION);
+    }
+
+    public RequestLimiter<StreamOp> build() {
+        checkNotNull(limiter);
+        checkNotNull(overlimitFunction);
+        checkNotNull(costFunction);
+        return new ComposableRequestLimiter(limiter, overlimitFunction, costFunction, statsLogger);
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java
new file mode 100644
index 0000000..de805aa
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/ServiceRequestLimiter.java
@@ -0,0 +1,103 @@
+/**
+ * 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.service.stream.limiter;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+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.service.stream.StreamManager;
+import org.apache.distributedlog.service.stream.StreamOp;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * Request limiter for the service instance (global request limiter).
+ */
+public class ServiceRequestLimiter extends DynamicRequestLimiter<StreamOp> {
+    private final StatsLogger limiterStatLogger;
+    private final MovingAverageRate serviceRps;
+    private final MovingAverageRate serviceBps;
+    private final StreamManager streamManager;
+
+    public ServiceRequestLimiter(DynamicDistributedLogConfiguration dynConf,
+                                 StatsLogger statsLogger,
+                                 MovingAverageRate serviceRps,
+                                 MovingAverageRate serviceBps,
+                                 StreamManager streamManager,
+                                 Feature disabledFeature) {
+        super(dynConf, statsLogger, disabledFeature);
+        this.limiterStatLogger = statsLogger;
+        this.streamManager = streamManager;
+        this.serviceRps = serviceRps;
+        this.serviceBps = serviceBps;
+        this.limiter = build();
+    }
+
+    @Override
+    public RequestLimiter<StreamOp> build() {
+        int rpsStreamAcquireLimit = dynConf.getRpsStreamAcquireServiceLimit();
+        int rpsSoftServiceLimit = dynConf.getRpsSoftServiceLimit();
+        int rpsHardServiceLimit = dynConf.getRpsHardServiceLimit();
+        int bpsStreamAcquireLimit = dynConf.getBpsStreamAcquireServiceLimit();
+        int bpsSoftServiceLimit = dynConf.getBpsSoftServiceLimit();
+        int bpsHardServiceLimit = dynConf.getBpsHardServiceLimit();
+
+        RequestLimiterBuilder rpsHardLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("rps_hard_limit"))
+            .limit(rpsHardServiceLimit)
+            .overlimit(new OverlimitFunction<StreamOp>() {
+                @Override
+                public void apply(StreamOp request) throws OverCapacityException {
+                    throw new OverCapacityException("Being rate limited: RPS limit exceeded for the service instance");
+                }
+            });
+
+        RequestLimiterBuilder rpsSoftLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("rps_soft_limit"))
+            .limit(rpsSoftServiceLimit);
+
+        RequestLimiterBuilder bpsHardLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("bps_hard_limit"))
+            .limit(bpsHardServiceLimit)
+            .overlimit(new OverlimitFunction<StreamOp>() {
+                @Override
+                public void apply(StreamOp request) throws OverCapacityException {
+                    throw new OverCapacityException("Being rate limited: BPS limit exceeded for the service instance");
+                }
+            });
+
+        RequestLimiterBuilder bpsSoftLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("bps_soft_limit"))
+            .limit(bpsSoftServiceLimit);
+
+        ChainedRequestLimiter.Builder<StreamOp> builder = new ChainedRequestLimiter.Builder<StreamOp>();
+        builder.addLimiter(new StreamAcquireLimiter(
+            streamManager, serviceRps, rpsStreamAcquireLimit, limiterStatLogger.scope("rps_acquire")));
+        builder.addLimiter(new StreamAcquireLimiter(
+            streamManager, serviceBps, bpsStreamAcquireLimit, limiterStatLogger.scope("bps_acquire")));
+        builder.addLimiter(bpsHardLimiterBuilder.build());
+        builder.addLimiter(bpsSoftLimiterBuilder.build());
+        builder.addLimiter(rpsHardLimiterBuilder.build());
+        builder.addLimiter(rpsSoftLimiterBuilder.build());
+        builder.statsLogger(limiterStatLogger);
+        return builder.build();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java
new file mode 100644
index 0000000..7675d6f
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamAcquireLimiter.java
@@ -0,0 +1,56 @@
+/**
+ * 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.service.stream.limiter;
+
+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.service.stream.StreamManager;
+import org.apache.distributedlog.service.stream.StreamOp;
+import org.apache.bookkeeper.stats.Counter;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * A special limiter on limiting acquiring new streams.
+ */
+public class StreamAcquireLimiter implements RequestLimiter<StreamOp> {
+    private final StreamManager streamManager;
+    private final MovingAverageRate serviceRps;
+    private final double serviceRpsLimit;
+    private final Counter overlimitCounter;
+
+    public StreamAcquireLimiter(StreamManager streamManager,
+                                MovingAverageRate serviceRps,
+                                double serviceRpsLimit,
+                                StatsLogger statsLogger) {
+        this.streamManager = streamManager;
+        this.serviceRps = serviceRps;
+        this.serviceRpsLimit = serviceRpsLimit;
+        this.overlimitCounter = statsLogger.getCounter("overlimit");
+    }
+
+    @Override
+    public void apply(StreamOp op) throws OverCapacityException {
+        String streamName = op.streamName();
+        if (serviceRpsLimit > -1 && serviceRps.get() > serviceRpsLimit && !streamManager.isAcquired(streamName)) {
+            overlimitCounter.inc();
+            throw new TooManyStreamsException("Request rate is too high to accept new stream " + streamName + ".");
+        }
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamRequestLimiter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamRequestLimiter.java
new file mode 100644
index 0000000..42b4e1e
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/StreamRequestLimiter.java
@@ -0,0 +1,87 @@
+/**
+ * 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.service.stream.limiter;
+
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.limiter.ChainedRequestLimiter;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
+import org.apache.distributedlog.limiter.RequestLimiter;
+import org.apache.distributedlog.service.stream.StreamOp;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.stats.StatsLogger;
+
+/**
+ * A dynamic request limiter on limiting stream operations.
+ */
+public class StreamRequestLimiter extends DynamicRequestLimiter<StreamOp> {
+    private final DynamicDistributedLogConfiguration dynConf;
+    private final StatsLogger limiterStatLogger;
+    private final String streamName;
+
+    public StreamRequestLimiter(String streamName,
+                                DynamicDistributedLogConfiguration dynConf,
+                                StatsLogger statsLogger,
+                                Feature disabledFeature) {
+        super(dynConf, statsLogger, disabledFeature);
+        this.limiterStatLogger = statsLogger;
+        this.dynConf = dynConf;
+        this.streamName = streamName;
+        this.limiter = build();
+    }
+
+    @Override
+    public RequestLimiter<StreamOp> build() {
+
+        // RPS hard, soft limits
+        RequestLimiterBuilder rpsHardLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("rps_hard_limit"))
+            .limit(dynConf.getRpsHardWriteLimit())
+            .overlimit(new OverlimitFunction<StreamOp>() {
+                @Override
+                public void apply(StreamOp op) throws OverCapacityException {
+                    throw new OverCapacityException("Being rate limited: RPS limit exceeded for stream " + streamName);
+                }
+            });
+        RequestLimiterBuilder rpsSoftLimiterBuilder = RequestLimiterBuilder.newRpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("rps_soft_limit"))
+            .limit(dynConf.getRpsSoftWriteLimit());
+
+        // BPS hard, soft limits
+        RequestLimiterBuilder bpsHardLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("bps_hard_limit"))
+            .limit(dynConf.getBpsHardWriteLimit())
+            .overlimit(new OverlimitFunction<StreamOp>() {
+                @Override
+                public void apply(StreamOp op) throws OverCapacityException {
+                    throw new OverCapacityException("Being rate limited: BPS limit exceeded for stream " + streamName);
+                }
+            });
+        RequestLimiterBuilder bpsSoftLimiterBuilder = RequestLimiterBuilder.newBpsLimiterBuilder()
+            .statsLogger(limiterStatLogger.scope("bps_soft_limit"))
+            .limit(dynConf.getBpsSoftWriteLimit());
+
+        ChainedRequestLimiter.Builder<StreamOp> builder = new ChainedRequestLimiter.Builder<StreamOp>();
+        builder.addLimiter(rpsSoftLimiterBuilder.build());
+        builder.addLimiter(rpsHardLimiterBuilder.build());
+        builder.addLimiter(bpsSoftLimiterBuilder.build());
+        builder.addLimiter(bpsHardLimiterBuilder.build());
+        builder.statsLogger(limiterStatLogger);
+        return builder.build();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/package-info.java
new file mode 100644
index 0000000..c666b08
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/limiter/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Request Rate Limiting.
+ */
+package org.apache.distributedlog.service.stream.limiter;
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/package-info.java
new file mode 100644
index 0000000..7429a85
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/stream/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Stream Related Operations.
+ */
+package org.apache.distributedlog.service.stream;
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/CacheableStreamPartitionConverter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/CacheableStreamPartitionConverter.java
new file mode 100644
index 0000000..72668c2
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/CacheableStreamPartitionConverter.java
@@ -0,0 +1,58 @@
+/**
+ * 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.service.streamset;
+
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * A stream-to-partition converter that caches the mapping between stream and partitions.
+ */
+public abstract class CacheableStreamPartitionConverter implements StreamPartitionConverter {
+
+    private final ConcurrentMap<String, Partition> partitions;
+
+    protected CacheableStreamPartitionConverter() {
+        this.partitions = new ConcurrentHashMap<String, Partition>();
+    }
+
+    @Override
+    public Partition convert(String streamName) {
+        Partition p = partitions.get(streamName);
+        if (null != p) {
+            return p;
+        }
+        // not found
+        Partition newPartition = newPartition(streamName);
+        Partition oldPartition = partitions.putIfAbsent(streamName, newPartition);
+        if (null == oldPartition) {
+            return newPartition;
+        } else {
+            return oldPartition;
+        }
+    }
+
+    /**
+     * Create the partition from <code>streamName</code>.
+     *
+     * @param streamName
+     *          stream name
+     * @return partition id of the stream
+     */
+    protected abstract Partition newPartition(String streamName);
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/DelimiterStreamPartitionConverter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/DelimiterStreamPartitionConverter.java
new file mode 100644
index 0000000..30b2896
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/DelimiterStreamPartitionConverter.java
@@ -0,0 +1,50 @@
+/**
+ * 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.service.streamset;
+
+import org.apache.commons.lang3.StringUtils;
+
+/**
+ * Stream Partition Converter that converts the stream name into a stream-to-partition mapping by delimiter.
+ */
+public class DelimiterStreamPartitionConverter extends CacheableStreamPartitionConverter {
+
+    private final String delimiter;
+
+    public DelimiterStreamPartitionConverter() {
+        this("_");
+    }
+
+    public DelimiterStreamPartitionConverter(String delimiter) {
+        this.delimiter = delimiter;
+    }
+
+    @Override
+    protected Partition newPartition(String streamName) {
+        String[] parts = StringUtils.split(streamName, delimiter);
+        if (null != parts && parts.length == 2) {
+            try {
+                int partition = Integer.parseInt(parts[1]);
+                return new Partition(parts[0], partition);
+            } catch (NumberFormatException nfe) {
+                // ignore the exception
+            }
+        }
+        return new Partition(streamName, 0);
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/IdentityStreamPartitionConverter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/IdentityStreamPartitionConverter.java
new file mode 100644
index 0000000..5be172f
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/IdentityStreamPartitionConverter.java
@@ -0,0 +1,28 @@
+/**
+ * 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.service.streamset;
+
+/**
+ * Map stream name to partition of the same name.
+ */
+public class IdentityStreamPartitionConverter extends CacheableStreamPartitionConverter {
+    @Override
+    protected Partition newPartition(String streamName) {
+        return new Partition(streamName, 0);
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/Partition.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/Partition.java
new file mode 100644
index 0000000..aa69276
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/Partition.java
@@ -0,0 +1,99 @@
+/**
+ * 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.service.streamset;
+
+import com.google.common.base.Objects;
+
+/**
+ * `Partition` defines the relationship between a `virtual` stream and a
+ * physical DL stream.
+ *
+ * <p>A `virtual` stream could be partitioned into multiple partitions
+ * and each partition is effectively a DL stream.
+ */
+public class Partition {
+
+    // Name of its parent stream.
+    private final String stream;
+
+    // Unique id of the partition within the stream.
+    // It can be just simply an index id.
+    public final int id;
+
+    public Partition(String stream, int id) {
+        this.stream = stream;
+        this.id = id;
+    }
+
+    /**
+     * Get the `virtual` stream name.
+     *
+     * @return the stream name.
+     */
+    public String getStream() {
+        return stream;
+    }
+
+    /**
+     * Get the partition id of this partition.
+     *
+     * @return partition id
+     */
+    public int getId() {
+        return id;
+    }
+
+    /**
+     * Get the 6 digit 0 padded id of this partition as a String.
+     * @return partition id
+     */
+    public String getPaddedId() {
+        return String.format("%06d", getId());
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) {
+            return true;
+        }
+        if (!(o instanceof Partition)) {
+            return false;
+        }
+        Partition partition = (Partition) o;
+
+        return id == partition.id && Objects.equal(stream, partition.stream);
+    }
+
+    @Override
+    public int hashCode() {
+        int result = stream.hashCode();
+        result = 31 * result + id;
+        return result;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder sb = new StringBuilder();
+        sb.append("Partition(")
+          .append(stream)
+          .append(", ")
+          .append(id)
+          .append(")");
+        return sb.toString();
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/PartitionMap.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/PartitionMap.java
new file mode 100644
index 0000000..bfcc5db
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/PartitionMap.java
@@ -0,0 +1,58 @@
+/**
+ * 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.service.streamset;
+
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+
+/**
+ * A mapping between a logical stream and a set of physical partitions.
+ */
+public class PartitionMap {
+
+    private final Map<String, Set<Partition>> partitionMap;
+
+    public PartitionMap() {
+        partitionMap = new HashMap<String, Set<Partition>>();
+    }
+
+    public synchronized boolean addPartition(Partition partition, int maxPartitions) {
+        if (maxPartitions <= 0) {
+            return true;
+        }
+        Set<Partition> partitions = partitionMap.get(partition.getStream());
+        if (null == partitions) {
+            partitions = new HashSet<Partition>();
+            partitions.add(partition);
+            partitionMap.put(partition.getStream(), partitions);
+            return true;
+        }
+        if (partitions.contains(partition) || partitions.size() < maxPartitions) {
+            partitions.add(partition);
+            return true;
+        }
+        return false;
+    }
+
+    public synchronized boolean removePartition(Partition partition) {
+        Set<Partition> partitions = partitionMap.get(partition.getStream());
+        return null != partitions && partitions.remove(partition);
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/StreamPartitionConverter.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/StreamPartitionConverter.java
new file mode 100644
index 0000000..3ea1337
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/StreamPartitionConverter.java
@@ -0,0 +1,35 @@
+/**
+ * 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.service.streamset;
+
+/**
+ * Map stream name to a partition.
+ *
+ * @see Partition
+ */
+public interface StreamPartitionConverter {
+
+    /**
+     * Convert the stream name to partition.
+     *
+     * @param streamName
+     *          stream name
+     * @return partition
+     */
+    Partition convert(String streamName);
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/package-info.java
new file mode 100644
index 0000000..d185e88
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/streamset/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * StreamSet - A logical set of streams.
+ */
+package org.apache.distributedlog.service.streamset;
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/ProxyTool.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/ProxyTool.java
new file mode 100644
index 0000000..3934eb5
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/ProxyTool.java
@@ -0,0 +1,350 @@
+/**
+ * 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.service.tools;
+
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.client.serverset.DLZkServerSet;
+import org.apache.distributedlog.service.ClientUtils;
+import org.apache.distributedlog.service.DLSocketAddress;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import org.apache.distributedlog.tools.Tool;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Await;
+import com.twitter.util.Duration;
+import java.net.InetSocketAddress;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+import org.apache.commons.cli.CommandLine;
+import org.apache.commons.cli.Options;
+import org.apache.commons.cli.ParseException;
+import org.apache.commons.lang3.tuple.Pair;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Tools to interact with proxies.
+ */
+public class ProxyTool extends Tool {
+
+    private static final Logger logger = LoggerFactory.getLogger(ProxyTool.class);
+
+    /**
+     * Abstract Cluster level command.
+     */
+    protected abstract static class ClusterCommand extends OptsCommand {
+
+        protected Options options = new Options();
+        protected URI uri;
+        protected final List<String> streams = new ArrayList<String>();
+
+        protected ClusterCommand(String name, String description) {
+            super(name, description);
+            options.addOption("u", "uri", true, "DistributedLog URI");
+            options.addOption("r", "prefix", true, "Prefix of stream name. E.g. 'QuantumLeapTest-'.");
+            options.addOption("e", "expression", true, "Expression to generate stream suffix. "
+                + "Currently we support range '0-9', list '1,2,3' and name '143'");
+        }
+
+        @Override
+        protected int runCmd(CommandLine commandLine) throws Exception {
+            try {
+                parseCommandLine(commandLine);
+            } catch (ParseException pe) {
+                System.err.println("ERROR: failed to parse commandline : '" + pe.getMessage() + "'");
+                printUsage();
+                return -1;
+            }
+
+            DLZkServerSet serverSet = DLZkServerSet.of(uri, 60000);
+            logger.info("Created serverset for {}", uri);
+            try {
+                DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
+                        .name("proxy_tool")
+                        .clientId(ClientId$.MODULE$.apply("proxy_tool"))
+                        .maxRedirects(2)
+                        .serverSet(serverSet.getServerSet())
+                        .clientBuilder(ClientBuilder.get()
+                            .connectionTimeout(Duration.fromSeconds(2))
+                            .tcpConnectTimeout(Duration.fromSeconds(2))
+                            .requestTimeout(Duration.fromSeconds(10))
+                            .hostConnectionLimit(1)
+                            .hostConnectionCoresize(1)
+                            .keepAlive(true)
+                            .failFast(false))
+                        .build();
+                try {
+                    return runCmd(client);
+                } finally {
+                    client.close();
+                }
+            } finally {
+                serverSet.close();
+            }
+        }
+
+        protected abstract int runCmd(DistributedLogClient client) throws Exception;
+
+        @Override
+        protected Options getOptions() {
+            return options;
+        }
+
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            if (!cmdline.hasOption("u")) {
+                throw new ParseException("No distributedlog uri provided.");
+            }
+            this.uri = URI.create(cmdline.getOptionValue("u"));
+
+            // get stream names
+            String streamPrefix = cmdline.hasOption("r") ? cmdline.getOptionValue("r") : "";
+            String streamExpression = null;
+            if (cmdline.hasOption("e")) {
+                streamExpression = cmdline.getOptionValue("e");
+            }
+            if (null == streamPrefix || null == streamExpression) {
+                throw new ParseException("Please specify stream prefix & expression.");
+            }
+            // parse the stream expression
+            if (streamExpression.contains("-")) {
+                // a range expression
+                String[] parts = streamExpression.split("-");
+                if (parts.length != 2) {
+                    throw new ParseException("Invalid stream index range : " + streamExpression);
+                }
+                try {
+                    int start = Integer.parseInt(parts[0]);
+                    int end = Integer.parseInt(parts[1]);
+                    if (start > end) {
+                        throw new ParseException("Invalid stream index range : " + streamExpression);
+                    }
+                    for (int i = start; i <= end; i++) {
+                        streams.add(streamPrefix + i);
+                    }
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid stream index range : " + streamExpression);
+                }
+            } else if (streamExpression.contains(",")) {
+                // a list expression
+                String[] parts = streamExpression.split(",");
+                try {
+                    for (String part : parts) {
+                        streams.add(streamPrefix + part);
+                    }
+                } catch (NumberFormatException nfe) {
+                    throw new ParseException("Invalid stream suffix list : " + streamExpression);
+                }
+            } else {
+                streams.add(streamPrefix + streamExpression);
+            }
+        }
+    }
+
+    /**
+     * Command to release ownership of a log stream.
+     */
+    static class ReleaseCommand extends ClusterCommand {
+
+        double rate = 100f;
+
+        ReleaseCommand() {
+            super("release", "Release Stream Ownerships");
+            options.addOption("t", "rate", true, "Rate to release streams");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (cmdline.hasOption("t")) {
+                rate = Double.parseDouble(cmdline.getOptionValue("t", "100"));
+            }
+        }
+
+        @Override
+        protected int runCmd(DistributedLogClient client) throws Exception {
+            RateLimiter rateLimiter = RateLimiter.create(rate);
+            for (String stream : streams) {
+                rateLimiter.acquire();
+                try {
+                    Await.result(client.release(stream));
+                    System.out.println("Release ownership of stream " + stream);
+                } catch (Exception e) {
+                    System.err.println("Failed to release ownership of stream " + stream);
+                    throw e;
+                }
+            }
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "release [options]";
+        }
+    }
+
+    /**
+     * Command to truncate a log stream.
+     */
+    static class TruncateCommand extends ClusterCommand {
+
+        DLSN dlsn = DLSN.InitialDLSN;
+
+        TruncateCommand() {
+            super("truncate", "Truncate streams until given dlsn.");
+            options.addOption("d", "dlsn", true, "DLSN to truncate until");
+        }
+
+        @Override
+        protected int runCmd(DistributedLogClient client) throws Exception {
+            System.out.println("Truncating streams : " + streams);
+            for (String stream : streams) {
+                boolean success = Await.result(client.truncate(stream, dlsn));
+                System.out.println("Truncate " + stream + " to " + dlsn + " : " + success);
+            }
+            return 0;
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("d")) {
+                throw new ParseException("No DLSN provided");
+            }
+            String[] dlsnStrs = cmdline.getOptionValue("d").split(",");
+            if (dlsnStrs.length != 3) {
+                throw new ParseException("Invalid DLSN : " + cmdline.getOptionValue("d"));
+            }
+            dlsn = new DLSN(Long.parseLong(dlsnStrs[0]), Long.parseLong(dlsnStrs[1]), Long.parseLong(dlsnStrs[2]));
+        }
+
+        @Override
+        protected String getUsage() {
+            return "truncate [options]";
+        }
+    }
+
+    /**
+     * Abstract command to operate on a single proxy server.
+     */
+    protected abstract static class ProxyCommand extends OptsCommand {
+
+        protected Options options = new Options();
+        protected InetSocketAddress address;
+
+        protected ProxyCommand(String name, String description) {
+            super(name, description);
+            options.addOption("H", "host", true, "Single Proxy Address");
+        }
+
+        @Override
+        protected Options getOptions() {
+            return options;
+        }
+
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            if (!cmdline.hasOption("H")) {
+                throw new ParseException("No proxy address provided");
+            }
+            address = DLSocketAddress.parseSocketAddress(cmdline.getOptionValue("H"));
+        }
+
+        @Override
+        protected int runCmd(CommandLine commandLine) throws Exception {
+            try {
+                parseCommandLine(commandLine);
+            } catch (ParseException pe) {
+                System.err.println("ERROR: failed to parse commandline : '" + pe.getMessage() + "'");
+                printUsage();
+                return -1;
+            }
+
+            DistributedLogClientBuilder clientBuilder = DistributedLogClientBuilder.newBuilder()
+                    .name("proxy_tool")
+                    .clientId(ClientId$.MODULE$.apply("proxy_tool"))
+                    .maxRedirects(2)
+                    .host(address)
+                    .clientBuilder(ClientBuilder.get()
+                            .connectionTimeout(Duration.fromSeconds(2))
+                            .tcpConnectTimeout(Duration.fromSeconds(2))
+                            .requestTimeout(Duration.fromSeconds(10))
+                            .hostConnectionLimit(1)
+                            .hostConnectionCoresize(1)
+                            .keepAlive(true)
+                            .failFast(false));
+            Pair<DistributedLogClient, MonitorServiceClient> clientPair =
+                    ClientUtils.buildClient(clientBuilder);
+            try {
+                return runCmd(clientPair);
+            } finally {
+                clientPair.getLeft().close();
+            }
+        }
+
+        protected abstract int runCmd(Pair<DistributedLogClient, MonitorServiceClient> client) throws Exception;
+    }
+
+    /**
+     * Command to enable/disable accepting new streams.
+     */
+    static class AcceptNewStreamCommand extends ProxyCommand {
+
+        boolean enabled = false;
+
+        AcceptNewStreamCommand() {
+            super("accept-new-stream", "Enable/Disable accepting new streams for one proxy");
+            options.addOption("e", "enabled", true, "Enable/Disable accepting new streams");
+        }
+
+        @Override
+        protected void parseCommandLine(CommandLine cmdline) throws ParseException {
+            super.parseCommandLine(cmdline);
+            if (!cmdline.hasOption("e")) {
+                throw new ParseException("No action 'enable/disable' provided");
+            }
+            enabled = Boolean.parseBoolean(cmdline.getOptionValue("e"));
+        }
+
+        @Override
+        protected int runCmd(Pair<DistributedLogClient, MonitorServiceClient> client)
+                throws Exception {
+            Await.result(client.getRight().setAcceptNewStream(enabled));
+            return 0;
+        }
+
+        @Override
+        protected String getUsage() {
+            return "accept-new-stream [options]";
+        }
+    }
+
+    public ProxyTool() {
+        super();
+        addCommand(new ReleaseCommand());
+        addCommand(new TruncateCommand());
+        addCommand(new AcceptNewStreamCommand());
+    }
+
+    @Override
+    protected String getName() {
+        return "proxy_tool";
+    }
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/package-info.java
new file mode 100644
index 0000000..92d0a7d
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/tools/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Service related tools.
+ */
+package org.apache.distributedlog.service.tools;
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/ServerUtils.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/ServerUtils.java
new file mode 100644
index 0000000..9ee93b4
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/ServerUtils.java
@@ -0,0 +1,49 @@
+/**
+ * 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.service.utils;
+
+import java.io.IOException;
+import java.net.InetAddress;
+
+/**
+ * Utils that used by servers.
+ */
+public class ServerUtils {
+
+  /**
+   * Retrieve the ledger allocator pool name.
+   *
+   * @param serverRegionId region id that that server is running
+   * @param shardId shard id of the server
+   * @param useHostname whether to use hostname as the ledger allocator pool name
+   * @return ledger allocator pool name
+   * @throws IOException
+   */
+    public static String getLedgerAllocatorPoolName(int serverRegionId,
+                                                    int shardId,
+                                                    boolean useHostname)
+        throws IOException {
+        if (useHostname) {
+            return String.format("allocator_%04d_%s", serverRegionId,
+                InetAddress.getLocalHost().getHostAddress());
+        } else {
+            return String.format("allocator_%04d_%010d", serverRegionId, shardId);
+        }
+    }
+
+}
diff --git a/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/package-info.java b/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/package-info.java
new file mode 100644
index 0000000..99cf736
--- /dev/null
+++ b/distributedlog-service/src/main/java/org/apache/distributedlog/service/utils/package-info.java
@@ -0,0 +1,21 @@
+/**
+ * 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.
+ */
+/**
+ * Utilities used by proxy servers.
+ */
+package org.apache.distributedlog.service.utils;
diff --git a/distributedlog-service/src/main/resources/findbugsExclude.xml b/distributedlog-service/src/main/resources/findbugsExclude.xml
index d28ea93..e101a4d 100644
--- a/distributedlog-service/src/main/resources/findbugsExclude.xml
+++ b/distributedlog-service/src/main/resources/findbugsExclude.xml
@@ -18,21 +18,21 @@
 <FindBugsFilter>
   <Match>
     <!-- generated code, we can't be held responsible for findbugs in it //-->
-    <Class name="~com\.twitter\.distributedlog\.thrift.*" />
+    <Class name="~org\.apache\.distributedlog\.thrift.*" />
   </Match>
   <Match>
     <!-- generated code, we can't be held responsible for findbugs in it //-->
-    <Class name="~com\.twitter\.distributedlog\.service\.placement\.thrift.*" />
+    <Class name="~org\.apache\.distributedlog\.service\.placement\.thrift.*" />
   </Match>
   <Match>
     <!-- it is safe to cast exception here. //-->
-    <Class name="com.twitter.distributedlog.service.DistributedLogServiceImpl$Stream$2" />
+    <Class name="org.apache.distributedlog.service.DistributedLogServiceImpl$Stream$2" />
     <Method name="onFailure" />
     <Bug pattern="BC_UNCONFIRMED_CAST" />
   </Match>
   <Match>
     <!-- it is safe to cast exception here. //-->
-    <Class name="com.twitter.distributedlog.service.stream.BulkWriteOp" />
+    <Class name="org.apache.distributedlog.service.stream.BulkWriteOp" />
     <Method name="isDefiniteFailure" />
     <Bug pattern="BC_IMPOSSIBLE_INSTANCEOF" />
   </Match>
diff --git a/distributedlog-service/src/main/thrift/metadata.thrift b/distributedlog-service/src/main/thrift/metadata.thrift
index 8f7b6ec..9cb3c72 100644
--- a/distributedlog-service/src/main/thrift/metadata.thrift
+++ b/distributedlog-service/src/main/thrift/metadata.thrift
@@ -15,7 +15,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-namespace java com.twitter.distributedlog.service.placement.thrift
+namespace java org.apache.distributedlog.service.placement.thrift
 
 struct StreamLoad {
     1: optional string stream
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/client/routing/LocalRoutingService.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/client/routing/LocalRoutingService.java
deleted file mode 100644
index 922e901..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/client/routing/LocalRoutingService.java
+++ /dev/null
@@ -1,146 +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 com.twitter.distributedlog.client.routing;
-
-import com.google.common.collect.Sets;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.stats.StatsReceiver;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.LinkedHashSet;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.CopyOnWriteArrayList;
-
-/**
- * A local routing service that used for testing.
- */
-public class LocalRoutingService implements RoutingService {
-
-    public static Builder newBuilder() {
-        return new Builder();
-    }
-
-    /**
-     * Builder to build a local routing service for testing.
-     */
-    public static class Builder implements RoutingService.Builder {
-
-        private Builder() {}
-
-        @Override
-        public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
-            return this;
-        }
-
-        @Override
-        public LocalRoutingService build() {
-            return new LocalRoutingService();
-        }
-    }
-
-    private final Map<String, LinkedHashSet<SocketAddress>> localAddresses =
-            new HashMap<String, LinkedHashSet<SocketAddress>>();
-    private final CopyOnWriteArrayList<RoutingListener> listeners =
-            new CopyOnWriteArrayList<RoutingListener>();
-
-    boolean allowRetrySameHost = true;
-
-    @Override
-    public void startService() {
-        // nop
-    }
-
-    @Override
-    public void stopService() {
-        // nop
-    }
-
-    @Override
-    public synchronized Set<SocketAddress> getHosts() {
-        Set<SocketAddress> hosts = Sets.newHashSet();
-        for (LinkedHashSet<SocketAddress> addresses : localAddresses.values()) {
-            hosts.addAll(addresses);
-        }
-        return hosts;
-    }
-
-    @Override
-    public RoutingService registerListener(RoutingListener listener) {
-        listeners.add(listener);
-        return this;
-    }
-
-    @Override
-    public RoutingService unregisterListener(RoutingListener listener) {
-        listeners.remove(listener);
-        return this;
-    }
-
-    public LocalRoutingService setAllowRetrySameHost(boolean enabled) {
-        allowRetrySameHost = enabled;
-        return this;
-    }
-
-    public LocalRoutingService addHost(String stream, SocketAddress address) {
-        boolean notify = false;
-        synchronized (this) {
-            LinkedHashSet<SocketAddress> addresses = localAddresses.get(stream);
-            if (null == addresses) {
-                addresses = new LinkedHashSet<SocketAddress>();
-                localAddresses.put(stream, addresses);
-            }
-            if (addresses.add(address)) {
-                notify = true;
-            }
-        }
-        if (notify) {
-            for (RoutingListener listener : listeners) {
-                listener.onServerJoin(address);
-            }
-        }
-        return this;
-    }
-
-    @Override
-    public synchronized SocketAddress getHost(String key, RoutingContext rContext)
-            throws NoBrokersAvailableException {
-        LinkedHashSet<SocketAddress> addresses = localAddresses.get(key);
-
-        SocketAddress candidate = null;
-        if (null != addresses) {
-            for (SocketAddress host : addresses) {
-                if (rContext.isTriedHost(host) && !allowRetrySameHost) {
-                    continue;
-                } else {
-                    candidate = host;
-                    break;
-                }
-            }
-        }
-        if (null != candidate) {
-            return candidate;
-        }
-        throw new NoBrokersAvailableException("No host available");
-    }
-
-    @Override
-    public void removeHost(SocketAddress address, Throwable reason) {
-        // nop
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/DistributedLogServerTestCase.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/DistributedLogServerTestCase.java
deleted file mode 100644
index f7e81dc..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/DistributedLogServerTestCase.java
+++ /dev/null
@@ -1,298 +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 com.twitter.distributedlog.service;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import com.google.common.base.Optional;
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.client.DistributedLogClientImpl;
-import com.twitter.distributedlog.client.resolver.DefaultRegionResolver;
-import com.twitter.distributedlog.client.routing.LocalRoutingService;
-import com.twitter.distributedlog.client.routing.RegionsRoutingService;
-import com.twitter.distributedlog.service.DistributedLogCluster.DLServer;
-import com.twitter.distributedlog.service.stream.StreamManager;
-import com.twitter.distributedlog.service.stream.StreamManagerImpl;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Duration;
-import java.net.SocketAddress;
-import java.net.URI;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import org.junit.After;
-import org.junit.AfterClass;
-import org.junit.Before;
-import org.junit.BeforeClass;
-
-/**
- * Base test case for distributedlog servers.
- */
-public abstract class DistributedLogServerTestCase {
-
-    protected static DistributedLogConfiguration conf =
-            new DistributedLogConfiguration().setLockTimeout(10)
-                    .setOutputBufferSize(0).setPeriodicFlushFrequencyMilliSeconds(10);
-    protected static DistributedLogConfiguration noAdHocConf =
-            new DistributedLogConfiguration().setLockTimeout(10).setCreateStreamIfNotExists(false)
-                    .setOutputBufferSize(0).setPeriodicFlushFrequencyMilliSeconds(10);
-    protected static DistributedLogCluster dlCluster;
-    protected static DistributedLogCluster noAdHocCluster;
-
-    /**
-     * A distributedlog client wrapper for testing.
-     */
-    protected static class DLClient {
-        public final LocalRoutingService routingService;
-        public DistributedLogClientBuilder dlClientBuilder;
-        public final DistributedLogClientImpl dlClient;
-
-        protected DLClient(String name,
-                           String streamNameRegex,
-                           Optional<String> serverSideRoutingFinagleName) {
-            routingService = LocalRoutingService.newBuilder().build();
-            dlClientBuilder = DistributedLogClientBuilder.newBuilder()
-                        .name(name)
-                        .clientId(ClientId$.MODULE$.apply(name))
-                        .routingService(routingService)
-                        .streamNameRegex(streamNameRegex)
-                        .handshakeWithClientInfo(true)
-                        .clientBuilder(ClientBuilder.get()
-                            .hostConnectionLimit(1)
-                            .connectionTimeout(Duration.fromSeconds(1))
-                            .requestTimeout(Duration.fromSeconds(60)));
-            if (serverSideRoutingFinagleName.isPresent()) {
-                dlClientBuilder =
-                        dlClientBuilder.serverRoutingServiceFinagleNameStr(serverSideRoutingFinagleName.get());
-            }
-            dlClient = (DistributedLogClientImpl) dlClientBuilder.build();
-        }
-
-        public void handshake() {
-            dlClient.handshake();
-        }
-
-        public void shutdown() {
-            dlClient.close();
-        }
-    }
-
-    /**
-     * A distributedlog client wrapper that talks to two regions.
-     */
-    protected static class TwoRegionDLClient {
-
-        public final LocalRoutingService localRoutingService;
-        public final LocalRoutingService remoteRoutingService;
-        public final DistributedLogClientBuilder dlClientBuilder;
-        public final DistributedLogClientImpl dlClient;
-
-        protected TwoRegionDLClient(String name, Map<SocketAddress, String> regionMap) {
-            localRoutingService = new LocalRoutingService();
-            remoteRoutingService = new LocalRoutingService();
-            RegionsRoutingService regionsRoutingService =
-                    RegionsRoutingService.of(new DefaultRegionResolver(regionMap),
-                            localRoutingService, remoteRoutingService);
-            dlClientBuilder = DistributedLogClientBuilder.newBuilder()
-                        .name(name)
-                        .clientId(ClientId$.MODULE$.apply(name))
-                        .routingService(regionsRoutingService)
-                        .streamNameRegex(".*")
-                        .handshakeWithClientInfo(true)
-                        .maxRedirects(2)
-                        .clientBuilder(ClientBuilder.get()
-                            .hostConnectionLimit(1)
-                            .connectionTimeout(Duration.fromSeconds(1))
-                            .requestTimeout(Duration.fromSeconds(10)));
-            dlClient = (DistributedLogClientImpl) dlClientBuilder.build();
-        }
-
-        public void shutdown() {
-            dlClient.close();
-        }
-    }
-
-    private final boolean clientSideRouting;
-    protected DLServer dlServer;
-    protected DLClient dlClient;
-    protected DLServer noAdHocServer;
-    protected DLClient noAdHocClient;
-
-    public static DistributedLogCluster createCluster(DistributedLogConfiguration conf) throws Exception {
-        return DistributedLogCluster.newBuilder()
-            .numBookies(3)
-            .shouldStartZK(true)
-            .zkServers("127.0.0.1")
-            .shouldStartProxy(false)
-            .dlConf(conf)
-            .bkConf(DLMTestUtil.loadTestBkConf())
-            .build();
-    }
-
-    @BeforeClass
-    public static void setupCluster() throws Exception {
-        dlCluster = createCluster(conf);
-        dlCluster.start();
-    }
-
-    public void setupNoAdHocCluster() throws Exception {
-        noAdHocCluster = createCluster(noAdHocConf);
-        noAdHocCluster.start();
-        noAdHocServer = new DLServer(noAdHocConf, noAdHocCluster.getUri(), 7002, false);
-        Optional<String> serverSideRoutingFinagleName = Optional.absent();
-        if (!clientSideRouting) {
-            serverSideRoutingFinagleName =
-                    Optional.of("inet!" + DLSocketAddress.toString(noAdHocServer.getAddress()));
-        }
-        noAdHocClient = createDistributedLogClient("no-ad-hoc-client", serverSideRoutingFinagleName);
-    }
-
-    public void tearDownNoAdHocCluster() throws Exception {
-        if (null != noAdHocClient) {
-            noAdHocClient.shutdown();
-        }
-        if (null != noAdHocServer) {
-            noAdHocServer.shutdown();
-        }
-    }
-
-    @AfterClass
-    public static void teardownCluster() throws Exception {
-        if (null != dlCluster) {
-            dlCluster.stop();
-        }
-        if (null != noAdHocCluster) {
-            noAdHocCluster.stop();
-        }
-    }
-
-    protected static URI getUri() {
-        return dlCluster.getUri();
-    }
-
-    protected DistributedLogServerTestCase(boolean clientSideRouting) {
-        this.clientSideRouting = clientSideRouting;
-    }
-
-    @Before
-    public void setup() throws Exception {
-        dlServer = createDistributedLogServer(7001);
-        Optional<String> serverSideRoutingFinagleName = Optional.absent();
-        if (!clientSideRouting) {
-            serverSideRoutingFinagleName =
-                    Optional.of("inet!" + DLSocketAddress.toString(dlServer.getAddress()));
-        }
-        dlClient = createDistributedLogClient("test", serverSideRoutingFinagleName);
-    }
-
-    @After
-    public void teardown() throws Exception {
-        if (null != dlClient) {
-            dlClient.shutdown();
-        }
-        if (null != dlServer) {
-            dlServer.shutdown();
-        }
-    }
-
-    protected DLServer createDistributedLogServer(int port) throws Exception {
-        return new DLServer(conf, dlCluster.getUri(), port, false);
-    }
-
-    protected DLServer createDistributedLogServer(DistributedLogConfiguration conf, int port)
-            throws Exception {
-        return new DLServer(conf, dlCluster.getUri(), port, false);
-    }
-
-    protected DLClient createDistributedLogClient(String clientName,
-                                                  Optional<String> serverSideRoutingFinagleName)
-            throws Exception {
-        return createDistributedLogClient(clientName, ".*", serverSideRoutingFinagleName);
-    }
-
-    protected DLClient createDistributedLogClient(String clientName,
-                                                  String streamNameRegex,
-                                                  Optional<String> serverSideRoutingFinagleName)
-            throws Exception {
-        return new DLClient(clientName, streamNameRegex, serverSideRoutingFinagleName);
-    }
-
-    protected TwoRegionDLClient createTwoRegionDLClient(String clientName,
-                                                        Map<SocketAddress, String> regionMap)
-            throws Exception {
-        return new TwoRegionDLClient(clientName, regionMap);
-    }
-
-    protected static void checkStreams(int numExpectedStreams, DLServer dlServer) {
-        StreamManager streamManager = dlServer.dlServer.getKey().getStreamManager();
-        assertEquals(numExpectedStreams, streamManager.numCached());
-        assertEquals(numExpectedStreams, streamManager.numAcquired());
-    }
-
-    protected static void checkStreams(Set<String> streams, DLServer dlServer) {
-        StreamManagerImpl streamManager = (StreamManagerImpl) dlServer.dlServer.getKey().getStreamManager();
-        Set<String> cachedStreams = streamManager.getCachedStreams().keySet();
-        Set<String> acquiredStreams = streamManager.getAcquiredStreams().keySet();
-
-        assertEquals(streams.size(), cachedStreams.size());
-        assertEquals(streams.size(), acquiredStreams.size());
-        assertTrue(Sets.difference(streams, cachedStreams).isEmpty());
-        assertTrue(Sets.difference(streams, acquiredStreams).isEmpty());
-    }
-
-    protected static void checkStream(String name, DLClient dlClient, DLServer dlServer,
-                                      int expectedNumProxiesInClient, int expectedClientCacheSize,
-                                      int expectedServerCacheSize, boolean existedInServer, boolean existedInClient) {
-        Map<SocketAddress, Set<String>> distribution = dlClient.dlClient.getStreamOwnershipDistribution();
-        assertEquals(expectedNumProxiesInClient, distribution.size());
-
-        if (expectedNumProxiesInClient > 0) {
-            Map.Entry<SocketAddress, Set<String>> localEntry =
-                    distribution.entrySet().iterator().next();
-            assertEquals(dlServer.getAddress(), localEntry.getKey());
-            assertEquals(expectedClientCacheSize, localEntry.getValue().size());
-            assertEquals(existedInClient, localEntry.getValue().contains(name));
-        }
-
-        StreamManagerImpl streamManager = (StreamManagerImpl) dlServer.dlServer.getKey().getStreamManager();
-        Set<String> cachedStreams = streamManager.getCachedStreams().keySet();
-        Set<String> acquiredStreams = streamManager.getCachedStreams().keySet();
-
-        assertEquals(expectedServerCacheSize, cachedStreams.size());
-        assertEquals(existedInServer, cachedStreams.contains(name));
-        assertEquals(expectedServerCacheSize, acquiredStreams.size());
-        assertEquals(existedInServer, acquiredStreams.contains(name));
-    }
-
-    protected static Map<SocketAddress, Set<String>> getStreamOwnershipDistribution(DLClient dlClient) {
-        return dlClient.dlClient.getStreamOwnershipDistribution();
-    }
-
-    protected static Set<String> getAllStreamsFromDistribution(Map<SocketAddress, Set<String>> distribution) {
-        Set<String> allStreams = new HashSet<String>();
-        for (Map.Entry<SocketAddress, Set<String>> entry : distribution.entrySet()) {
-            allStreams.addAll(entry.getValue());
-        }
-        return allStreams;
-    }
-
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerBase.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerBase.java
deleted file mode 100644
index 24d7f07..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerBase.java
+++ /dev/null
@@ -1,720 +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 com.twitter.distributedlog.service;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static com.twitter.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.AsyncLogReader;
-import com.twitter.distributedlog.DLMTestUtil;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.LogReader;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.TestZooKeeperClientBuilder;
-import com.twitter.distributedlog.ZooKeeperClient;
-import com.twitter.distributedlog.acl.AccessControlManager;
-import com.twitter.distributedlog.annotations.DistributedLogAnnotations;
-import com.twitter.distributedlog.client.routing.LocalRoutingService;
-import com.twitter.distributedlog.exceptions.DLException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.impl.acl.ZKAccessControl;
-import com.twitter.distributedlog.impl.metadata.BKDLConfig;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.service.stream.StreamManagerImpl;
-import com.twitter.distributedlog.thrift.AccessControlEntry;
-import com.twitter.distributedlog.thrift.service.BulkWriteResponse;
-import com.twitter.distributedlog.thrift.service.HeartbeatOptions;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.distributedlog.thrift.service.WriteContext;
-import com.twitter.distributedlog.util.FailpointUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.finagle.builder.ClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import com.twitter.util.Futures;
-import java.net.SocketAddress;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import org.junit.Ignore;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test Case for {@link DistributedLogServer}.
- */
-public abstract class TestDistributedLogServerBase extends DistributedLogServerTestCase {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestDistributedLogServerBase.class);
-
-    @Rule
-    public TestName testName = new TestName();
-
-    protected TestDistributedLogServerBase(boolean clientSideRouting) {
-        super(clientSideRouting);
-    }
-
-    /**
-     * {@link https://issues.apache.org/jira/browse/DL-27}.
-     */
-    @DistributedLogAnnotations.FlakyTest
-    @Ignore
-    @Test(timeout = 60000)
-    public void testBasicWrite() throws Exception {
-        String name = "dlserver-basic-write";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        for (long i = 1; i <= 10; i++) {
-            logger.debug("Write entry {} to stream {}.", i, name);
-            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())));
-        }
-
-        HeartbeatOptions hbOptions = new HeartbeatOptions();
-        hbOptions.setSendHeartBeatToReader(true);
-        // make sure the first log segment of each stream created
-        FutureUtils.result(dlClient.dlClient.heartbeat(name));
-
-        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
-        LogReader reader = dlm.getInputStream(1);
-        int numRead = 0;
-        LogRecord r = reader.readNext(false);
-        while (null != r) {
-            ++numRead;
-            int i = Integer.parseInt(new String(r.getPayload()));
-            assertEquals(numRead, i);
-            r = reader.readNext(false);
-        }
-        assertEquals(10, numRead);
-        reader.close();
-        dlm.close();
-    }
-
-    /**
-     * Sanity check to make sure both checksum flag values work.
-     */
-    @Test(timeout = 60000)
-    public void testChecksumFlag() throws Exception {
-        String name = "testChecksumFlag";
-        LocalRoutingService routingService = LocalRoutingService.newBuilder().build();
-        routingService.addHost(name, dlServer.getAddress());
-        DistributedLogClientBuilder dlClientBuilder = DistributedLogClientBuilder.newBuilder()
-            .name(name)
-            .clientId(ClientId$.MODULE$.apply("test"))
-            .routingService(routingService)
-            .handshakeWithClientInfo(true)
-            .clientBuilder(ClientBuilder.get()
-                .hostConnectionLimit(1)
-                .connectionTimeout(Duration.fromSeconds(1))
-                .requestTimeout(Duration.fromSeconds(60)))
-            .checksum(false);
-        DistributedLogClient dlClient = dlClientBuilder.build();
-        Await.result(dlClient.write(name, ByteBuffer.wrap(("1").getBytes())));
-        dlClient.close();
-
-        dlClient = dlClientBuilder.checksum(true).build();
-        Await.result(dlClient.write(name, ByteBuffer.wrap(("2").getBytes())));
-        dlClient.close();
-    }
-
-    private void runSimpleBulkWriteTest(int writeCount) throws Exception {
-        String name = String.format("dlserver-bulk-write-%d", writeCount);
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount);
-        for (long i = 1; i <= writeCount; i++) {
-            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
-        }
-
-        logger.debug("Write {} entries to stream {}.", writeCount, name);
-        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
-        assertEquals(futures.size(), writeCount);
-        for (Future<DLSN> future : futures) {
-            // No throw == pass.
-            DLSN dlsn = Await.result(future, Duration.fromSeconds(10));
-        }
-
-        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
-        LogReader reader = dlm.getInputStream(1);
-        int numRead = 0;
-        LogRecord r = reader.readNext(false);
-        while (null != r) {
-            int i = Integer.parseInt(new String(r.getPayload()));
-            assertEquals(numRead + 1, i);
-            ++numRead;
-            r = reader.readNext(false);
-        }
-        assertEquals(writeCount, numRead);
-        reader.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWrite() throws Exception {
-        runSimpleBulkWriteTest(100);
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWriteSingleWrite() throws Exception {
-        runSimpleBulkWriteTest(1);
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWriteEmptyList() throws Exception {
-        String name = String.format("dlserver-bulk-write-%d", 0);
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>();
-        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
-
-        assertEquals(0, futures.size());
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWriteNullArg() throws Exception {
-
-        String name = String.format("dlserver-bulk-write-%s", "null");
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>();
-        writes.add(null);
-
-        try {
-            dlClient.dlClient.writeBulk(name, writes);
-            fail("should not have succeeded");
-        } catch (NullPointerException npe) {
-            // expected
-            logger.info("Expected to catch NullPointException.");
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWriteEmptyBuffer() throws Exception {
-        String name = String.format("dlserver-bulk-write-%s", "empty");
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>();
-        writes.add(ByteBuffer.wrap(("").getBytes()));
-        writes.add(ByteBuffer.wrap(("").getBytes()));
-        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
-        assertEquals(2, futures.size());
-        for (Future<DLSN> future : futures) {
-            // No throw == pass
-            DLSN dlsn = Await.result(future, Duration.fromSeconds(10));
-        }
-    }
-
-    void failDueToWrongException(Exception ex) {
-        logger.info("testBulkWritePartialFailure: ", ex);
-        fail(String.format("failed with wrong exception %s", ex.getClass().getName()));
-    }
-
-    int validateAllFailedAsCancelled(List<Future<DLSN>> futures, int start, int finish) {
-        int failed = 0;
-        for (int i = start; i < finish; i++) {
-            Future<DLSN> future = futures.get(i);
-            try {
-                Await.result(future, Duration.fromSeconds(10));
-                fail("future should have failed!");
-            } catch (DLException cre) {
-                ++failed;
-            } catch (Exception ex) {
-                failDueToWrongException(ex);
-            }
-        }
-        return failed;
-    }
-
-    void validateFailedAsLogRecordTooLong(Future<DLSN> future) {
-        try {
-            Await.result(future, Duration.fromSeconds(10));
-            fail("should have failed");
-        } catch (DLException dle) {
-            assertEquals(StatusCode.TOO_LARGE_RECORD, dle.getCode());
-        } catch (Exception ex) {
-            failDueToWrongException(ex);
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWritePartialFailure() throws Exception {
-        String name = String.format("dlserver-bulk-write-%s", "partial-failure");
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        final int writeCount = 100;
-
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount * 2 + 1);
-        for (long i = 1; i <= writeCount; i++) {
-            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
-        }
-        // Too big, will cause partial failure.
-        ByteBuffer buf = ByteBuffer.allocate(MAX_LOGRECORD_SIZE + 1);
-        writes.add(buf);
-        for (long i = 1; i <= writeCount; i++) {
-            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
-        }
-
-        // Count succeeded.
-        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
-        int succeeded = 0;
-        for (int i = 0; i < writeCount; i++) {
-            Future<DLSN> future = futures.get(i);
-            try {
-                Await.result(future, Duration.fromSeconds(10));
-                ++succeeded;
-            } catch (Exception ex) {
-                failDueToWrongException(ex);
-            }
-        }
-
-        validateFailedAsLogRecordTooLong(futures.get(writeCount));
-        FutureUtils.result(Futures.collect(futures.subList(writeCount + 1, 2 * writeCount + 1)));
-        assertEquals(writeCount, succeeded);
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWriteTotalFailureFirstWriteFailed() throws Exception {
-        String name = String.format("dlserver-bulk-write-%s", "first-write-failed");
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        final int writeCount = 100;
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount + 1);
-        ByteBuffer buf = ByteBuffer.allocate(MAX_LOGRECORD_SIZE + 1);
-        writes.add(buf);
-        for (long i = 1; i <= writeCount; i++) {
-            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
-        }
-
-        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
-        validateFailedAsLogRecordTooLong(futures.get(0));
-        FutureUtils.result(Futures.collect(futures.subList(1, writeCount + 1)));
-    }
-
-    @Test(timeout = 60000)
-    public void testBulkWriteTotalFailureLostLock() throws Exception {
-        String name = String.format("dlserver-bulk-write-%s", "lost-lock");
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        final int writeCount = 8;
-        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount + 1);
-        ByteBuffer buf = ByteBuffer.allocate(8);
-        writes.add(buf);
-        for (long i = 1; i <= writeCount; i++) {
-            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
-        }
-        // Warm it up with a write.
-        Await.result(dlClient.dlClient.write(name, ByteBuffer.allocate(8)));
-
-        // Failpoint a lost lock, make sure the failure gets promoted to an operation failure.
-        DistributedLogServiceImpl svcImpl = (DistributedLogServiceImpl) dlServer.dlServer.getLeft();
-        try {
-            FailpointUtils.setFailpoint(
-                FailpointUtils.FailPointName.FP_WriteInternalLostLock,
-                FailpointUtils.FailPointActions.FailPointAction_Default
-            );
-            Future<BulkWriteResponse> futures = svcImpl.writeBulkWithContext(name, writes, new WriteContext());
-            assertEquals(StatusCode.LOCKING_EXCEPTION, Await.result(futures).header.code);
-        } finally {
-            FailpointUtils.removeFailpoint(
-                FailpointUtils.FailPointName.FP_WriteInternalLostLock
-            );
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testHeartbeat() throws Exception {
-        String name = "dlserver-heartbeat";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        for (long i = 1; i <= 10; i++) {
-            logger.debug("Send heartbeat {} to stream {}.", i, name);
-            dlClient.dlClient.check(name).get();
-        }
-
-        logger.debug("Write entry one to stream {}.", name);
-        dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes())).get();
-
-        Thread.sleep(1000);
-
-        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
-        LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
-        int numRead = 0;
-        // eid=0 => control records
-        // other 9 heartbeats will not trigger writing any control records.
-        // eid=1 => user entry
-        long startEntryId = 1;
-        LogRecordWithDLSN r = reader.readNext(false);
-        while (null != r) {
-            int i = Integer.parseInt(new String(r.getPayload()));
-            assertEquals(numRead + 1, i);
-            assertEquals(r.getDlsn().compareTo(new DLSN(1, startEntryId, 0)), 0);
-            ++numRead;
-            ++startEntryId;
-            r = reader.readNext(false);
-        }
-        assertEquals(1, numRead);
-    }
-
-    @Test(timeout = 60000)
-    public void testFenceWrite() throws Exception {
-        String name = "dlserver-fence-write";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        for (long i = 1; i <= 10; i++) {
-            logger.debug("Write entry {} to stream {}.", i, name);
-            dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())).get();
-        }
-
-        Thread.sleep(1000);
-
-        logger.info("Fencing stream {}.", name);
-        DLMTestUtil.fenceStream(conf, getUri(), name);
-        logger.info("Fenced stream {}.", name);
-
-        for (long i = 11; i <= 20; i++) {
-            logger.debug("Write entry {} to stream {}.", i, name);
-            dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())).get();
-        }
-
-        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
-        LogReader reader = dlm.getInputStream(1);
-        int numRead = 0;
-        LogRecord r = reader.readNext(false);
-        while (null != r) {
-            int i = Integer.parseInt(new String(r.getPayload()));
-            assertEquals(numRead + 1, i);
-            ++numRead;
-            r = reader.readNext(false);
-        }
-        assertEquals(20, numRead);
-        reader.close();
-        dlm.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testDeleteStream() throws Exception {
-        String name = "dlserver-delete-stream";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        long txid = 101;
-        for (long i = 1; i <= 10; i++) {
-            long curTxId = txid++;
-            logger.debug("Write entry {} to stream {}.", curTxId, name);
-            dlClient.dlClient.write(name,
-                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
-        }
-
-        checkStream(1, 1, 1, name, dlServer.getAddress(), true, true);
-
-        dlClient.dlClient.delete(name).get();
-
-        checkStream(0, 0, 0, name, dlServer.getAddress(), false, false);
-
-        Thread.sleep(1000);
-
-        DistributedLogManager dlm101 = DLMTestUtil.createNewDLM(name, conf, getUri());
-        AsyncLogReader reader101 = FutureUtils.result(dlm101.openAsyncLogReader(DLSN.InitialDLSN));
-        try {
-            FutureUtils.result(reader101.readNext());
-            fail("Should fail with LogNotFoundException since the stream is deleted");
-        } catch (LogNotFoundException lnfe) {
-            // expected
-        }
-        FutureUtils.result(reader101.asyncClose());
-        dlm101.close();
-
-        txid = 201;
-        for (long i = 1; i <= 10; i++) {
-            long curTxId = txid++;
-            logger.debug("Write entry {} to stream {}.", curTxId, name);
-            DLSN dlsn = dlClient.dlClient.write(name,
-                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
-        }
-        Thread.sleep(1000);
-
-        DistributedLogManager dlm201 = DLMTestUtil.createNewDLM(name, conf, getUri());
-        LogReader reader201 = dlm201.getInputStream(1);
-        int numRead = 0;
-        int curTxId = 201;
-        LogRecord r = reader201.readNext(false);
-        while (null != r) {
-            int i = Integer.parseInt(new String(r.getPayload()));
-            assertEquals(curTxId++, i);
-            ++numRead;
-            r = reader201.readNext(false);
-        }
-        assertEquals(10, numRead);
-        reader201.close();
-        dlm201.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateStream() throws Exception {
-        try {
-            setupNoAdHocCluster();
-            final String name = "dlserver-create-stream";
-
-            noAdHocClient.routingService.addHost("dlserver-create-stream", noAdHocServer.getAddress());
-            assertFalse(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
-            assertTrue(Await.ready(noAdHocClient.dlClient.create(name)).isReturn());
-
-            long txid = 101;
-            for (long i = 1; i <= 10; i++) {
-                long curTxId = txid++;
-                logger.debug("Write entry {} to stream {}.", curTxId, name);
-                noAdHocClient.dlClient.write(name,
-                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
-            }
-
-            assertTrue(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
-        } finally {
-            tearDownNoAdHocCluster();
-        }
-    }
-
-    /**
-     * This tests that create has touch like behavior in that trying to create the stream twice, simply does nothing.
-     */
-    @Test(timeout = 60000)
-    public void testCreateStreamTwice() throws Exception {
-        try {
-            setupNoAdHocCluster();
-            final String name = "dlserver-create-stream-twice";
-
-            noAdHocClient.routingService.addHost("dlserver-create-stream-twice", noAdHocServer.getAddress());
-            assertFalse(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
-            assertTrue(Await.ready(noAdHocClient.dlClient.create(name)).isReturn());
-
-            long txid = 101;
-            for (long i = 1; i <= 10; i++) {
-                long curTxId = txid++;
-                logger.debug("Write entry {} to stream {}.", curTxId, name);
-                noAdHocClient.dlClient.write(name,
-                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
-            }
-
-            assertTrue(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
-
-            // create again
-            assertTrue(Await.ready(noAdHocClient.dlClient.create(name)).isReturn());
-            assertTrue(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
-        } finally {
-            tearDownNoAdHocCluster();
-        }
-    }
-
-
-
-    @Test(timeout = 60000)
-    public void testTruncateStream() throws Exception {
-        String name = "dlserver-truncate-stream";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        long txid = 1;
-        Map<Long, DLSN> txid2DLSN = new HashMap<Long, DLSN>();
-        for (int s = 1; s <= 2; s++) {
-            for (long i = 1; i <= 10; i++) {
-                long curTxId = txid++;
-                logger.debug("Write entry {} to stream {}.", curTxId, name);
-                DLSN dlsn = dlClient.dlClient.write(name,
-                        ByteBuffer.wrap(("" + curTxId).getBytes())).get();
-                txid2DLSN.put(curTxId, dlsn);
-            }
-            if (s == 1) {
-                dlClient.dlClient.release(name).get();
-            }
-        }
-
-        DLSN dlsnToDelete = txid2DLSN.get(11L);
-        dlClient.dlClient.truncate(name, dlsnToDelete).get();
-
-        DistributedLogManager readDLM = DLMTestUtil.createNewDLM(name, conf, getUri());
-        LogReader reader = readDLM.getInputStream(1);
-        int numRead = 0;
-        int curTxId = 11;
-        LogRecord r = reader.readNext(false);
-        while (null != r) {
-            int i = Integer.parseInt(new String(r.getPayload()));
-            assertEquals(curTxId++, i);
-            ++numRead;
-            r = reader.readNext(false);
-        }
-        assertEquals(10, numRead);
-        reader.close();
-        readDLM.close();
-    }
-
-    @Test(timeout = 60000)
-    public void testRequestDenied() throws Exception {
-        String name = "request-denied";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        AccessControlEntry ace = new AccessControlEntry();
-        ace.setDenyWrite(true);
-        ZooKeeperClient zkc = TestZooKeeperClientBuilder
-                .newBuilder()
-                .uri(getUri())
-                .connectionTimeoutMs(60000)
-                .sessionTimeoutMs(60000)
-                .build();
-        DistributedLogNamespace dlNamespace = dlServer.dlServer.getLeft().getDistributedLogNamespace();
-        BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, getUri());
-        String zkPath = getUri().getPath() + "/" + bkdlConfig.getACLRootPath() + "/" + name;
-        ZKAccessControl accessControl = new ZKAccessControl(ace, zkPath);
-        accessControl.create(zkc);
-
-        AccessControlManager acm = dlNamespace.createAccessControlManager();
-        while (acm.allowWrite(name)) {
-            Thread.sleep(100);
-        }
-
-        try {
-            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
-            fail("Should fail with request denied exception");
-        } catch (DLException dle) {
-            assertEquals(StatusCode.REQUEST_DENIED, dle.getCode());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testNoneStreamNameRegex() throws Exception {
-        String streamNamePrefix = "none-stream-name-regex-";
-        int numStreams = 5;
-        Set<String> streams = new HashSet<String>();
-
-        for (int i = 0; i < numStreams; i++) {
-            streams.add(streamNamePrefix + i);
-        }
-        testStreamNameRegex(streams, ".*", streams);
-    }
-
-    @Test(timeout = 60000)
-    public void testStreamNameRegex() throws Exception {
-        String streamNamePrefix = "stream-name-regex-";
-        int numStreams = 5;
-        Set<String> streams = new HashSet<String>();
-        Set<String> expectedStreams = new HashSet<String>();
-        String streamNameRegex = streamNamePrefix + "1";
-
-        for (int i = 0; i < numStreams; i++) {
-            streams.add(streamNamePrefix + i);
-        }
-        expectedStreams.add(streamNamePrefix + "1");
-
-        testStreamNameRegex(streams, streamNameRegex, expectedStreams);
-    }
-
-    private void testStreamNameRegex(Set<String> streams, String streamNameRegex,
-                                     Set<String> expectedStreams)
-            throws Exception {
-        for (String streamName : streams) {
-            dlClient.routingService.addHost(streamName, dlServer.getAddress());
-            Await.result(dlClient.dlClient.write(streamName,
-                    ByteBuffer.wrap(streamName.getBytes(UTF_8))));
-        }
-
-        DLClient client = createDistributedLogClient(
-                "test-stream-name-regex",
-                streamNameRegex,
-                Optional.<String>absent());
-        try {
-            client.routingService.addHost("unknown", dlServer.getAddress());
-            client.handshake();
-            Map<SocketAddress, Set<String>> distribution =
-                    client.dlClient.getStreamOwnershipDistribution();
-            assertEquals(1, distribution.size());
-            Set<String> cachedStreams = distribution.values().iterator().next();
-            assertNotNull(cachedStreams);
-            assertEquals(expectedStreams.size(), cachedStreams.size());
-
-            for (String streamName : cachedStreams) {
-                assertTrue(expectedStreams.contains(streamName));
-            }
-        } finally {
-            client.shutdown();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testReleaseStream() throws Exception {
-        String name = "dlserver-release-stream";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-
-        Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
-        checkStream(1, 1, 1, name, dlServer.getAddress(), true, true);
-
-        // release the stream
-        Await.result(dlClient.dlClient.release(name));
-        checkStream(0, 0, 0, name, dlServer.getAddress(), false, false);
-    }
-
-    protected void checkStream(int expectedNumProxiesInClient, int expectedClientCacheSize, int expectedServerCacheSize,
-                             String name, SocketAddress owner, boolean existedInServer, boolean existedInClient) {
-        Map<SocketAddress, Set<String>> distribution = dlClient.dlClient.getStreamOwnershipDistribution();
-        assertEquals(expectedNumProxiesInClient, distribution.size());
-
-        if (expectedNumProxiesInClient > 0) {
-            Map.Entry<SocketAddress, Set<String>> localEntry =
-                    distribution.entrySet().iterator().next();
-            assertEquals(owner, localEntry.getKey());
-            assertEquals(expectedClientCacheSize, localEntry.getValue().size());
-            assertEquals(existedInClient, localEntry.getValue().contains(name));
-        }
-
-
-        StreamManagerImpl streamManager = (StreamManagerImpl) dlServer.dlServer.getKey().getStreamManager();
-        Set<String> cachedStreams = streamManager.getCachedStreams().keySet();
-        Set<String> acquiredStreams = streamManager.getCachedStreams().keySet();
-
-        assertEquals(expectedServerCacheSize, cachedStreams.size());
-        assertEquals(existedInServer, cachedStreams.contains(name));
-        assertEquals(expectedServerCacheSize, acquiredStreams.size());
-        assertEquals(existedInServer, acquiredStreams.contains(name));
-    }
-
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerClientRouting.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerClientRouting.java
deleted file mode 100644
index b776543..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerClientRouting.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 com.twitter.distributedlog.service;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.fail;
-
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.util.Await;
-import java.nio.ByteBuffer;
-import org.junit.Test;
-
-/**
- * Test the server with client side routing.
- */
-public class TestDistributedLogServerClientRouting extends TestDistributedLogServerBase {
-
-    public TestDistributedLogServerClientRouting() {
-        super(true);
-    }
-
-    @Test(timeout = 60000)
-    public void testAcceptNewStream() throws Exception {
-        String name = "dlserver-accept-new-stream";
-
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-        dlClient.routingService.setAllowRetrySameHost(false);
-
-        Await.result(dlClient.dlClient.setAcceptNewStream(false));
-
-        try {
-            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
-            fail("Should fail because the proxy couldn't accept new stream");
-        } catch (NoBrokersAvailableException nbae) {
-            // expected
-        }
-        checkStream(0, 0, 0, name, dlServer.getAddress(), false, false);
-
-        Await.result(dlServer.dlServer.getLeft().setAcceptNewStream(true));
-        Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
-        checkStream(1, 1, 1, name, dlServer.getAddress(), true, true);
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerServerRouting.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerServerRouting.java
deleted file mode 100644
index f3dfd11..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogServerServerRouting.java
+++ /dev/null
@@ -1,28 +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 com.twitter.distributedlog.service;
-
-/**
- * Test the server with client side routing.
- */
-public class TestDistributedLogServerServerRouting extends TestDistributedLogServerBase {
-
-    public TestDistributedLogServerServerRouting() {
-        super(false);
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogService.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogService.java
deleted file mode 100644
index d9d2b21..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestDistributedLogService.java
+++ /dev/null
@@ -1,833 +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 com.twitter.distributedlog.service;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.TestDistributedLogBase;
-import com.twitter.distributedlog.acl.DefaultAccessControlManager;
-import com.twitter.distributedlog.client.routing.LocalRoutingService;
-import com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException;
-import com.twitter.distributedlog.exceptions.StreamUnavailableException;
-import com.twitter.distributedlog.service.config.NullStreamConfigProvider;
-import com.twitter.distributedlog.service.config.ServerConfiguration;
-import com.twitter.distributedlog.service.placement.EqualLoadAppraiser;
-import com.twitter.distributedlog.service.stream.Stream;
-import com.twitter.distributedlog.service.stream.StreamImpl;
-import com.twitter.distributedlog.service.stream.StreamImpl.StreamStatus;
-import com.twitter.distributedlog.service.stream.StreamManagerImpl;
-import com.twitter.distributedlog.service.stream.WriteOp;
-import com.twitter.distributedlog.service.streamset.DelimiterStreamPartitionConverter;
-import com.twitter.distributedlog.service.streamset.IdentityStreamPartitionConverter;
-import com.twitter.distributedlog.service.streamset.StreamPartitionConverter;
-import com.twitter.distributedlog.thrift.service.HeartbeatOptions;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.distributedlog.thrift.service.WriteContext;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.util.ConfUtils;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.distributedlog.util.ProtocolUtils;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.util.ReflectionUtils;
-import org.apache.commons.configuration.ConfigurationException;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test Case for DistributedLog Service.
- */
-public class TestDistributedLogService extends TestDistributedLogBase {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestDistributedLogService.class);
-
-    @Rule
-    public TestName testName = new TestName();
-
-    private ServerConfiguration serverConf;
-    private DistributedLogConfiguration dlConf;
-    private URI uri;
-    private final CountDownLatch latch = new CountDownLatch(1);
-    private DistributedLogServiceImpl service;
-
-    @Before
-    @Override
-    public void setup() throws Exception {
-        super.setup();
-        dlConf = new DistributedLogConfiguration();
-        dlConf.addConfiguration(conf);
-        dlConf.setLockTimeout(0)
-                .setOutputBufferSize(0)
-                .setPeriodicFlushFrequencyMilliSeconds(10)
-                .setSchedulerShutdownTimeoutMs(100);
-        serverConf = newLocalServerConf();
-        uri = createDLMURI("/" + testName.getMethodName());
-        ensureURICreated(uri);
-        service = createService(serverConf, dlConf, latch);
-    }
-
-    @After
-    @Override
-    public void teardown() throws Exception {
-        if (null != service) {
-            service.shutdown();
-        }
-        super.teardown();
-    }
-
-    private DistributedLogConfiguration newLocalConf() {
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(dlConf);
-        return confLocal;
-    }
-
-    private ServerConfiguration newLocalServerConf() {
-        ServerConfiguration serverConf = new ServerConfiguration();
-        serverConf.loadConf(dlConf);
-        serverConf.setServerThreads(1);
-        return serverConf;
-    }
-
-    private DistributedLogServiceImpl createService(
-            ServerConfiguration serverConf,
-            DistributedLogConfiguration dlConf) throws Exception {
-        return createService(serverConf, dlConf, new CountDownLatch(1));
-    }
-
-    private DistributedLogServiceImpl createService(
-            ServerConfiguration serverConf,
-            DistributedLogConfiguration dlConf,
-            CountDownLatch latch) throws Exception {
-        // Build the stream partition converter
-        StreamPartitionConverter converter;
-        try {
-            converter = ReflectionUtils.newInstance(serverConf.getStreamPartitionConverterClass());
-        } catch (ConfigurationException e) {
-            logger.warn("Failed to load configured stream-to-partition converter. Fallback to use {}",
-                    IdentityStreamPartitionConverter.class.getName());
-            converter = new IdentityStreamPartitionConverter();
-        }
-        return new DistributedLogServiceImpl(
-            serverConf,
-            dlConf,
-            ConfUtils.getConstDynConf(dlConf),
-            new NullStreamConfigProvider(),
-            uri,
-            converter,
-            new LocalRoutingService(),
-            NullStatsLogger.INSTANCE,
-            NullStatsLogger.INSTANCE,
-            latch,
-            new EqualLoadAppraiser());
-    }
-
-    private StreamImpl createUnstartedStream(DistributedLogServiceImpl service,
-                                             String name) throws Exception {
-        StreamImpl stream = (StreamImpl) service.newStream(name);
-        stream.initialize();
-        return stream;
-    }
-
-    private ByteBuffer createRecord(long txid) {
-        return ByteBuffer.wrap(("record-" + txid).getBytes(UTF_8));
-    }
-
-    private WriteOp createWriteOp(DistributedLogServiceImpl service,
-                                  String streamName,
-                                  long txid) {
-        ByteBuffer data = createRecord(txid);
-        return service.newWriteOp(streamName, data, null);
-    }
-
-    @Test(timeout = 60000)
-    public void testAcquireStreams() throws Exception {
-        String streamName = testName.getMethodName();
-        StreamImpl s0 = createUnstartedStream(service, streamName);
-        ServerConfiguration serverConf1 = new ServerConfiguration();
-        serverConf1.addConfiguration(serverConf);
-        serverConf1.setServerPort(9999);
-        DistributedLogServiceImpl service1 = createService(serverConf1, dlConf);
-        StreamImpl s1 = createUnstartedStream(service1, streamName);
-
-        // create write ops
-        WriteOp op0 = createWriteOp(service, streamName, 0L);
-        s0.submit(op0);
-
-        WriteOp op1 = createWriteOp(service1, streamName, 1L);
-        s1.submit(op1);
-
-        // check pending size
-        assertEquals("Write Op 0 should be pending in service 0",
-                1, s0.numPendingOps());
-        assertEquals("Write Op 1 should be pending in service 1",
-                1, s1.numPendingOps());
-
-        // start acquiring s0
-        s0.start();
-        WriteResponse wr0 = Await.result(op0.result());
-        assertEquals("Op 0 should succeed",
-                StatusCode.SUCCESS, wr0.getHeader().getCode());
-        assertEquals("Service 0 should acquire stream",
-                StreamStatus.INITIALIZED, s0.getStatus());
-        assertNotNull(s0.getManager());
-        assertNotNull(s0.getWriter());
-        assertNull(s0.getLastException());
-
-        // start acquiring s1
-        s1.start();
-        WriteResponse wr1 = Await.result(op1.result());
-        assertEquals("Op 1 should fail",
-                StatusCode.FOUND, wr1.getHeader().getCode());
-        // the stream will be set to ERROR and then be closed.
-        assertTrue("Service 1 should be in unavailable state",
-                StreamStatus.isUnavailable(s1.getStatus()));
-        assertNotNull(s1.getManager());
-        assertNull(s1.getWriter());
-        assertNotNull(s1.getLastException());
-        assertTrue(s1.getLastException() instanceof OwnershipAcquireFailedException);
-
-        service1.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testAcquireStreamsWhenExceedMaxCachedPartitions() throws Exception {
-        String streamName = testName.getMethodName() + "_0000";
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(dlConf);
-        confLocal.setMaxCachedPartitionsPerProxy(1);
-
-        ServerConfiguration serverConfLocal = new ServerConfiguration();
-        serverConfLocal.addConfiguration(serverConf);
-        serverConfLocal.setStreamPartitionConverterClass(DelimiterStreamPartitionConverter.class);
-
-        DistributedLogServiceImpl serviceLocal = createService(serverConfLocal, confLocal);
-        Stream stream = serviceLocal.getLogWriter(streamName);
-
-        // stream is cached
-        assertNotNull(stream);
-        assertEquals(1, serviceLocal.getStreamManager().numCached());
-
-        // create write ops
-        WriteOp op0 = createWriteOp(service, streamName, 0L);
-        stream.submit(op0);
-        WriteResponse wr0 = Await.result(op0.result());
-        assertEquals("Op 0 should succeed",
-                StatusCode.SUCCESS, wr0.getHeader().getCode());
-        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
-
-        // should fail to acquire another partition
-        try {
-            serviceLocal.getLogWriter(testName.getMethodName() + "_0001");
-            fail("Should fail to acquire new streams");
-        } catch (StreamUnavailableException sue) {
-            // expected
-        }
-        assertEquals(1, serviceLocal.getStreamManager().numCached());
-        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
-
-        // should be able to acquire partitions from other streams
-        String anotherStreamName = testName.getMethodName() + "-another_0001";
-        Stream anotherStream = serviceLocal.getLogWriter(anotherStreamName);
-        assertNotNull(anotherStream);
-        assertEquals(2, serviceLocal.getStreamManager().numCached());
-
-        // create write ops
-        WriteOp op1 = createWriteOp(service, anotherStreamName, 0L);
-        anotherStream.submit(op1);
-        WriteResponse wr1 = Await.result(op1.result());
-        assertEquals("Op 1 should succeed",
-                StatusCode.SUCCESS, wr1.getHeader().getCode());
-        assertEquals(2, serviceLocal.getStreamManager().numAcquired());
-    }
-
-    @Test(timeout = 60000)
-    public void testAcquireStreamsWhenExceedMaxAcquiredPartitions() throws Exception {
-        String streamName = testName.getMethodName() + "_0000";
-
-        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
-        confLocal.addConfiguration(dlConf);
-        confLocal.setMaxCachedPartitionsPerProxy(-1);
-        confLocal.setMaxAcquiredPartitionsPerProxy(1);
-
-        ServerConfiguration serverConfLocal = new ServerConfiguration();
-        serverConfLocal.addConfiguration(serverConf);
-        serverConfLocal.setStreamPartitionConverterClass(DelimiterStreamPartitionConverter.class);
-
-        DistributedLogServiceImpl serviceLocal = createService(serverConfLocal, confLocal);
-        Stream stream = serviceLocal.getLogWriter(streamName);
-
-        // stream is cached
-        assertNotNull(stream);
-        assertEquals(1, serviceLocal.getStreamManager().numCached());
-
-        // create write ops
-        WriteOp op0 = createWriteOp(service, streamName, 0L);
-        stream.submit(op0);
-        WriteResponse wr0 = Await.result(op0.result());
-        assertEquals("Op 0 should succeed",
-                StatusCode.SUCCESS, wr0.getHeader().getCode());
-        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
-
-        // should be able to cache partitions from same stream
-        String anotherStreamName = testName.getMethodName() + "_0001";
-        Stream anotherStream = serviceLocal.getLogWriter(anotherStreamName);
-        assertNotNull(anotherStream);
-        assertEquals(2, serviceLocal.getStreamManager().numCached());
-
-        // create write ops
-        WriteOp op1 = createWriteOp(service, anotherStreamName, 0L);
-        anotherStream.submit(op1);
-        WriteResponse wr1 = Await.result(op1.result());
-        assertEquals("Op 1 should fail",
-                StatusCode.STREAM_UNAVAILABLE, wr1.getHeader().getCode());
-        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseShouldErrorOutPendingOps() throws Exception {
-        String streamName = testName.getMethodName();
-        StreamImpl s = createUnstartedStream(service, streamName);
-
-        int numWrites = 10;
-        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>(numWrites);
-        for (int i = 0; i < numWrites; i++) {
-            WriteOp op = createWriteOp(service, streamName, i);
-            s.submit(op);
-            futureList.add(op.result());
-        }
-        assertEquals(numWrites, s.numPendingOps());
-        Await.result(s.requestClose("close stream"));
-        assertEquals("Stream " + streamName + " is set to " + StreamStatus.CLOSED,
-                StreamStatus.CLOSED, s.getStatus());
-        for (int i = 0; i < numWrites; i++) {
-            Future<WriteResponse> future = futureList.get(i);
-            WriteResponse wr = Await.result(future);
-            assertEquals("Pending op should fail with " + StatusCode.STREAM_UNAVAILABLE,
-                    StatusCode.STREAM_UNAVAILABLE, wr.getHeader().getCode());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseTwice() throws Exception {
-        String streamName = testName.getMethodName();
-        StreamImpl s = createUnstartedStream(service, streamName);
-
-        int numWrites = 10;
-        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>(numWrites);
-        for (int i = 0; i < numWrites; i++) {
-            WriteOp op = createWriteOp(service, streamName, i);
-            s.submit(op);
-            futureList.add(op.result());
-        }
-        assertEquals(numWrites, s.numPendingOps());
-
-        Future<Void> closeFuture0 = s.requestClose("close 0");
-        assertTrue("Stream " + streamName + " should be set to " + StreamStatus.CLOSING,
-                StreamStatus.CLOSING == s.getStatus()
-                    || StreamStatus.CLOSED == s.getStatus());
-        Future<Void> closeFuture1 = s.requestClose("close 1");
-        assertTrue("Stream " + streamName + " should be set to " + StreamStatus.CLOSING,
-                StreamStatus.CLOSING == s.getStatus()
-                    || StreamStatus.CLOSED == s.getStatus());
-
-        Await.result(closeFuture0);
-        assertEquals("Stream " + streamName + " should be set to " + StreamStatus.CLOSED,
-                StreamStatus.CLOSED, s.getStatus());
-        Await.result(closeFuture1);
-        assertEquals("Stream " + streamName + " should be set to " + StreamStatus.CLOSED,
-                StreamStatus.CLOSED, s.getStatus());
-
-        for (int i = 0; i < numWrites; i++) {
-            Future<WriteResponse> future = futureList.get(i);
-            WriteResponse wr = Await.result(future);
-            assertEquals("Pending op should fail with " + StatusCode.STREAM_UNAVAILABLE,
-                    StatusCode.STREAM_UNAVAILABLE, wr.getHeader().getCode());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testFailRequestsDuringClosing() throws Exception {
-        String streamName = testName.getMethodName();
-        StreamImpl s = createUnstartedStream(service, streamName);
-
-        Future<Void> closeFuture = s.requestClose("close");
-        assertTrue("Stream " + streamName + " should be set to " + StreamStatus.CLOSING,
-                StreamStatus.CLOSING == s.getStatus()
-                    || StreamStatus.CLOSED == s.getStatus());
-        WriteOp op1 = createWriteOp(service, streamName, 0L);
-        s.submit(op1);
-        WriteResponse response1 = Await.result(op1.result());
-        assertEquals("Op should fail with " + StatusCode.STREAM_UNAVAILABLE + " if it is closing",
-                StatusCode.STREAM_UNAVAILABLE, response1.getHeader().getCode());
-
-        Await.result(closeFuture);
-        assertEquals("Stream " + streamName + " should be set to " + StreamStatus.CLOSED,
-                StreamStatus.CLOSED, s.getStatus());
-        WriteOp op2 = createWriteOp(service, streamName, 1L);
-        s.submit(op2);
-        WriteResponse response2 = Await.result(op2.result());
-        assertEquals("Op should fail with " + StatusCode.STREAM_UNAVAILABLE + " if it is closed",
-                StatusCode.STREAM_UNAVAILABLE, response2.getHeader().getCode());
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceTimeout() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
-                .setImmediateFlushEnabled(false)
-                .setPeriodicFlushFrequencyMilliSeconds(0);
-        ServerConfiguration serverConfLocal = newLocalServerConf();
-        serverConfLocal.addConfiguration(serverConf);
-        serverConfLocal.setServiceTimeoutMs(200)
-                .setStreamProbationTimeoutMs(100);
-        String streamName = testName.getMethodName();
-        // create a new service with 200ms timeout
-        DistributedLogServiceImpl localService = createService(serverConfLocal, confLocal);
-        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
-
-        int numWrites = 10;
-        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>(numWrites);
-        for (int i = 0; i < numWrites; i++) {
-            futureList.add(localService.write(streamName, createRecord(i)));
-        }
-
-        assertTrue("Stream " + streamName + " should be cached",
-                streamManager.getCachedStreams().containsKey(streamName));
-
-        StreamImpl s = (StreamImpl) streamManager.getCachedStreams().get(streamName);
-        // the stream should be set CLOSING
-        while (StreamStatus.CLOSING != s.getStatus()
-            && StreamStatus.CLOSED != s.getStatus()) {
-            TimeUnit.MILLISECONDS.sleep(20);
-        }
-        assertNotNull("Writer should be initialized", s.getWriter());
-        assertNull("No exception should be thrown", s.getLastException());
-        Future<Void> closeFuture = s.getCloseFuture();
-        Await.result(closeFuture);
-        for (int i = 0; i < numWrites; i++) {
-            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,
-                    StatusCode.BK_TRANSMIT_ERROR == response.getHeader().getCode()
-                        || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
-                        || StatusCode.WRITE_CANCELLED_EXCEPTION == response.getHeader().getCode());
-        }
-
-        while (streamManager.getCachedStreams().containsKey(streamName)) {
-            TimeUnit.MILLISECONDS.sleep(20);
-        }
-
-        assertFalse("Stream should be removed from cache",
-                streamManager.getCachedStreams().containsKey(streamName));
-        assertFalse("Stream should be removed from acquired cache",
-                streamManager.getAcquiredStreams().containsKey(streamName));
-
-        localService.shutdown();
-    }
-
-    private DistributedLogServiceImpl createConfiguredLocalService() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setOutputBufferSize(0)
-                .setImmediateFlushEnabled(true)
-                .setPeriodicFlushFrequencyMilliSeconds(0);
-        return createService(serverConf, confLocal);
-    }
-
-    private ByteBuffer getTestDataBuffer() {
-        return ByteBuffer.wrap("test-data".getBytes());
-    }
-
-    @Test(timeout = 60000)
-    public void testNonDurableWrite() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
-                .setImmediateFlushEnabled(false)
-                .setPeriodicFlushFrequencyMilliSeconds(0)
-                .setDurableWriteEnabled(false);
-        ServerConfiguration serverConfLocal = new ServerConfiguration();
-        serverConfLocal.addConfiguration(serverConf);
-        serverConfLocal.enableDurableWrite(false);
-        serverConfLocal.setServiceTimeoutMs(Integer.MAX_VALUE)
-                .setStreamProbationTimeoutMs(Integer.MAX_VALUE);
-        String streamName = testName.getMethodName();
-        DistributedLogServiceImpl localService =
-                createService(serverConfLocal, confLocal);
-        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
-
-        int numWrites = 10;
-        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>();
-        for (int i = 0; i < numWrites; i++) {
-            futureList.add(localService.write(streamName, createRecord(i)));
-        }
-        assertTrue("Stream " + streamName + " should be cached",
-                streamManager.getCachedStreams().containsKey(streamName));
-        List<WriteResponse> resultList = FutureUtils.result(Future.collect(futureList));
-        for (WriteResponse wr : resultList) {
-            assertEquals(DLSN.InvalidDLSN, DLSN.deserialize(wr.getDlsn()));
-        }
-
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteOpNoChecksum() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext();
-        Future<WriteResponse> result = localService.writeWithContext("test", getTestDataBuffer(), ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testTruncateOpNoChecksum() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext();
-        Future<WriteResponse> result = localService.truncate("test", new DLSN(1, 2, 3).serialize(), ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testStreamOpNoChecksum() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext();
-        HeartbeatOptions option = new HeartbeatOptions();
-        option.setSendHeartBeatToReader(true);
-
-        // hearbeat to acquire the stream and then release the stream
-        Future<WriteResponse> result = localService.heartbeatWithOptions("test", ctx, option);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
-        result = localService.release("test", ctx);
-        resp = Await.result(result);
-        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
-
-        // heartbeat to acquire the stream and then delete the stream
-        result = localService.heartbeatWithOptions("test", ctx, option);
-        resp = Await.result(result);
-        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
-        result = localService.delete("test", ctx);
-        resp = Await.result(result);
-        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
-
-        // shutdown the local service
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteOpChecksumBadChecksum() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext().setCrc32(999);
-        Future<WriteResponse> result = localService.writeWithContext("test", getTestDataBuffer(), ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteOpChecksumBadStream() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext().setCrc32(
-            ProtocolUtils.writeOpCRC32("test", getTestDataBuffer().array()));
-        Future<WriteResponse> result = localService.writeWithContext("test1", getTestDataBuffer(), ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testWriteOpChecksumBadData() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        ByteBuffer buffer = getTestDataBuffer();
-        WriteContext ctx = new WriteContext().setCrc32(
-            ProtocolUtils.writeOpCRC32("test", buffer.array()));
-
-        // Overwrite 1 byte to corrupt data.
-        buffer.put(1, (byte) 0xab);
-        Future<WriteResponse> result = localService.writeWithContext("test", buffer, ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testStreamOpChecksumBadChecksum() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext().setCrc32(999);
-        Future<WriteResponse> result = localService.heartbeat("test", ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        result = localService.release("test", ctx);
-        resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        result = localService.delete("test", ctx);
-        resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testTruncateOpChecksumBadChecksum() throws Exception {
-        DistributedLogServiceImpl localService = createConfiguredLocalService();
-        WriteContext ctx = new WriteContext().setCrc32(999);
-        Future<WriteResponse> result = localService.truncate("test", new DLSN(1, 2, 3).serialize(), ctx);
-        WriteResponse resp = Await.result(result);
-        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
-        localService.shutdown();
-    }
-
-    private WriteOp getWriteOp(String name, SettableFeature disabledFeature, Long checksum) {
-        return new WriteOp(name,
-            ByteBuffer.wrap("test".getBytes()),
-            new NullStatsLogger(),
-            new NullStatsLogger(),
-            new IdentityStreamPartitionConverter(),
-            new ServerConfiguration(),
-            (byte) 0,
-            checksum,
-            false,
-            disabledFeature,
-            DefaultAccessControlManager.INSTANCE);
-    }
-
-    @Test(timeout = 60000)
-    public void testStreamOpBadChecksumWithChecksumDisabled() throws Exception {
-        String streamName = testName.getMethodName();
-
-        SettableFeature disabledFeature = new SettableFeature("", 0);
-
-        WriteOp writeOp0 = getWriteOp(streamName, disabledFeature, 919191L);
-        WriteOp writeOp1 = getWriteOp(streamName, disabledFeature, 919191L);
-
-        try {
-            writeOp0.preExecute();
-            fail("should have thrown");
-        } catch (Exception ex) {
-        }
-
-        disabledFeature.set(1);
-        writeOp1.preExecute();
-    }
-
-    @Test(timeout = 60000)
-    public void testStreamOpGoodChecksumWithChecksumDisabled() throws Exception {
-        String streamName = testName.getMethodName();
-
-        SettableFeature disabledFeature = new SettableFeature("", 1);
-        WriteOp writeOp0 = getWriteOp(
-            streamName,
-            disabledFeature,
-            ProtocolUtils.writeOpCRC32(streamName, "test".getBytes()));
-        WriteOp writeOp1 = getWriteOp(
-            streamName,
-            disabledFeature,
-            ProtocolUtils.writeOpCRC32(streamName, "test".getBytes()));
-
-        writeOp0.preExecute();
-        disabledFeature.set(0);
-        writeOp1.preExecute();
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseStreamsShouldFlush() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
-                .setImmediateFlushEnabled(false)
-                .setPeriodicFlushFrequencyMilliSeconds(0);
-
-        String streamNamePrefix = testName.getMethodName();
-        DistributedLogServiceImpl localService = createService(serverConf, confLocal);
-        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
-
-        int numStreams = 10;
-        int numWrites = 10;
-        List<Future<WriteResponse>> futureList =
-                Lists.newArrayListWithExpectedSize(numStreams * numWrites);
-        for (int i = 0; i < numStreams; i++) {
-            String streamName = streamNamePrefix + "-" + i;
-            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));
-            for (int j = 0; j < numWrites; j++) {
-                futureList.add(localService.write(streamName, createRecord(i * numWrites + j)));
-            }
-        }
-
-        assertEquals("There should be " + numStreams + " streams in cache",
-                numStreams, streamManager.getCachedStreams().size());
-        while (streamManager.getAcquiredStreams().size() < numStreams) {
-            TimeUnit.MILLISECONDS.sleep(20);
-        }
-
-        Future<List<Void>> closeResult = localService.closeStreams();
-        List<Void> closedStreams = Await.result(closeResult);
-        assertEquals("There should be " + numStreams + " streams closed",
-                numStreams, closedStreams.size());
-        // all writes should be flushed
-        for (Future<WriteResponse> future : futureList) {
-            WriteResponse response = Await.result(future);
-            assertTrue("Op should succeed or be rejected : " + response.getHeader().getCode(),
-                    StatusCode.SUCCESS == response.getHeader().getCode()
-                        || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
-                        || StatusCode.STREAM_UNAVAILABLE == response.getHeader().getCode());
-        }
-        assertTrue("There should be no streams in the cache",
-                streamManager.getCachedStreams().isEmpty());
-        assertTrue("There should be no streams in the acquired cache",
-                streamManager.getAcquiredStreams().isEmpty());
-
-        localService.shutdown();
-    }
-
-    @Test(timeout = 60000)
-    public void testCloseStreamsShouldAbort() throws Exception {
-        DistributedLogConfiguration confLocal = newLocalConf();
-        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
-                .setImmediateFlushEnabled(false)
-                .setPeriodicFlushFrequencyMilliSeconds(0);
-
-        String streamNamePrefix = testName.getMethodName();
-        DistributedLogServiceImpl localService = createService(serverConf, confLocal);
-        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
-
-        int numStreams = 10;
-        int numWrites = 10;
-        List<Future<WriteResponse>> futureList =
-                Lists.newArrayListWithExpectedSize(numStreams * numWrites);
-        for (int i = 0; i < numStreams; i++) {
-            String streamName = streamNamePrefix + "-" + i;
-            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));
-            for (int j = 0; j < numWrites; j++) {
-                futureList.add(localService.write(streamName, createRecord(i * numWrites + j)));
-            }
-        }
-
-        assertEquals("There should be " + numStreams + " streams in cache",
-                numStreams, streamManager.getCachedStreams().size());
-        while (streamManager.getAcquiredStreams().size() < numStreams) {
-            TimeUnit.MILLISECONDS.sleep(20);
-        }
-
-        for (Stream s : streamManager.getAcquiredStreams().values()) {
-            StreamImpl stream = (StreamImpl) s;
-            stream.setStatus(StreamStatus.ERROR);
-        }
-
-        Future<List<Void>> closeResult = localService.closeStreams();
-        List<Void> closedStreams = Await.result(closeResult);
-        assertEquals("There should be " + numStreams + " streams closed",
-                numStreams, closedStreams.size());
-        // all writes should be flushed
-        for (Future<WriteResponse> future : futureList) {
-            WriteResponse response = Await.result(future);
-            assertTrue("Op should fail with " + StatusCode.BK_TRANSMIT_ERROR + " or be rejected : "
-                    + response.getHeader().getCode(),
-                    StatusCode.BK_TRANSMIT_ERROR == response.getHeader().getCode()
-                        || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
-                        || StatusCode.WRITE_CANCELLED_EXCEPTION == response.getHeader().getCode());
-        }
-        // acquired streams should all been removed after we close them
-        assertTrue("There should be no streams in the acquired cache",
-            streamManager.getAcquiredStreams().isEmpty());
-        localService.shutdown();
-        // cached streams wouldn't be removed immediately after streams are closed
-        // but they should be removed after we shutdown the service
-        assertTrue("There should be no streams in the cache after shutting down the service",
-            streamManager.getCachedStreams().isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testShutdown() throws Exception {
-        service.shutdown();
-        StreamManagerImpl streamManager = (StreamManagerImpl) service.getStreamManager();
-        WriteResponse response =
-                Await.result(service.write(testName.getMethodName(), createRecord(0L)));
-        assertEquals("Write should fail with " + StatusCode.SERVICE_UNAVAILABLE,
-                StatusCode.SERVICE_UNAVAILABLE, response.getHeader().getCode());
-        assertTrue("There should be no streams created after shutdown",
-                streamManager.getCachedStreams().isEmpty());
-        assertTrue("There should be no streams acquired after shutdown",
-                streamManager.getAcquiredStreams().isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testGetOwner() throws Exception {
-        ((LocalRoutingService) service.getRoutingService())
-                .addHost("stream-0", service.getServiceAddress().getSocketAddress())
-                .setAllowRetrySameHost(false);
-
-        service.startPlacementPolicy();
-
-        WriteResponse response = FutureUtils.result(service.getOwner("stream-1", new WriteContext()));
-        assertEquals(StatusCode.FOUND, response.getHeader().getCode());
-        assertEquals(service.getServiceAddress().toString(),
-                response.getHeader().getLocation());
-
-        // service cache "stream-2"
-        StreamImpl stream = (StreamImpl) service.getStreamManager().getOrCreateStream("stream-2", false);
-        // create write ops to stream-2 to make service acquire the stream
-        WriteOp op = createWriteOp(service, "stream-2", 0L);
-        stream.submit(op);
-        stream.start();
-        WriteResponse wr = Await.result(op.result());
-        assertEquals("Op should succeed",
-                StatusCode.SUCCESS, wr.getHeader().getCode());
-        assertEquals("Service should acquire stream",
-                StreamStatus.INITIALIZED, stream.getStatus());
-        assertNotNull(stream.getManager());
-        assertNotNull(stream.getWriter());
-        assertNull(stream.getLastException());
-
-        // the stream is acquired
-        response = FutureUtils.result(service.getOwner("stream-2", new WriteContext()));
-        assertEquals(StatusCode.FOUND, response.getHeader().getCode());
-        assertEquals(service.getServiceAddress().toString(),
-                response.getHeader().getLocation());
-    }
-
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestRegionUnavailable.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestRegionUnavailable.java
deleted file mode 100644
index 50c915f..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestRegionUnavailable.java
+++ /dev/null
@@ -1,140 +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 com.twitter.distributedlog.service;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.feature.DefaultFeatureProvider;
-import com.twitter.distributedlog.service.DistributedLogCluster.DLServer;
-import java.net.SocketAddress;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import org.apache.bookkeeper.feature.Feature;
-import org.apache.bookkeeper.feature.FeatureProvider;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test Case for {@link com.twitter.distributedlog.exceptions.RegionUnavailableException}.
- */
-public class TestRegionUnavailable extends DistributedLogServerTestCase {
-
-    /**
-     * A feature provider for testing.
-     */
-    public static class TestFeatureProvider extends DefaultFeatureProvider {
-
-        public TestFeatureProvider(String rootScope,
-                                   DistributedLogConfiguration conf,
-                                   StatsLogger statsLogger) {
-            super(rootScope, conf, statsLogger);
-        }
-
-        @Override
-        protected Feature makeFeature(String featureName) {
-            if (featureName.contains(ServerFeatureKeys.REGION_STOP_ACCEPT_NEW_STREAM.name().toLowerCase())) {
-                return new SettableFeature(featureName, 10000);
-            }
-            return super.makeFeature(featureName);
-        }
-
-        @Override
-        protected FeatureProvider makeProvider(String fullScopeName) {
-            return super.makeProvider(fullScopeName);
-        }
-    }
-
-    private final int numServersPerDC = 3;
-    private final List<DLServer> localCluster;
-    private final List<DLServer> remoteCluster;
-    private TwoRegionDLClient client;
-
-    public TestRegionUnavailable() {
-        super(true);
-        this.localCluster = new ArrayList<DLServer>();
-        this.remoteCluster = new ArrayList<DLServer>();
-    }
-
-    @Before
-    @Override
-    public void setup() throws Exception {
-        DistributedLogConfiguration localConf = new DistributedLogConfiguration();
-        localConf.addConfiguration(conf);
-        localConf.setFeatureProviderClass(TestFeatureProvider.class);
-        DistributedLogConfiguration remoteConf = new DistributedLogConfiguration();
-        remoteConf.addConfiguration(conf);
-        super.setup();
-        int localPort = 9010;
-        int remotePort = 9020;
-        for (int i = 0; i < numServersPerDC; i++) {
-            localCluster.add(createDistributedLogServer(localConf, localPort + i));
-            remoteCluster.add(createDistributedLogServer(remoteConf, remotePort + i));
-        }
-        Map<SocketAddress, String> regionMap = new HashMap<SocketAddress, String>();
-        for (DLServer server : localCluster) {
-            regionMap.put(server.getAddress(), "local");
-        }
-        for (DLServer server : remoteCluster) {
-            regionMap.put(server.getAddress(), "remote");
-        }
-        client = createTwoRegionDLClient("two_regions_client", regionMap);
-
-    }
-
-    private void registerStream(String streamName) {
-        for (DLServer server : localCluster) {
-            client.localRoutingService.addHost(streamName, server.getAddress());
-        }
-        client.remoteRoutingService.addHost(streamName, remoteCluster.get(0).getAddress());
-    }
-
-    @After
-    @Override
-    public void teardown() throws Exception {
-        super.teardown();
-        if (null != client) {
-            client.shutdown();
-        }
-        for (DLServer server : localCluster) {
-            server.shutdown();
-        }
-        for (DLServer server : remoteCluster) {
-            server.shutdown();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testRegionUnavailable() throws Exception {
-        String name = "dlserver-region-unavailable";
-        registerStream(name);
-
-        for (long i = 1; i <= 10; i++) {
-            client.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())).get();
-        }
-
-        // check local region
-        for (DLServer server : localCluster) {
-            checkStreams(0, server);
-        }
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestStatsFilter.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestStatsFilter.java
deleted file mode 100644
index d8ef302..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/TestStatsFilter.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 com.twitter.distributedlog.service;
-
-import static org.junit.Assert.assertEquals;
-
-import com.twitter.finagle.Service;
-import com.twitter.finagle.service.ConstantService;
-import com.twitter.util.Await;
-import com.twitter.util.Future;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.bookkeeper.stats.StatsLogger;
-import org.junit.Test;
-
-/**
- * Test Case for {@link StatsFilter}.
- */
-public class TestStatsFilter {
-
-    class RuntimeExService<Req, Rep> extends Service<Req, Rep> {
-        public Future<Rep> apply(Req request) {
-            throw new RuntimeException("test");
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceSuccess() throws Exception {
-        StatsLogger stats = new NullStatsLogger();
-        StatsFilter<String, String> filter = new StatsFilter<String, String>(stats);
-        Future<String> result = filter.apply("", new ConstantService<String, String>(Future.value("result")));
-        assertEquals("result", Await.result(result));
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceFailure() throws Exception {
-        StatsLogger stats = new NullStatsLogger();
-        StatsFilter<String, String> filter = new StatsFilter<String, String>(stats);
-        try {
-            filter.apply("", new RuntimeExService<String, String>());
-        } catch (RuntimeException ex) {
-        }
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestBalancerUtils.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestBalancerUtils.java
deleted file mode 100644
index 2ae3a23..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestBalancerUtils.java
+++ /dev/null
@@ -1,65 +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 com.twitter.distributedlog.service.balancer;
-
-import static org.junit.Assert.assertEquals;
-
-import java.util.HashMap;
-import java.util.Map;
-import org.junit.Test;
-
-/**
- * Test Case for {@link BalancerUtils}.
- */
-public class TestBalancerUtils {
-
-    @Test(timeout = 60000)
-    public void testCalculateNumStreamsToRebalance() {
-        String myNode = "mynode";
-
-        // empty load distribution
-        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
-                myNode, new HashMap<String, Integer>(), 0, 10));
-        // my node doesn't exist in load distribution
-        Map<String, Integer> loadDistribution = new HashMap<String, Integer>();
-        loadDistribution.put("node1", 10);
-        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
-                myNode, loadDistribution, 0, 10));
-        // my node doesn't reach rebalance water mark
-        loadDistribution.clear();
-        loadDistribution.put("node1", 1);
-        loadDistribution.put(myNode, 100);
-        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
-                myNode, loadDistribution, 200, 10));
-        // my node is below average in the cluster.
-        loadDistribution.clear();
-        loadDistribution.put(myNode, 1);
-        loadDistribution.put("node1", 99);
-        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
-                myNode, loadDistribution, 0, 10));
-        // my node is above average in the cluster
-        assertEquals(49, BalancerUtils.calculateNumStreamsToRebalance(
-                "node1", loadDistribution, 0, 10));
-        // my node is at the tolerance range
-        loadDistribution.clear();
-        loadDistribution.put(myNode, 55);
-        loadDistribution.put("node1", 45);
-        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
-                myNode, loadDistribution, 0, 10));
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestClusterBalancer.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestClusterBalancer.java
deleted file mode 100644
index 8a24e21..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestClusterBalancer.java
+++ /dev/null
@@ -1,189 +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 com.twitter.distributedlog.service.balancer;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.fail;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-import com.twitter.distributedlog.client.monitor.MonitorServiceClient;
-import com.twitter.distributedlog.service.DLSocketAddress;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogCluster.DLServer;
-import com.twitter.distributedlog.service.DistributedLogServerTestCase;
-import com.twitter.util.Await;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-import org.apache.commons.lang3.tuple.Pair;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Ignore;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test Case for {@link ClusterBalancer}.
- */
-public class TestClusterBalancer extends DistributedLogServerTestCase {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestClusterBalancer.class);
-
-    private final int numServers = 5;
-    private final List<DLServer> cluster;
-    private DLClient client;
-
-    public TestClusterBalancer() {
-        super(true);
-        this.cluster = new ArrayList<DLServer>();
-    }
-
-    @Before
-    @Override
-    public void setup() throws Exception {
-        super.setup();
-        int initPort = 9001;
-        for (int i = 0; i < numServers; i++) {
-            cluster.add(createDistributedLogServer(initPort + i));
-        }
-        client = createDistributedLogClient("cluster_client", Optional.<String>absent());
-    }
-
-    @After
-    @Override
-    public void teardown() throws Exception {
-        super.teardown();
-        if (null != client) {
-            client.shutdown();
-        }
-        for (DLServer server: cluster) {
-            server.shutdown();
-        }
-    }
-
-    private void initStreams(String namePrefix) {
-        logger.info("Init streams with prefix {}", namePrefix);
-        // Stream Distribution: 5, 4, 3, 2, 1
-        initStreams(namePrefix, 5, 1, 0);
-        initStreams(namePrefix, 4, 6, 1);
-        initStreams(namePrefix, 3, 10, 2);
-        initStreams(namePrefix, 2, 13, 3);
-        initStreams(namePrefix, 1, 15, 4);
-    }
-
-    private void initStreams(String namePrefix, int numStreams, int streamId, int proxyId) {
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + (streamId++);
-            client.routingService.addHost(name, cluster.get(proxyId).getAddress());
-        }
-    }
-
-    private void writeStreams(String namePrefix) throws Exception {
-        logger.info("Write streams with prefix {}", namePrefix);
-        writeStreams(namePrefix, 5, 1);
-        writeStreams(namePrefix, 4, 6);
-        writeStreams(namePrefix, 3, 10);
-        writeStreams(namePrefix, 2, 13);
-        writeStreams(namePrefix, 1, 15);
-    }
-
-    private void writeStreams(String namePrefix, int numStreams, int streamId) throws Exception {
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + (streamId++);
-            try {
-                Await.result(client.dlClient.write(name, ByteBuffer.wrap(name.getBytes(UTF_8))));
-            } catch (Exception e) {
-                logger.error("Error writing stream {} : ", name, e);
-                throw e;
-            }
-        }
-    }
-
-    private void validateStreams(String namePrefix) throws Exception {
-        logger.info("Validate streams with prefix {}", namePrefix);
-        validateStreams(namePrefix, 5, 1, 0);
-        validateStreams(namePrefix, 4, 6, 1);
-        validateStreams(namePrefix, 3, 10, 2);
-        validateStreams(namePrefix, 2, 13, 3);
-        validateStreams(namePrefix, 1, 15, 4);
-    }
-
-    private void validateStreams(String namePrefix, int numStreams, int streamId, int proxyIdx) {
-        Set<String> expectedStreams = new HashSet<String>();
-        for (int i = 0; i < numStreams; i++) {
-            expectedStreams.add(namePrefix + (streamId++));
-        }
-        checkStreams(expectedStreams, cluster.get(proxyIdx));
-    }
-
-    @Ignore
-    @Test(timeout = 60000)
-    public void testBalanceAll() throws Exception {
-        String namePrefix = "clusterbalancer-balance-all-";
-
-        initStreams(namePrefix);
-        writeStreams(namePrefix);
-        validateStreams(namePrefix);
-
-        Optional<RateLimiter> rateLimiter = Optional.absent();
-
-        Balancer balancer = new ClusterBalancer(client.dlClientBuilder,
-                Pair.of((DistributedLogClient) client.dlClient, (MonitorServiceClient) client.dlClient));
-        logger.info("Rebalancing from 'unknown' target");
-        try {
-            balancer.balanceAll("unknown", 10, rateLimiter);
-            fail("Should fail on balanceAll from 'unknown' target.");
-        } catch (IllegalArgumentException iae) {
-            // expected
-        }
-        validateStreams(namePrefix);
-
-        logger.info("Rebalancing from 'unexisted' host");
-        String addr = DLSocketAddress.toString(DLSocketAddress.getSocketAddress(9999));
-        balancer.balanceAll(addr, 10, rateLimiter);
-        validateStreams(namePrefix);
-
-        addr = DLSocketAddress.toString(cluster.get(0).getAddress());
-        logger.info("Rebalancing from host {}.", addr);
-        balancer.balanceAll(addr, 10, rateLimiter);
-        checkStreams(0, cluster.get(0));
-        checkStreams(4, cluster.get(1));
-        checkStreams(3, cluster.get(2));
-        checkStreams(4, cluster.get(3));
-        checkStreams(4, cluster.get(4));
-
-        addr = DLSocketAddress.toString(cluster.get(2).getAddress());
-        logger.info("Rebalancing from host {}.", addr);
-        balancer.balanceAll(addr, 10, rateLimiter);
-        checkStreams(3, cluster.get(0));
-        checkStreams(4, cluster.get(1));
-        checkStreams(0, cluster.get(2));
-        checkStreams(4, cluster.get(3));
-        checkStreams(4, cluster.get(4));
-
-        logger.info("Rebalancing the cluster");
-        balancer.balance(0, 0.0f, 10, rateLimiter);
-        for (int i = 0; i < 5; i++) {
-            checkStreams(3, cluster.get(i));
-        }
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestCountBasedStreamChooser.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestCountBasedStreamChooser.java
deleted file mode 100644
index d9c2ad1..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestCountBasedStreamChooser.java
+++ /dev/null
@@ -1,204 +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 com.twitter.distributedlog.service.balancer;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-import static org.junit.Assert.assertNull;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import com.google.common.collect.Sets;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-import org.junit.Test;
-
-/**
- * Test Case for {@link CountBasedStreamChooser}.
- */
-public class TestCountBasedStreamChooser {
-
-    @Test(timeout = 60000)
-    public void testEmptyStreamDistribution() {
-        try {
-            new CountBasedStreamChooser(new HashMap<SocketAddress, Set<String>>());
-            fail("Should fail constructing stream chooser if the stream distribution is empty");
-        } catch (IllegalArgumentException iae) {
-            // expected
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testMultipleHostsWithEmptyStreams() {
-        for (int i = 1; i <= 3; i++) {
-            Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
-            int port = 1000;
-            for (int j = 0; j < i; j++) {
-                SocketAddress address = new InetSocketAddress("127.0.0.1", port + j);
-                streamDistribution.put(address, new HashSet<String>());
-            }
-
-            CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
-            for (int k = 0; k < i + 1; k++) {
-                assertNull(chooser.choose());
-            }
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testSingleHostWithStreams() {
-        for (int i = 0; i < 3; i++) {
-            Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
-
-            Set<String> streams = new HashSet<String>();
-            for (int j = 0; j < 3; j++) {
-                streams.add("SingleHostStream-" + j);
-            }
-
-            int port = 1000;
-            SocketAddress address = new InetSocketAddress("127.0.0.1", port);
-            streamDistribution.put(address, streams);
-
-            for (int k = 1; k <= i; k++) {
-                address = new InetSocketAddress("127.0.0.1", port + k);
-                streamDistribution.put(address, new HashSet<String>());
-            }
-
-            Set<String> choosenStreams = new HashSet<String>();
-
-            CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
-            for (int l = 0; l < 3 + i + 1; l++) {
-                String s = chooser.choose();
-                if (null != s) {
-                    choosenStreams.add(s);
-                }
-            }
-
-            assertEquals(streams.size(), choosenStreams.size());
-            assertTrue(Sets.difference(streams, choosenStreams).immutableCopy().isEmpty());
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testHostsHaveSameNumberStreams() {
-        Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
-        Set<String> allStreams = new HashSet<String>();
-
-        int numHosts = 3;
-        int numStreamsPerHost = 3;
-
-        int port = 1000;
-        for (int i = 1; i <= numHosts; i++) {
-            SocketAddress address = new InetSocketAddress("127.0.0.1", port + i);
-            Set<String> streams = new HashSet<String>();
-
-            for (int j = 1; j <= numStreamsPerHost; j++) {
-                String streamName = "HostsHaveSameNumberStreams-" + i + "-" + j;
-                streams.add(streamName);
-                allStreams.add(streamName);
-            }
-
-            streamDistribution.put(address, streams);
-        }
-
-        Set<String> streamsChoosen = new HashSet<String>();
-        CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
-        for (int i = 1; i <= numStreamsPerHost; i++) {
-            for (int j = 1; j <= numHosts; j++) {
-                String s = chooser.choose();
-                assertNotNull(s);
-                streamsChoosen.add(s);
-            }
-            for (int j = 0; j < numHosts; j++) {
-                assertEquals(numStreamsPerHost - i, chooser.streamsDistribution.get(j).getRight().size());
-            }
-        }
-        assertNull(chooser.choose());
-        assertEquals(numHosts * numStreamsPerHost, streamsChoosen.size());
-        assertTrue(Sets.difference(allStreams, streamsChoosen).isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testHostsHaveDifferentNumberStreams() {
-        Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
-        Set<String> allStreams = new HashSet<String>();
-
-        int numHosts = 6;
-        int maxStreamsPerHost = 4;
-
-        int port = 1000;
-        for (int i = 0; i < numHosts; i++) {
-            int group = i / 2;
-            int numStreamsThisGroup = maxStreamsPerHost - group;
-
-            SocketAddress address = new InetSocketAddress("127.0.0.1", port + i);
-            Set<String> streams = new HashSet<String>();
-
-            for (int j = 1; j <= numStreamsThisGroup; j++) {
-                String streamName = "HostsHaveDifferentNumberStreams-" + i + "-" + j;
-                streams.add(streamName);
-                allStreams.add(streamName);
-            }
-
-            streamDistribution.put(address, streams);
-        }
-
-        Set<String> streamsChoosen = new HashSet<String>();
-        CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
-
-        for (int i = 0; i < allStreams.size(); i++) {
-            String s = chooser.choose();
-            assertNotNull(s);
-            streamsChoosen.add(s);
-        }
-        assertNull(chooser.choose());
-        assertEquals(allStreams.size(), streamsChoosen.size());
-        assertTrue(Sets.difference(allStreams, streamsChoosen).isEmpty());
-    }
-
-    @Test(timeout = 60000)
-    public void testLimitedStreamChooser() {
-        Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
-
-        Set<String> streams = new HashSet<String>();
-        for (int j = 0; j < 10; j++) {
-            streams.add("SingleHostStream-" + j);
-        }
-
-        int port = 1000;
-        SocketAddress address = new InetSocketAddress("127.0.0.1", port);
-        streamDistribution.put(address, streams);
-
-        Set<String> choosenStreams = new HashSet<String>();
-
-        CountBasedStreamChooser underlying = new CountBasedStreamChooser(streamDistribution);
-        LimitedStreamChooser chooser = LimitedStreamChooser.of(underlying, 1);
-        for (int l = 0; l < 10; l++) {
-            String s = chooser.choose();
-            if (null != s) {
-                choosenStreams.add(s);
-            }
-        }
-
-        assertEquals(1, choosenStreams.size());
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestSimpleBalancer.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestSimpleBalancer.java
deleted file mode 100644
index 04656bc..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestSimpleBalancer.java
+++ /dev/null
@@ -1,180 +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 com.twitter.distributedlog.service.balancer;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.RateLimiter;
-import com.twitter.distributedlog.service.DistributedLogCluster.DLServer;
-import com.twitter.distributedlog.service.DistributedLogServerTestCase;
-import com.twitter.util.Await;
-import java.nio.ByteBuffer;
-import java.util.Set;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
-/**
- * Test Case for {@link SimpleBalancer}.
- */
-public class TestSimpleBalancer extends DistributedLogServerTestCase {
-
-    private static final Logger logger = LoggerFactory.getLogger(TestSimpleBalancer.class);
-
-    DLClient targetClient;
-    DLServer targetServer;
-
-    public TestSimpleBalancer() {
-        super(true);
-    }
-
-    @Before
-    @Override
-    public void setup() throws Exception {
-        super.setup();
-        targetServer = createDistributedLogServer(7003);
-        targetClient = createDistributedLogClient("target", Optional.<String>absent());
-    }
-
-    @After
-    @Override
-    public void teardown() throws Exception {
-        super.teardown();
-        if (null != targetClient) {
-            targetClient.shutdown();
-        }
-        if (null != targetServer) {
-            targetServer.shutdown();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testBalanceAll() throws Exception {
-        String namePrefix = "simplebalancer-balance-all-";
-        int numStreams = 10;
-
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            // src client
-            dlClient.routingService.addHost(name, dlServer.getAddress());
-            // target client
-            targetClient.routingService.addHost(name, targetServer.getAddress());
-        }
-
-        // write to multiple streams
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes(UTF_8))));
-        }
-
-        // validation
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            checkStream(name, dlClient, dlServer, 1, numStreams, numStreams, true, true);
-            checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
-        }
-
-        Optional<RateLimiter> rateLimiter = Optional.absent();
-
-        Balancer balancer = new SimpleBalancer("source", dlClient.dlClient, dlClient.dlClient,
-                                               "target", targetClient.dlClient, targetClient.dlClient);
-        logger.info("Rebalancing from 'unknown' target");
-        try {
-            balancer.balanceAll("unknown", 10, rateLimiter);
-            fail("Should fail on balanceAll from 'unknown' target.");
-        } catch (IllegalArgumentException iae) {
-            // expected
-        }
-
-        // nothing to balance from 'target'
-        logger.info("Rebalancing from 'target' target");
-        balancer.balanceAll("target", 1, rateLimiter);
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            checkStream(name, dlClient, dlServer, 1, numStreams, numStreams, true, true);
-            checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
-        }
-
-        // balance all streams from 'source'
-        logger.info("Rebalancing from 'source' target");
-        balancer.balanceAll("source", 10, rateLimiter);
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            checkStream(name, targetClient, targetServer, 1, numStreams, numStreams, true, true);
-            checkStream(name, dlClient, dlServer, 0, 0, 0, false, false);
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testBalanceStreams() throws Exception {
-        String namePrefix = "simplebalancer-balance-streams-";
-        int numStreams = 10;
-
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            // src client
-            dlClient.routingService.addHost(name, dlServer.getAddress());
-            // target client
-            targetClient.routingService.addHost(name, targetServer.getAddress());
-        }
-
-        // write to multiple streams
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes(UTF_8))));
-        }
-
-        // validation
-        for (int i = 0; i < numStreams; i++) {
-            String name = namePrefix + i;
-            checkStream(name, dlClient, dlServer, 1, numStreams, numStreams, true, true);
-            checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
-        }
-
-        Optional<RateLimiter> rateLimiter = Optional.absent();
-
-        Balancer balancer = new SimpleBalancer("source", dlClient.dlClient, dlClient.dlClient,
-                                               "target", targetClient.dlClient, targetClient.dlClient);
-
-        // balance all streams from 'source'
-        logger.info("Rebalancing streams between targets");
-        balancer.balance(0, 0, 10, rateLimiter);
-
-        Set<String> sourceStreams = getAllStreamsFromDistribution(getStreamOwnershipDistribution(dlClient));
-        Set<String> targetStreams = getAllStreamsFromDistribution(getStreamOwnershipDistribution(targetClient));
-
-        assertEquals(numStreams / 2, sourceStreams.size());
-        assertEquals(numStreams / 2, targetStreams.size());
-
-        for (String name : sourceStreams) {
-            checkStream(name, dlClient, dlServer, 1, numStreams / 2, numStreams / 2, true, true);
-            checkStream(name, targetClient, targetServer, 1, numStreams / 2, numStreams / 2, false, false);
-        }
-
-        for (String name : targetStreams) {
-            checkStream(name, targetClient, targetServer, 1, numStreams / 2, numStreams / 2, true, true);
-            checkStream(name, dlClient, dlServer, 1, numStreams / 2, numStreams / 2, false, false);
-        }
-    }
-
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestStreamMover.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestStreamMover.java
deleted file mode 100644
index d666cf7..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/balancer/TestStreamMover.java
+++ /dev/null
@@ -1,86 +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 com.twitter.distributedlog.service.balancer;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.junit.Assert.assertTrue;
-
-import com.google.common.base.Optional;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogCluster.DLServer;
-import com.twitter.distributedlog.service.DistributedLogServerTestCase;
-import com.twitter.util.Await;
-import java.nio.ByteBuffer;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test Case for {@link StreamMover}.
- */
-public class TestStreamMover extends DistributedLogServerTestCase {
-
-    DLClient targetClient;
-    DLServer targetServer;
-
-    public TestStreamMover() {
-        super(true);
-    }
-
-    @Before
-    @Override
-    public void setup() throws Exception {
-        super.setup();
-        targetServer = createDistributedLogServer(7003);
-        targetClient = createDistributedLogClient("target", Optional.<String>absent());
-    }
-
-    @After
-    @Override
-    public void teardown() throws Exception {
-        super.teardown();
-        if (null != targetClient) {
-            targetClient.shutdown();
-        }
-        if (null != targetServer) {
-            targetServer.shutdown();
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testMoveStream() throws Exception {
-        String name = "dlserver-move-stream";
-
-        // src client
-        dlClient.routingService.addHost(name, dlServer.getAddress());
-        // target client
-        targetClient.routingService.addHost(name, targetServer.getAddress());
-
-        // src client write a record to that stream
-        Await.result(((DistributedLogClient) dlClient.dlClient).write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
-        checkStream(name, dlClient, dlServer, 1, 1, 1, true, true);
-        checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
-
-        StreamMover streamMover = new StreamMoverImpl("source", dlClient.dlClient, dlClient.dlClient,
-                                                      "target", targetClient.dlClient, targetClient.dlClient);
-        assertTrue(streamMover.moveStream(name));
-        checkStream(name, dlClient, dlServer, 0, 0, 0, false, false);
-        checkStream(name, targetClient, targetServer, 1, 1, 1, true, true);
-    }
-
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/config/TestServerConfiguration.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/config/TestServerConfiguration.java
deleted file mode 100644
index 85ceb95..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/config/TestServerConfiguration.java
+++ /dev/null
@@ -1,68 +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 com.twitter.distributedlog.service.config;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.junit.Test;
-
-/**
- * Test Case for {@link ServerConfiguration}.
- */
-public class TestServerConfiguration {
-
-    @Test(timeout = 60000, expected = IllegalArgumentException.class)
-    public void testUnassignedShardId() {
-        new ServerConfiguration().validate();
-    }
-
-    @Test(timeout = 60000)
-    public void testAssignedShardId() {
-        ServerConfiguration conf = new ServerConfiguration();
-        conf.setServerShardId(100);
-        conf.validate();
-        assertEquals(100, conf.getServerShardId());
-    }
-
-    @Test(timeout = 60000, expected = IllegalArgumentException.class)
-    public void testInvalidServerThreads() {
-        ServerConfiguration conf = new ServerConfiguration();
-        conf.setServerShardId(100);
-        conf.setServerThreads(-1);
-        conf.validate();
-    }
-
-    @Test(timeout = 60000, expected = IllegalArgumentException.class)
-    public void testInvalidDlsnVersion() {
-        ServerConfiguration conf = new ServerConfiguration();
-        conf.setServerShardId(100);
-        conf.setDlsnVersion((byte) 9999);
-        conf.validate();
-    }
-
-    @Test(timeout = 60000)
-    public void testUseHostnameAsAllocatorPoolName() {
-        ServerConfiguration conf = new ServerConfiguration();
-        assertFalse("Should not use hostname by default", conf.isUseHostnameAsAllocatorPoolName());
-        conf.setUseHostnameAsAllocatorPoolName(true);
-        assertTrue("Should use hostname now", conf.isUseHostnameAsAllocatorPoolName());
-    }
-
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/config/TestStreamConfigProvider.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/config/TestStreamConfigProvider.java
deleted file mode 100644
index 462f4f3..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/config/TestStreamConfigProvider.java
+++ /dev/null
@@ -1,140 +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 com.twitter.distributedlog.service.config;
-
-import static com.twitter.distributedlog.DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import com.google.common.base.Optional;
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.config.PropertiesWriter;
-import com.twitter.distributedlog.service.streamset.IdentityStreamPartitionConverter;
-import com.twitter.distributedlog.service.streamset.StreamPartitionConverter;
-import java.util.concurrent.Executors;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import org.junit.Test;
-
-/**
- * Test Case for {@link StreamConfigProvider}.
- */
-public class TestStreamConfigProvider {
-    private static final String DEFAULT_CONFIG_DIR = "conf";
-    private final String defaultConfigPath;
-    private final ScheduledExecutorService configExecutorService;
-
-    public TestStreamConfigProvider() throws Exception {
-        this.configExecutorService = Executors.newScheduledThreadPool(1,
-                new ThreadFactoryBuilder().setNameFormat("DistributedLogService-Dyncfg-%d").build());
-        PropertiesWriter writer = new PropertiesWriter();
-        writer.save();
-        this.defaultConfigPath = writer.getFile().getPath();
-    }
-
-    StreamConfigProvider getServiceProvider(StreamPartitionConverter converter)
-            throws Exception {
-        return getServiceProvider(converter, DEFAULT_CONFIG_DIR);
-    }
-
-    StreamConfigProvider getServiceProvider(
-            StreamPartitionConverter converter,
-            String configPath,
-            String defaultPath) throws Exception {
-        return new ServiceStreamConfigProvider(
-                configPath,
-                defaultPath,
-                converter,
-                configExecutorService,
-                1,
-                TimeUnit.SECONDS);
-    }
-
-    StreamConfigProvider getServiceProvider(
-            StreamPartitionConverter converter,
-            String configPath) throws Exception {
-        return getServiceProvider(converter, configPath, defaultConfigPath);
-    }
-
-    StreamConfigProvider getDefaultProvider(String configFile) throws Exception {
-        return new DefaultStreamConfigProvider(configFile, configExecutorService, 1, TimeUnit.SECONDS);
-    }
-
-    StreamConfigProvider getNullProvider() throws Exception {
-        return new NullStreamConfigProvider();
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceProviderWithConfigRouters() throws Exception {
-        getServiceProvider(new IdentityStreamPartitionConverter());
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceProviderWithMissingConfig() throws Exception {
-        StreamConfigProvider provider = getServiceProvider(new IdentityStreamPartitionConverter());
-        Optional<DynamicDistributedLogConfiguration> config = provider.getDynamicStreamConfig("stream1");
-        assertTrue(config.isPresent());
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceProviderWithDefaultConfigPath() throws Exception {
-        // Default config with property set.
-        PropertiesWriter writer1 = new PropertiesWriter();
-        writer1.setProperty("rpsStreamAcquireServiceLimit", "191919");
-        writer1.save();
-        String fallbackConfPath1 = writer1.getFile().getPath();
-        StreamConfigProvider provider1 = getServiceProvider(new IdentityStreamPartitionConverter(),
-                DEFAULT_CONFIG_DIR, fallbackConfPath1);
-        Optional<DynamicDistributedLogConfiguration> config1 = provider1.getDynamicStreamConfig("stream1");
-
-        // Empty default config.
-        PropertiesWriter writer2 = new PropertiesWriter();
-        writer2.save();
-        String fallbackConfPath2 = writer2.getFile().getPath();
-        StreamConfigProvider provider2 = getServiceProvider(new IdentityStreamPartitionConverter(),
-                DEFAULT_CONFIG_DIR, fallbackConfPath2);
-        Optional<DynamicDistributedLogConfiguration> config2 = provider2.getDynamicStreamConfig("stream1");
-
-        assertEquals(191919, config1.get().getRpsStreamAcquireServiceLimit());
-        assertEquals(-1, config2.get().getRpsStreamAcquireServiceLimit());
-    }
-
-    @Test(timeout = 60000)
-    public void testDefaultProvider() throws Exception {
-        PropertiesWriter writer = new PropertiesWriter();
-        writer.setProperty(BKDL_RETENTION_PERIOD_IN_HOURS, "99");
-        writer.save();
-        StreamConfigProvider provider = getDefaultProvider(writer.getFile().getPath());
-        Optional<DynamicDistributedLogConfiguration> config1 = provider.getDynamicStreamConfig("stream1");
-        Optional<DynamicDistributedLogConfiguration> config2 = provider.getDynamicStreamConfig("stream2");
-        assertTrue(config1.isPresent());
-        assertTrue(config1.get() == config2.get());
-        assertEquals(99, config1.get().getRetentionPeriodHours());
-    }
-
-    @Test(timeout = 60000)
-    public void testNullProvider() throws Exception {
-        StreamConfigProvider provider = getNullProvider();
-        Optional<DynamicDistributedLogConfiguration> config1 = provider.getDynamicStreamConfig("stream1");
-        Optional<DynamicDistributedLogConfiguration> config2 = provider.getDynamicStreamConfig("stream2");
-        assertFalse(config1.isPresent());
-        assertTrue(config1 == config2);
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
deleted file mode 100644
index a12a64e..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
+++ /dev/null
@@ -1,176 +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
- * <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 com.twitter.distributedlog.service.placement;
-
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-import static org.mockito.Matchers.anyString;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import com.twitter.distributedlog.client.routing.RoutingService;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.util.Await;
-import com.twitter.util.Duration;
-import com.twitter.util.Future;
-import java.io.IOException;
-import java.net.InetSocketAddress;
-import java.net.SocketAddress;
-import java.util.LinkedHashSet;
-import java.util.Random;
-import java.util.Set;
-import java.util.TreeSet;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.junit.Test;
-import org.mockito.ArgumentCaptor;
-import org.mockito.invocation.InvocationOnMock;
-import org.mockito.stubbing.Answer;
-
-/**
- * Test Case for {@link LeastLoadPlacementPolicy}.
- */
-public class TestLeastLoadPlacementPolicy {
-
-    @Test(timeout = 10000)
-    public void testCalculateBalances() throws Exception {
-        int numSevers = new Random().nextInt(20) + 1;
-        int numStreams = new Random().nextInt(200) + 1;
-        RoutingService mockRoutingService = mock(RoutingService.class);
-        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
-        LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
-            new EqualLoadAppraiser(),
-            mockRoutingService,
-            mockNamespace,
-            null,
-            Duration.fromSeconds(600),
-            new NullStatsLogger());
-        TreeSet<ServerLoad> serverLoads =
-            Await.result(leastLoadPlacementPolicy.calculate(generateServers(numSevers), generateStreams(numStreams)));
-        long lowLoadPerServer = numStreams / numSevers;
-        long highLoadPerServer = lowLoadPerServer + 1;
-        for (ServerLoad serverLoad : serverLoads) {
-            long load = serverLoad.getLoad();
-            assertEquals(load, serverLoad.getStreamLoads().size());
-            assertTrue(String.format("Load %d is not between %d and %d",
-                load, lowLoadPerServer, highLoadPerServer), load == lowLoadPerServer || load == highLoadPerServer);
-        }
-    }
-
-    @Test(timeout = 10000)
-    public void testRefreshAndPlaceStream() throws Exception {
-        int numSevers = new Random().nextInt(20) + 1;
-        int numStreams = new Random().nextInt(200) + 1;
-        RoutingService mockRoutingService = mock(RoutingService.class);
-        when(mockRoutingService.getHosts()).thenReturn(generateSocketAddresses(numSevers));
-        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
-        try {
-            when(mockNamespace.getLogs()).thenReturn(generateStreams(numStreams).iterator());
-        } catch (IOException e) {
-            fail();
-        }
-        PlacementStateManager mockPlacementStateManager = mock(PlacementStateManager.class);
-        LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
-            new EqualLoadAppraiser(),
-            mockRoutingService,
-            mockNamespace,
-            mockPlacementStateManager,
-            Duration.fromSeconds(600),
-            new NullStatsLogger());
-        leastLoadPlacementPolicy.refresh();
-
-        final ArgumentCaptor<TreeSet> captor = ArgumentCaptor.forClass(TreeSet.class);
-        verify(mockPlacementStateManager).saveOwnership(captor.capture());
-        TreeSet<ServerLoad> serverLoads = (TreeSet<ServerLoad>) captor.getValue();
-        ServerLoad next = serverLoads.first();
-        String serverPlacement = Await.result(leastLoadPlacementPolicy.placeStream("newstream1"));
-        assertEquals(next.getServer(), serverPlacement);
-    }
-
-    @Test(timeout = 10000)
-    public void testCalculateUnequalWeight() throws Exception {
-        int numSevers = new Random().nextInt(20) + 1;
-        int numStreams = new Random().nextInt(200) + 1;
-    /* 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);
-        LoadAppraiser mockLoadAppraiser = mock(LoadAppraiser.class);
-        when(mockLoadAppraiser.getStreamLoad(anyString())).then(new Answer<Future<StreamLoad>>() {
-            @Override
-            public Future<StreamLoad> answer(InvocationOnMock invocationOnMock) throws Throwable {
-                int load = new Random().nextInt(100000);
-                if (load > maxLoad.get()) {
-                    maxLoad.set(load);
-                }
-                return Future.value(new StreamLoad(invocationOnMock.getArguments()[0].toString(), load));
-            }
-        });
-        LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
-            mockLoadAppraiser,
-            mockRoutingService,
-            mockNamespace,
-            null,
-            Duration.fromSeconds(600),
-            new NullStatsLogger());
-        TreeSet<ServerLoad> serverLoads =
-            Await.result(leastLoadPlacementPolicy.calculate(generateServers(numSevers), generateStreams(numStreams)));
-        long highestLoadSeen = Long.MIN_VALUE;
-        long lowestLoadSeen = Long.MAX_VALUE;
-        for (ServerLoad serverLoad : serverLoads) {
-            long load = serverLoad.getLoad();
-            if (load < lowestLoadSeen) {
-                lowestLoadSeen = load;
-            }
-            if (load > highestLoadSeen) {
-                highestLoadSeen = load;
-            }
-        }
-        assertTrue("Unexpected placement for " + numStreams + " streams to "
-                + numSevers + " servers : highest load = " + highestLoadSeen
-                + ", lowest load = " + lowestLoadSeen + ", max stream load = " + maxLoad.get(),
-            highestLoadSeen - lowestLoadSeen < maxLoad.get());
-    }
-
-    private Set<SocketAddress> generateSocketAddresses(int num) {
-        LinkedHashSet<SocketAddress> socketAddresses = new LinkedHashSet<SocketAddress>();
-        for (int i = 0; i < num; i++) {
-            socketAddresses.add(new InetSocketAddress(i));
-        }
-        return socketAddresses;
-    }
-
-    private Set<String> generateStreams(int num) {
-        LinkedHashSet<String> streams = new LinkedHashSet<String>();
-        for (int i = 0; i < num; i++) {
-            streams.add("stream_" + i);
-        }
-        return streams;
-    }
-
-    private Set<String> generateServers(int num) {
-        LinkedHashSet<String> servers = new LinkedHashSet<String>();
-        for (int i = 0; i < num; i++) {
-            servers.add("server_" + i);
-        }
-        return servers;
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/placement/TestServerLoad.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/placement/TestServerLoad.java
deleted file mode 100644
index 42aeddd..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/placement/TestServerLoad.java
+++ /dev/null
@@ -1,50 +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
- * <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 com.twitter.distributedlog.service.placement;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import org.junit.Test;
-
-/**
- * Test Case for {@link ServerLoad}.
- */
-public class TestServerLoad {
-
-    @Test(timeout = 60000)
-    public void testSerializeDeserialize() throws IOException {
-        final ServerLoad serverLoad = new ServerLoad("th1s1s@s3rv3rn@m3");
-        for (int i = 0; i < 20; i++) {
-            serverLoad.addStream(new StreamLoad("stream-" + i, i));
-        }
-        assertEquals(serverLoad, ServerLoad.deserialize(serverLoad.serialize()));
-    }
-
-    @Test(timeout = 60000)
-    public void testGetLoad() throws IOException {
-        final ServerLoad serverLoad = new ServerLoad("th1s1s@s3rv3rn@m3");
-        assertEquals(0, serverLoad.getLoad());
-        serverLoad.addStream(new StreamLoad("stream-" + 1, 3));
-        assertEquals(3, serverLoad.getLoad());
-        serverLoad.addStream(new StreamLoad("stream-" + 2, 7));
-        assertEquals(10, serverLoad.getLoad());
-        serverLoad.addStream(new StreamLoad("stream-" + 3, 1));
-        assertEquals(11, serverLoad.getLoad());
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/placement/TestStreamLoad.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/placement/TestStreamLoad.java
deleted file mode 100644
index aac30d4..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/placement/TestStreamLoad.java
+++ /dev/null
@@ -1,37 +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
- * <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 com.twitter.distributedlog.service.placement;
-
-import static org.junit.Assert.assertEquals;
-
-import java.io.IOException;
-import org.junit.Test;
-
-/**
- * Test Case for {@link StreamLoad}.
- */
-public class TestStreamLoad {
-
-    @Test(timeout = 10000)
-    public void testSerializeDeserialize() throws IOException {
-        final String streamName = "aHellaRandomStreamName";
-        final int load = 1337;
-        final StreamLoad streamLoad = new StreamLoad(streamName, load);
-        assertEquals(streamLoad, StreamLoad.deserialize(streamLoad.serialize()));
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/placement/TestZKPlacementStateManager.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/placement/TestZKPlacementStateManager.java
deleted file mode 100644
index 1d11219..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/placement/TestZKPlacementStateManager.java
+++ /dev/null
@@ -1,136 +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
- * <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 com.twitter.distributedlog.service.placement;
-
-import static com.twitter.distributedlog.LocalDLMEmulator.DLOG_NAMESPACE;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertNotNull;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import java.io.IOException;
-import java.net.URI;
-import java.util.SortedSet;
-import java.util.TreeSet;
-import java.util.concurrent.LinkedBlockingQueue;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.apache.curator.test.TestingServer;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test Case for {@link ZKPlacementStateManager}.
- */
-public class TestZKPlacementStateManager {
-    private TestingServer zkTestServer;
-    private String zkServers;
-    private URI uri;
-    private ZKPlacementStateManager zkPlacementStateManager;
-
-    @Before
-    public void startZookeeper() throws Exception {
-        zkTestServer = new TestingServer(2181);
-        zkServers = "127.0.0.1:2181";
-        uri = new URI("distributedlog-bk://" + zkServers + DLOG_NAMESPACE + "/bknamespace");
-        zkPlacementStateManager =
-            new ZKPlacementStateManager(uri, new DistributedLogConfiguration(), NullStatsLogger.INSTANCE);
-    }
-
-    @Test(timeout = 60000)
-    public void testSaveLoad() throws Exception {
-        TreeSet<ServerLoad> ownerships = new TreeSet<ServerLoad>();
-        zkPlacementStateManager.saveOwnership(ownerships);
-        SortedSet<ServerLoad> loadedOwnerships = zkPlacementStateManager.loadOwnership();
-        assertEquals(ownerships, loadedOwnerships);
-
-        ownerships.add(new ServerLoad("emptyServer"));
-        zkPlacementStateManager.saveOwnership(ownerships);
-        loadedOwnerships = zkPlacementStateManager.loadOwnership();
-        assertEquals(ownerships, loadedOwnerships);
-
-        ServerLoad sl1 = new ServerLoad("server1");
-        sl1.addStream(new StreamLoad("stream1", 3));
-        sl1.addStream(new StreamLoad("stream2", 4));
-        ServerLoad sl2 = new ServerLoad("server2");
-        sl2.addStream(new StreamLoad("stream3", 1));
-        ownerships.add(sl1);
-        ownerships.add(sl2);
-        zkPlacementStateManager.saveOwnership(ownerships);
-        loadedOwnerships = zkPlacementStateManager.loadOwnership();
-        assertEquals(ownerships, loadedOwnerships);
-
-        loadedOwnerships.remove(sl1);
-        zkPlacementStateManager.saveOwnership(ownerships);
-        loadedOwnerships = zkPlacementStateManager.loadOwnership();
-        assertEquals(ownerships, loadedOwnerships);
-    }
-
-    private TreeSet<ServerLoad> waitForServerLoadsNotificationAsc(
-        LinkedBlockingQueue<TreeSet<ServerLoad>> notificationQueue,
-        int expectedNumServerLoads) throws InterruptedException {
-        TreeSet<ServerLoad> notification = notificationQueue.take();
-        assertNotNull(notification);
-        while (notification.size() < expectedNumServerLoads) {
-            notification = notificationQueue.take();
-        }
-        assertEquals(expectedNumServerLoads, notification.size());
-        return notification;
-    }
-
-    @Test(timeout = 60000)
-    public void testWatchIndefinitely() throws Exception {
-        TreeSet<ServerLoad> ownerships = new TreeSet<ServerLoad>();
-        ownerships.add(new ServerLoad("server1"));
-        final LinkedBlockingQueue<TreeSet<ServerLoad>> serverLoadNotifications =
-            new LinkedBlockingQueue<TreeSet<ServerLoad>>();
-        PlacementStateManager.PlacementCallback callback = new PlacementStateManager.PlacementCallback() {
-            @Override
-            public void callback(TreeSet<ServerLoad> serverLoads) {
-                serverLoadNotifications.add(serverLoads);
-            }
-        };
-        zkPlacementStateManager.saveOwnership(ownerships); // need to initialize the zk path before watching
-        zkPlacementStateManager.watch(callback);
-        // cannot verify the callback here as it may call before the verify is called
-
-        zkPlacementStateManager.saveOwnership(ownerships);
-        assertEquals(ownerships, waitForServerLoadsNotificationAsc(serverLoadNotifications, 1));
-
-        ServerLoad server2 = new ServerLoad("server2");
-        server2.addStream(new StreamLoad("hella-important-stream", 415));
-        ownerships.add(server2);
-        zkPlacementStateManager.saveOwnership(ownerships);
-        assertEquals(ownerships, waitForServerLoadsNotificationAsc(serverLoadNotifications, 2));
-    }
-
-    @Test(timeout = 60000)
-    public void testZkFormatting() throws Exception {
-        final String server = "host/10.0.0.0:31351";
-        final String zkFormattedServer = "host--10.0.0.0:31351";
-        URI uri = new URI("distributedlog-bk://" + zkServers + DLOG_NAMESPACE + "/bknamespace");
-        ZKPlacementStateManager zkPlacementStateManager =
-            new ZKPlacementStateManager(uri, new DistributedLogConfiguration(), NullStatsLogger.INSTANCE);
-        assertEquals(zkFormattedServer, zkPlacementStateManager.serverToZkFormat(server));
-        assertEquals(server, zkPlacementStateManager.zkFormatToServer(zkFormattedServer));
-    }
-
-    @After
-    public void stopZookeeper() throws IOException {
-        zkTestServer.stop();
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/stream/TestStreamManager.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/stream/TestStreamManager.java
deleted file mode 100644
index 283c290..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/stream/TestStreamManager.java
+++ /dev/null
@@ -1,135 +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 com.twitter.distributedlog.service.stream;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.verify;
-import static org.mockito.Mockito.when;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.service.config.StreamConfigProvider;
-import com.twitter.distributedlog.service.streamset.Partition;
-import com.twitter.distributedlog.service.streamset.StreamPartitionConverter;
-import com.twitter.util.Await;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.ScheduledThreadPoolExecutor;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-/**
- * Test Case for StreamManager.
- */
-public class TestStreamManager {
-
-    @Rule
-    public TestName testName = new TestName();
-
-    ScheduledExecutorService mockExecutorService = mock(ScheduledExecutorService.class);
-
-    @Test(timeout = 60000)
-    public void testCollectionMethods() throws Exception {
-        Stream mockStream = mock(Stream.class);
-        when(mockStream.getStreamName()).thenReturn("stream1");
-        when(mockStream.getPartition()).thenReturn(new Partition("stream1", 0));
-        StreamFactory mockStreamFactory = mock(StreamFactory.class);
-        StreamPartitionConverter mockPartitionConverter = mock(StreamPartitionConverter.class);
-        StreamConfigProvider mockStreamConfigProvider = mock(StreamConfigProvider.class);
-        when(mockStreamFactory.create(
-                (String) any(),
-                (DynamicDistributedLogConfiguration) any(),
-                (StreamManager) any())).thenReturn(mockStream);
-        StreamManager streamManager = new StreamManagerImpl(
-                "",
-                new DistributedLogConfiguration(),
-                mockExecutorService,
-                mockStreamFactory,
-                mockPartitionConverter,
-                mockStreamConfigProvider,
-                mock(DistributedLogNamespace.class));
-
-        assertFalse(streamManager.isAcquired("stream1"));
-        assertEquals(0, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-
-        streamManager.notifyAcquired(mockStream);
-        assertTrue(streamManager.isAcquired("stream1"));
-        assertEquals(1, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-
-        streamManager.notifyReleased(mockStream);
-        assertFalse(streamManager.isAcquired("stream1"));
-        assertEquals(0, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-
-        streamManager.notifyAcquired(mockStream);
-        assertTrue(streamManager.isAcquired("stream1"));
-        assertEquals(1, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-
-        streamManager.notifyAcquired(mockStream);
-        assertTrue(streamManager.isAcquired("stream1"));
-        assertEquals(1, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-
-        streamManager.notifyReleased(mockStream);
-        assertFalse(streamManager.isAcquired("stream1"));
-        assertEquals(0, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-
-        streamManager.notifyReleased(mockStream);
-        assertFalse(streamManager.isAcquired("stream1"));
-        assertEquals(0, streamManager.numAcquired());
-        assertEquals(0, streamManager.numCached());
-    }
-
-    @Test(timeout = 60000)
-    public void testCreateStream() throws Exception {
-        Stream mockStream = mock(Stream.class);
-        final String streamName = "stream1";
-        when(mockStream.getStreamName()).thenReturn(streamName);
-        StreamFactory mockStreamFactory = mock(StreamFactory.class);
-        StreamPartitionConverter mockPartitionConverter = mock(StreamPartitionConverter.class);
-        StreamConfigProvider mockStreamConfigProvider = mock(StreamConfigProvider.class);
-        when(mockStreamFactory.create(
-            (String) any(),
-            (DynamicDistributedLogConfiguration) any(),
-            (StreamManager) any())
-        ).thenReturn(mockStream);
-        DistributedLogNamespace dlNamespace = mock(DistributedLogNamespace.class);
-        ScheduledExecutorService executorService = new ScheduledThreadPoolExecutor(1);
-
-        StreamManager streamManager = new StreamManagerImpl(
-                "",
-                new DistributedLogConfiguration(),
-                executorService,
-                mockStreamFactory,
-                mockPartitionConverter,
-                mockStreamConfigProvider,
-                dlNamespace);
-
-        assertTrue(Await.ready(streamManager.createStreamAsync(streamName)).isReturn());
-        verify(dlNamespace).createLog(streamName);
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/stream/TestStreamOp.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/stream/TestStreamOp.java
deleted file mode 100644
index 7e52ff2..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/stream/TestStreamOp.java
+++ /dev/null
@@ -1,95 +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 com.twitter.distributedlog.service.stream;
-
-import static org.junit.Assert.assertEquals;
-import static org.mockito.Mockito.any;
-import static org.mockito.Mockito.mock;
-import static org.mockito.Mockito.when;
-
-import com.twitter.distributedlog.AsyncLogWriter;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecord;
-import com.twitter.distributedlog.acl.DefaultAccessControlManager;
-import com.twitter.distributedlog.exceptions.InternalServerException;
-import com.twitter.distributedlog.service.ResponseUtils;
-import com.twitter.distributedlog.service.config.ServerConfiguration;
-import com.twitter.distributedlog.service.streamset.IdentityStreamPartitionConverter;
-import com.twitter.distributedlog.thrift.service.StatusCode;
-import com.twitter.distributedlog.thrift.service.WriteResponse;
-import com.twitter.distributedlog.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;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TestName;
-
-/**
- * Test Case for StreamOps.
- */
-public class TestStreamOp {
-
-    @Rule
-    public TestName testName = new TestName();
-
-    private WriteOp getWriteOp() {
-        SettableFeature disabledFeature = new SettableFeature("", 0);
-        return new WriteOp("test",
-            ByteBuffer.wrap("test".getBytes()),
-            new NullStatsLogger(),
-            new NullStatsLogger(),
-            new IdentityStreamPartitionConverter(),
-            new ServerConfiguration(),
-            (byte) 0,
-            null,
-            false,
-            disabledFeature,
-            DefaultAccessControlManager.INSTANCE);
-    }
-
-    @Test(timeout = 60000)
-    public void testResponseFailedTwice() throws Exception {
-        WriteOp writeOp = getWriteOp();
-        writeOp.fail(new InternalServerException("test1"));
-        writeOp.fail(new InternalServerException("test2"));
-
-        WriteResponse response = Await.result(writeOp.result());
-        assertEquals(StatusCode.INTERNAL_SERVER_ERROR, response.getHeader().getCode());
-        assertEquals(ResponseUtils.exceptionToHeader(new InternalServerException("test1")), response.getHeader());
-    }
-
-    @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.getStreamName()).thenReturn("test");
-        WriteOp writeOp = getWriteOp();
-        writeOp.execute(writer, new Sequencer() {
-            public long nextId() {
-                return 0;
-            }
-        }, new Object());
-        writeOp.fail(new InternalServerException("test2"));
-
-        WriteResponse response = Await.result(writeOp.result());
-        assertEquals(StatusCode.SUCCESS, response.getHeader().getCode());
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
deleted file mode 100644
index a65e51a..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
+++ /dev/null
@@ -1,301 +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 com.twitter.distributedlog.service.stream.limiter;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.fail;
-
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.config.ConcurrentConstConfiguration;
-import com.twitter.distributedlog.config.DynamicDistributedLogConfiguration;
-import com.twitter.distributedlog.exceptions.OverCapacityException;
-import com.twitter.distributedlog.limiter.ChainedRequestLimiter;
-import com.twitter.distributedlog.limiter.ComposableRequestLimiter;
-import com.twitter.distributedlog.limiter.ComposableRequestLimiter.CostFunction;
-import com.twitter.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
-import com.twitter.distributedlog.limiter.GuavaRateLimiter;
-import com.twitter.distributedlog.limiter.RateLimiter;
-import com.twitter.distributedlog.limiter.RequestLimiter;
-import java.util.concurrent.atomic.AtomicInteger;
-import org.apache.bookkeeper.feature.SettableFeature;
-import org.apache.bookkeeper.stats.NullStatsLogger;
-import org.junit.Test;
-
-/**
- * Test Case for {@link ServiceRequestLimiter}.
- */
-public class TestServiceRequestLimiter {
-
-    /**
-     * Mock Request.
-     */
-    class MockRequest {
-        int size;
-        MockRequest() {
-            this(1);
-        }
-        MockRequest(int size) {
-            this.size = size;
-        }
-        int getSize() {
-            return size;
-        }
-    }
-
-    /**
-     * Mock request limiter.
-     */
-    class MockRequestLimiter implements RequestLimiter<MockRequest> {
-        public void apply(MockRequest request) {
-        }
-    }
-
-    /**
-     * Counter based limiter.
-     */
-    static class CounterLimiter implements RateLimiter {
-        final int limit;
-        int count;
-
-        public CounterLimiter(int limit) {
-            this.limit = limit;
-            this.count = 0;
-        }
-
-        @Override
-        public boolean acquire(int permits) {
-            if (++count > limit) {
-                return false;
-            }
-            return true;
-        }
-    }
-
-    /**
-     * Mock hard request limiter.
-     */
-    class MockHardRequestLimiter implements RequestLimiter<MockRequest> {
-
-        RequestLimiter<MockRequest> limiter;
-        int limitHitCount;
-
-        MockHardRequestLimiter(int limit) {
-            this(GuavaRateLimiter.of(limit));
-        }
-
-        MockHardRequestLimiter(RateLimiter limiter) {
-            this.limiter = new ComposableRequestLimiter<MockRequest>(
-                limiter,
-                new OverlimitFunction<MockRequest>() {
-                    public void apply(MockRequest request) throws OverCapacityException {
-                        limitHitCount++;
-                        throw new OverCapacityException("Limit exceeded");
-                    }
-                },
-                new CostFunction<MockRequest>() {
-                    public int apply(MockRequest request) {
-                        return request.getSize();
-                    }
-                },
-                NullStatsLogger.INSTANCE);
-        }
-
-        @Override
-        public void apply(MockRequest op) throws OverCapacityException {
-            limiter.apply(op);
-        }
-
-        public int getLimitHitCount() {
-            return limitHitCount;
-        }
-    }
-
-    /**
-     * Mock soft request limiter.
-     */
-    class MockSoftRequestLimiter implements RequestLimiter<MockRequest> {
-
-        RequestLimiter<MockRequest> limiter;
-        int limitHitCount;
-
-        MockSoftRequestLimiter(int limit) {
-            this(GuavaRateLimiter.of(limit));
-        }
-
-        MockSoftRequestLimiter(RateLimiter limiter) {
-            this.limiter = new ComposableRequestLimiter<MockRequest>(
-                limiter,
-                new OverlimitFunction<MockRequest>() {
-                    public void apply(MockRequest request) throws OverCapacityException {
-                        limitHitCount++;
-                    }
-                },
-                new CostFunction<MockRequest>() {
-                    public int apply(MockRequest request) {
-                        return request.getSize();
-                    }
-                },
-                NullStatsLogger.INSTANCE);
-        }
-
-        @Override
-        public void apply(MockRequest op) throws OverCapacityException {
-            limiter.apply(op);
-        }
-
-        public int getLimitHitCount() {
-            return limitHitCount;
-        }
-    }
-
-    @Test(timeout = 60000)
-    public void testDynamicLimiter() throws Exception {
-        final AtomicInteger id = new AtomicInteger(0);
-        final DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(
-                new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
-        DynamicRequestLimiter<MockRequest> limiter = new DynamicRequestLimiter<MockRequest>(
-                dynConf, NullStatsLogger.INSTANCE, new SettableFeature("", 0)) {
-            @Override
-            public RequestLimiter<MockRequest> build() {
-                id.getAndIncrement();
-                return new MockRequestLimiter();
-            }
-        };
-        limiter.initialize();
-        assertEquals(1, id.get());
-        dynConf.setProperty("test1", 1);
-        assertEquals(2, id.get());
-        dynConf.setProperty("test2", 2);
-        assertEquals(3, id.get());
-    }
-
-    @Test(timeout = 60000)
-    public void testDynamicLimiterWithDisabledFeature() throws Exception {
-        final DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(
-                new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
-        final MockSoftRequestLimiter rateLimiter = new MockSoftRequestLimiter(0);
-        final SettableFeature disabledFeature = new SettableFeature("", 0);
-        DynamicRequestLimiter<MockRequest> limiter = new DynamicRequestLimiter<MockRequest>(
-                dynConf, NullStatsLogger.INSTANCE, disabledFeature) {
-            @Override
-            public RequestLimiter<MockRequest> build() {
-                return rateLimiter;
-            }
-        };
-        limiter.initialize();
-        assertEquals(0, rateLimiter.getLimitHitCount());
-
-        // Not disabled, rate limiter was invoked
-        limiter.apply(new MockRequest(Integer.MAX_VALUE));
-        assertEquals(1, rateLimiter.getLimitHitCount());
-
-        // Disabled, rate limiter not invoked
-        disabledFeature.set(1);
-        limiter.apply(new MockRequest(Integer.MAX_VALUE));
-        assertEquals(1, rateLimiter.getLimitHitCount());
-    }
-
-    @Test(timeout = 60000)
-    public void testDynamicLimiterWithException() throws Exception {
-        final AtomicInteger id = new AtomicInteger(0);
-        final DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(
-                new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
-        DynamicRequestLimiter<MockRequest> limiter = new DynamicRequestLimiter<MockRequest>(
-                dynConf, NullStatsLogger.INSTANCE, new SettableFeature("", 0)) {
-            @Override
-            public RequestLimiter<MockRequest> build() {
-                if (id.incrementAndGet() >= 2) {
-                    throw new RuntimeException("exception in dynamic limiter build()");
-                }
-                return new MockRequestLimiter();
-            }
-        };
-        limiter.initialize();
-        assertEquals(1, id.get());
-        try {
-            dynConf.setProperty("test1", 1);
-            fail("should have thrown on config failure");
-        } catch (RuntimeException ex) {
-        }
-        assertEquals(2, id.get());
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceRequestLimiter() throws Exception {
-        MockHardRequestLimiter limiter = new MockHardRequestLimiter(new CounterLimiter(1));
-        limiter.apply(new MockRequest());
-        try {
-            limiter.apply(new MockRequest());
-        } catch (OverCapacityException ex) {
-        }
-        assertEquals(1, limiter.getLimitHitCount());
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceRequestLimiterWithDefaultRate() throws Exception {
-        MockHardRequestLimiter limiter = new MockHardRequestLimiter(-1);
-        limiter.apply(new MockRequest(Integer.MAX_VALUE));
-        limiter.apply(new MockRequest(Integer.MAX_VALUE));
-        assertEquals(0, limiter.getLimitHitCount());
-    }
-
-    @Test(timeout = 60000)
-    public void testServiceRequestLimiterWithZeroRate() throws Exception {
-        MockHardRequestLimiter limiter = new MockHardRequestLimiter(0);
-        try {
-            limiter.apply(new MockRequest(1));
-            fail("should have failed with overcap");
-        } catch (OverCapacityException ex) {
-        }
-        assertEquals(1, limiter.getLimitHitCount());
-    }
-
-    @Test(timeout = 60000)
-    public void testChainedServiceRequestLimiter() throws Exception {
-        MockSoftRequestLimiter softLimiter = new MockSoftRequestLimiter(new CounterLimiter(1));
-        MockHardRequestLimiter hardLimiter = new MockHardRequestLimiter(new CounterLimiter(3));
-
-        RequestLimiter<MockRequest> limiter =
-                new ChainedRequestLimiter.Builder<MockRequest>()
-                .addLimiter(softLimiter)
-                .addLimiter(hardLimiter)
-                .build();
-
-        assertEquals(0, softLimiter.getLimitHitCount());
-        assertEquals(0, hardLimiter.getLimitHitCount());
-
-        limiter.apply(new MockRequest());
-        assertEquals(0, softLimiter.getLimitHitCount());
-        assertEquals(0, hardLimiter.getLimitHitCount());
-
-        limiter.apply(new MockRequest());
-        assertEquals(1, softLimiter.getLimitHitCount());
-        assertEquals(0, hardLimiter.getLimitHitCount());
-
-        limiter.apply(new MockRequest());
-        assertEquals(2, softLimiter.getLimitHitCount());
-        assertEquals(0, hardLimiter.getLimitHitCount());
-
-        try {
-            limiter.apply(new MockRequest());
-        } catch (OverCapacityException ex) {
-        }
-        assertEquals(3, softLimiter.getLimitHitCount());
-        assertEquals(1, hardLimiter.getLimitHitCount());
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java
deleted file mode 100644
index af7b7e5..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java
+++ /dev/null
@@ -1,52 +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 com.twitter.distributedlog.service.streamset;
-
-import static org.junit.Assert.assertEquals;
-
-import org.junit.Test;
-
-/**
- * Test Cases for {@link DelimiterStreamPartitionConverter}.
- */
-public class TestDelimiterStreamPartitionConverter {
-
-    @Test(timeout = 20000)
-    public void testNormalStream() throws Exception {
-        StreamPartitionConverter converter = new DelimiterStreamPartitionConverter();
-        assertEquals(new Partition("distributedlog-smoketest", 1),
-                converter.convert("distributedlog-smoketest_1"));
-        assertEquals(new Partition("distributedlog-smoketest-", 1),
-                converter.convert("distributedlog-smoketest-_1"));
-        assertEquals(new Partition("distributedlog-smoketest", 1),
-                converter.convert("distributedlog-smoketest_000001"));
-    }
-
-    private void assertIdentify(String streamName, StreamPartitionConverter converter) {
-        assertEquals(new Partition(streamName, 0), converter.convert(streamName));
-    }
-
-    @Test(timeout = 20000)
-    public void testUnknownStream() throws Exception {
-        StreamPartitionConverter converter = new DelimiterStreamPartitionConverter();
-        assertIdentify("test1", converter);
-        assertIdentify("test1-000001", converter);
-        assertIdentify("test1_test1_000001", converter);
-        assertIdentify("test1_test1", converter);
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.java
deleted file mode 100644
index d0c6660..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.java
+++ /dev/null
@@ -1,43 +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 com.twitter.distributedlog.service.streamset;
-
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertTrue;
-
-import org.junit.Test;
-
-/**
- * Test Case for {@link IdentityStreamPartitionConverter}.
- */
-public class TestIdentityStreamPartitionConverter {
-
-    @Test(timeout = 20000)
-    public void testIdentityConverter() {
-        String streamName = "test-identity-converter";
-
-        IdentityStreamPartitionConverter converter =
-                new IdentityStreamPartitionConverter();
-
-        Partition p0 = converter.convert(streamName);
-        assertEquals(new Partition(streamName, 0), p0);
-
-        Partition p1 = converter.convert(streamName);
-        assertTrue(p0 == p1);
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/streamset/TestPartitionMap.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/streamset/TestPartitionMap.java
deleted file mode 100644
index 529397e..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/streamset/TestPartitionMap.java
+++ /dev/null
@@ -1,68 +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 com.twitter.distributedlog.service.streamset;
-
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-
-import org.junit.Test;
-
-/**
- * Test {@link PartitionMap}.
- */
-public class TestPartitionMap {
-
-    @Test(timeout = 20000)
-    public void testAddPartitionNegativeMax() {
-        PartitionMap map = new PartitionMap();
-        for (int i = 0; i < 20; i++) {
-            assertTrue(map.addPartition(new Partition("test", i), -1));
-        }
-    }
-
-    @Test(timeout = 20000)
-    public void testAddPartitionMultipleTimes() {
-        PartitionMap map = new PartitionMap();
-        for (int i = 0; i < 20; i++) {
-            assertTrue(map.addPartition(new Partition("test", 0), 3));
-        }
-    }
-
-    @Test(timeout = 20000)
-    public void testAddPartition() {
-        PartitionMap map = new PartitionMap();
-        for (int i = 0; i < 3; i++) {
-            assertTrue(map.addPartition(new Partition("test", i), 3));
-        }
-        for (int i = 3; i < 20; i++) {
-            assertFalse(map.addPartition(new Partition("test", i), 3));
-        }
-    }
-
-    @Test(timeout = 20000)
-    public void testRemovePartition() {
-        PartitionMap map = new PartitionMap();
-        for (int i = 0; i < 3; i++) {
-            assertTrue(map.addPartition(new Partition("test", i), 3));
-        }
-        assertFalse(map.addPartition(new Partition("test", 3), 3));
-        assertFalse(map.removePartition(new Partition("test", 3)));
-        assertTrue(map.removePartition(new Partition("test", 0)));
-        assertTrue(map.addPartition(new Partition("test", 3), 3));
-    }
-}
diff --git a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/utils/TestServerUtils.java b/distributedlog-service/src/test/java/com/twitter/distributedlog/service/utils/TestServerUtils.java
deleted file mode 100644
index 776205c..0000000
--- a/distributedlog-service/src/test/java/com/twitter/distributedlog/service/utils/TestServerUtils.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 com.twitter.distributedlog.service.utils;
-
-import static org.junit.Assert.assertEquals;
-
-import java.net.InetAddress;
-import org.junit.Test;
-
-/**
- * Test Case for {@link ServerUtils}.
- */
-public class TestServerUtils {
-
-    @Test(timeout = 6000)
-    public void testGetLedgerAllocatorPoolName() throws Exception {
-        int region = 123;
-        int shard = 999;
-        String hostname = InetAddress.getLocalHost().getHostAddress();
-        assertEquals("allocator_0123_0000000999",
-            ServerUtils.getLedgerAllocatorPoolName(region, shard, false));
-        assertEquals("allocator_0123_" + hostname,
-            ServerUtils.getLedgerAllocatorPoolName(region, shard, true));
-    }
-
-}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/client/routing/LocalRoutingService.java b/distributedlog-service/src/test/java/org/apache/distributedlog/client/routing/LocalRoutingService.java
new file mode 100644
index 0000000..a9ddae5
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/client/routing/LocalRoutingService.java
@@ -0,0 +1,146 @@
+/**
+ * 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.client.routing;
+
+import com.google.common.collect.Sets;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.stats.StatsReceiver;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.LinkedHashSet;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.CopyOnWriteArrayList;
+
+/**
+ * A local routing service that used for testing.
+ */
+public class LocalRoutingService implements RoutingService {
+
+    public static Builder newBuilder() {
+        return new Builder();
+    }
+
+    /**
+     * Builder to build a local routing service for testing.
+     */
+    public static class Builder implements RoutingService.Builder {
+
+        private Builder() {}
+
+        @Override
+        public RoutingService.Builder statsReceiver(StatsReceiver statsReceiver) {
+            return this;
+        }
+
+        @Override
+        public LocalRoutingService build() {
+            return new LocalRoutingService();
+        }
+    }
+
+    private final Map<String, LinkedHashSet<SocketAddress>> localAddresses =
+            new HashMap<String, LinkedHashSet<SocketAddress>>();
+    private final CopyOnWriteArrayList<RoutingListener> listeners =
+            new CopyOnWriteArrayList<RoutingListener>();
+
+    boolean allowRetrySameHost = true;
+
+    @Override
+    public void startService() {
+        // nop
+    }
+
+    @Override
+    public void stopService() {
+        // nop
+    }
+
+    @Override
+    public synchronized Set<SocketAddress> getHosts() {
+        Set<SocketAddress> hosts = Sets.newHashSet();
+        for (LinkedHashSet<SocketAddress> addresses : localAddresses.values()) {
+            hosts.addAll(addresses);
+        }
+        return hosts;
+    }
+
+    @Override
+    public RoutingService registerListener(RoutingListener listener) {
+        listeners.add(listener);
+        return this;
+    }
+
+    @Override
+    public RoutingService unregisterListener(RoutingListener listener) {
+        listeners.remove(listener);
+        return this;
+    }
+
+    public LocalRoutingService setAllowRetrySameHost(boolean enabled) {
+        allowRetrySameHost = enabled;
+        return this;
+    }
+
+    public LocalRoutingService addHost(String stream, SocketAddress address) {
+        boolean notify = false;
+        synchronized (this) {
+            LinkedHashSet<SocketAddress> addresses = localAddresses.get(stream);
+            if (null == addresses) {
+                addresses = new LinkedHashSet<SocketAddress>();
+                localAddresses.put(stream, addresses);
+            }
+            if (addresses.add(address)) {
+                notify = true;
+            }
+        }
+        if (notify) {
+            for (RoutingListener listener : listeners) {
+                listener.onServerJoin(address);
+            }
+        }
+        return this;
+    }
+
+    @Override
+    public synchronized SocketAddress getHost(String key, RoutingContext rContext)
+            throws NoBrokersAvailableException {
+        LinkedHashSet<SocketAddress> addresses = localAddresses.get(key);
+
+        SocketAddress candidate = null;
+        if (null != addresses) {
+            for (SocketAddress host : addresses) {
+                if (rContext.isTriedHost(host) && !allowRetrySameHost) {
+                    continue;
+                } else {
+                    candidate = host;
+                    break;
+                }
+            }
+        }
+        if (null != candidate) {
+            return candidate;
+        }
+        throw new NoBrokersAvailableException("No host available");
+    }
+
+    @Override
+    public void removeHost(SocketAddress address, Throwable reason) {
+        // nop
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/DistributedLogServerTestCase.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/DistributedLogServerTestCase.java
new file mode 100644
index 0000000..58b5b2a
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/DistributedLogServerTestCase.java
@@ -0,0 +1,298 @@
+/**
+ * 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.service;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.base.Optional;
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.DLMTestUtil;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.client.DistributedLogClientImpl;
+import org.apache.distributedlog.client.resolver.DefaultRegionResolver;
+import org.apache.distributedlog.client.routing.LocalRoutingService;
+import org.apache.distributedlog.client.routing.RegionsRoutingService;
+import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
+import org.apache.distributedlog.service.stream.StreamManager;
+import org.apache.distributedlog.service.stream.StreamManagerImpl;
+import com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Duration;
+import java.net.SocketAddress;
+import java.net.URI;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.junit.After;
+import org.junit.AfterClass;
+import org.junit.Before;
+import org.junit.BeforeClass;
+
+/**
+ * Base test case for distributedlog servers.
+ */
+public abstract class DistributedLogServerTestCase {
+
+    protected static DistributedLogConfiguration conf =
+            new DistributedLogConfiguration().setLockTimeout(10)
+                    .setOutputBufferSize(0).setPeriodicFlushFrequencyMilliSeconds(10);
+    protected static DistributedLogConfiguration noAdHocConf =
+            new DistributedLogConfiguration().setLockTimeout(10).setCreateStreamIfNotExists(false)
+                    .setOutputBufferSize(0).setPeriodicFlushFrequencyMilliSeconds(10);
+    protected static DistributedLogCluster dlCluster;
+    protected static DistributedLogCluster noAdHocCluster;
+
+    /**
+     * A distributedlog client wrapper for testing.
+     */
+    protected static class DLClient {
+        public final LocalRoutingService routingService;
+        public DistributedLogClientBuilder dlClientBuilder;
+        public final DistributedLogClientImpl dlClient;
+
+        protected DLClient(String name,
+                           String streamNameRegex,
+                           Optional<String> serverSideRoutingFinagleName) {
+            routingService = LocalRoutingService.newBuilder().build();
+            dlClientBuilder = DistributedLogClientBuilder.newBuilder()
+                        .name(name)
+                        .clientId(ClientId$.MODULE$.apply(name))
+                        .routingService(routingService)
+                        .streamNameRegex(streamNameRegex)
+                        .handshakeWithClientInfo(true)
+                        .clientBuilder(ClientBuilder.get()
+                            .hostConnectionLimit(1)
+                            .connectionTimeout(Duration.fromSeconds(1))
+                            .requestTimeout(Duration.fromSeconds(60)));
+            if (serverSideRoutingFinagleName.isPresent()) {
+                dlClientBuilder =
+                        dlClientBuilder.serverRoutingServiceFinagleNameStr(serverSideRoutingFinagleName.get());
+            }
+            dlClient = (DistributedLogClientImpl) dlClientBuilder.build();
+        }
+
+        public void handshake() {
+            dlClient.handshake();
+        }
+
+        public void shutdown() {
+            dlClient.close();
+        }
+    }
+
+    /**
+     * A distributedlog client wrapper that talks to two regions.
+     */
+    protected static class TwoRegionDLClient {
+
+        public final LocalRoutingService localRoutingService;
+        public final LocalRoutingService remoteRoutingService;
+        public final DistributedLogClientBuilder dlClientBuilder;
+        public final DistributedLogClientImpl dlClient;
+
+        protected TwoRegionDLClient(String name, Map<SocketAddress, String> regionMap) {
+            localRoutingService = new LocalRoutingService();
+            remoteRoutingService = new LocalRoutingService();
+            RegionsRoutingService regionsRoutingService =
+                    RegionsRoutingService.of(new DefaultRegionResolver(regionMap),
+                            localRoutingService, remoteRoutingService);
+            dlClientBuilder = DistributedLogClientBuilder.newBuilder()
+                        .name(name)
+                        .clientId(ClientId$.MODULE$.apply(name))
+                        .routingService(regionsRoutingService)
+                        .streamNameRegex(".*")
+                        .handshakeWithClientInfo(true)
+                        .maxRedirects(2)
+                        .clientBuilder(ClientBuilder.get()
+                            .hostConnectionLimit(1)
+                            .connectionTimeout(Duration.fromSeconds(1))
+                            .requestTimeout(Duration.fromSeconds(10)));
+            dlClient = (DistributedLogClientImpl) dlClientBuilder.build();
+        }
+
+        public void shutdown() {
+            dlClient.close();
+        }
+    }
+
+    private final boolean clientSideRouting;
+    protected DLServer dlServer;
+    protected DLClient dlClient;
+    protected DLServer noAdHocServer;
+    protected DLClient noAdHocClient;
+
+    public static DistributedLogCluster createCluster(DistributedLogConfiguration conf) throws Exception {
+        return DistributedLogCluster.newBuilder()
+            .numBookies(3)
+            .shouldStartZK(true)
+            .zkServers("127.0.0.1")
+            .shouldStartProxy(false)
+            .dlConf(conf)
+            .bkConf(DLMTestUtil.loadTestBkConf())
+            .build();
+    }
+
+    @BeforeClass
+    public static void setupCluster() throws Exception {
+        dlCluster = createCluster(conf);
+        dlCluster.start();
+    }
+
+    public void setupNoAdHocCluster() throws Exception {
+        noAdHocCluster = createCluster(noAdHocConf);
+        noAdHocCluster.start();
+        noAdHocServer = new DLServer(noAdHocConf, noAdHocCluster.getUri(), 7002, false);
+        Optional<String> serverSideRoutingFinagleName = Optional.absent();
+        if (!clientSideRouting) {
+            serverSideRoutingFinagleName =
+                    Optional.of("inet!" + DLSocketAddress.toString(noAdHocServer.getAddress()));
+        }
+        noAdHocClient = createDistributedLogClient("no-ad-hoc-client", serverSideRoutingFinagleName);
+    }
+
+    public void tearDownNoAdHocCluster() throws Exception {
+        if (null != noAdHocClient) {
+            noAdHocClient.shutdown();
+        }
+        if (null != noAdHocServer) {
+            noAdHocServer.shutdown();
+        }
+    }
+
+    @AfterClass
+    public static void teardownCluster() throws Exception {
+        if (null != dlCluster) {
+            dlCluster.stop();
+        }
+        if (null != noAdHocCluster) {
+            noAdHocCluster.stop();
+        }
+    }
+
+    protected static URI getUri() {
+        return dlCluster.getUri();
+    }
+
+    protected DistributedLogServerTestCase(boolean clientSideRouting) {
+        this.clientSideRouting = clientSideRouting;
+    }
+
+    @Before
+    public void setup() throws Exception {
+        dlServer = createDistributedLogServer(7001);
+        Optional<String> serverSideRoutingFinagleName = Optional.absent();
+        if (!clientSideRouting) {
+            serverSideRoutingFinagleName =
+                    Optional.of("inet!" + DLSocketAddress.toString(dlServer.getAddress()));
+        }
+        dlClient = createDistributedLogClient("test", serverSideRoutingFinagleName);
+    }
+
+    @After
+    public void teardown() throws Exception {
+        if (null != dlClient) {
+            dlClient.shutdown();
+        }
+        if (null != dlServer) {
+            dlServer.shutdown();
+        }
+    }
+
+    protected DLServer createDistributedLogServer(int port) throws Exception {
+        return new DLServer(conf, dlCluster.getUri(), port, false);
+    }
+
+    protected DLServer createDistributedLogServer(DistributedLogConfiguration conf, int port)
+            throws Exception {
+        return new DLServer(conf, dlCluster.getUri(), port, false);
+    }
+
+    protected DLClient createDistributedLogClient(String clientName,
+                                                  Optional<String> serverSideRoutingFinagleName)
+            throws Exception {
+        return createDistributedLogClient(clientName, ".*", serverSideRoutingFinagleName);
+    }
+
+    protected DLClient createDistributedLogClient(String clientName,
+                                                  String streamNameRegex,
+                                                  Optional<String> serverSideRoutingFinagleName)
+            throws Exception {
+        return new DLClient(clientName, streamNameRegex, serverSideRoutingFinagleName);
+    }
+
+    protected TwoRegionDLClient createTwoRegionDLClient(String clientName,
+                                                        Map<SocketAddress, String> regionMap)
+            throws Exception {
+        return new TwoRegionDLClient(clientName, regionMap);
+    }
+
+    protected static void checkStreams(int numExpectedStreams, DLServer dlServer) {
+        StreamManager streamManager = dlServer.dlServer.getKey().getStreamManager();
+        assertEquals(numExpectedStreams, streamManager.numCached());
+        assertEquals(numExpectedStreams, streamManager.numAcquired());
+    }
+
+    protected static void checkStreams(Set<String> streams, DLServer dlServer) {
+        StreamManagerImpl streamManager = (StreamManagerImpl) dlServer.dlServer.getKey().getStreamManager();
+        Set<String> cachedStreams = streamManager.getCachedStreams().keySet();
+        Set<String> acquiredStreams = streamManager.getAcquiredStreams().keySet();
+
+        assertEquals(streams.size(), cachedStreams.size());
+        assertEquals(streams.size(), acquiredStreams.size());
+        assertTrue(Sets.difference(streams, cachedStreams).isEmpty());
+        assertTrue(Sets.difference(streams, acquiredStreams).isEmpty());
+    }
+
+    protected static void checkStream(String name, DLClient dlClient, DLServer dlServer,
+                                      int expectedNumProxiesInClient, int expectedClientCacheSize,
+                                      int expectedServerCacheSize, boolean existedInServer, boolean existedInClient) {
+        Map<SocketAddress, Set<String>> distribution = dlClient.dlClient.getStreamOwnershipDistribution();
+        assertEquals(expectedNumProxiesInClient, distribution.size());
+
+        if (expectedNumProxiesInClient > 0) {
+            Map.Entry<SocketAddress, Set<String>> localEntry =
+                    distribution.entrySet().iterator().next();
+            assertEquals(dlServer.getAddress(), localEntry.getKey());
+            assertEquals(expectedClientCacheSize, localEntry.getValue().size());
+            assertEquals(existedInClient, localEntry.getValue().contains(name));
+        }
+
+        StreamManagerImpl streamManager = (StreamManagerImpl) dlServer.dlServer.getKey().getStreamManager();
+        Set<String> cachedStreams = streamManager.getCachedStreams().keySet();
+        Set<String> acquiredStreams = streamManager.getCachedStreams().keySet();
+
+        assertEquals(expectedServerCacheSize, cachedStreams.size());
+        assertEquals(existedInServer, cachedStreams.contains(name));
+        assertEquals(expectedServerCacheSize, acquiredStreams.size());
+        assertEquals(existedInServer, acquiredStreams.contains(name));
+    }
+
+    protected static Map<SocketAddress, Set<String>> getStreamOwnershipDistribution(DLClient dlClient) {
+        return dlClient.dlClient.getStreamOwnershipDistribution();
+    }
+
+    protected static Set<String> getAllStreamsFromDistribution(Map<SocketAddress, Set<String>> distribution) {
+        Set<String> allStreams = new HashSet<String>();
+        for (Map.Entry<SocketAddress, Set<String>> entry : distribution.entrySet()) {
+            allStreams.addAll(entry.getValue());
+        }
+        return allStreams;
+    }
+
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java
new file mode 100644
index 0000000..29a3617
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerBase.java
@@ -0,0 +1,720 @@
+/**
+ * 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.service;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.apache.distributedlog.LogRecord.MAX_LOGRECORD_SIZE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.base.Optional;
+import org.apache.distributedlog.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.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.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.service.stream.StreamManagerImpl;
+import org.apache.distributedlog.thrift.AccessControlEntry;
+import org.apache.distributedlog.thrift.service.BulkWriteResponse;
+import org.apache.distributedlog.thrift.service.HeartbeatOptions;
+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 com.twitter.finagle.builder.ClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Await;
+import com.twitter.util.Duration;
+import com.twitter.util.Future;
+import com.twitter.util.Futures;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import org.junit.Ignore;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Case for {@link DistributedLogServer}.
+ */
+public abstract class TestDistributedLogServerBase extends DistributedLogServerTestCase {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestDistributedLogServerBase.class);
+
+    @Rule
+    public TestName testName = new TestName();
+
+    protected TestDistributedLogServerBase(boolean clientSideRouting) {
+        super(clientSideRouting);
+    }
+
+    /**
+     * {@link https://issues.apache.org/jira/browse/DL-27}.
+     */
+    @DistributedLogAnnotations.FlakyTest
+    @Ignore
+    @Test(timeout = 60000)
+    public void testBasicWrite() throws Exception {
+        String name = "dlserver-basic-write";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        for (long i = 1; i <= 10; i++) {
+            logger.debug("Write entry {} to stream {}.", i, name);
+            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())));
+        }
+
+        HeartbeatOptions hbOptions = new HeartbeatOptions();
+        hbOptions.setSendHeartBeatToReader(true);
+        // make sure the first log segment of each stream created
+        FutureUtils.result(dlClient.dlClient.heartbeat(name));
+
+        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
+        LogReader reader = dlm.getInputStream(1);
+        int numRead = 0;
+        LogRecord r = reader.readNext(false);
+        while (null != r) {
+            ++numRead;
+            int i = Integer.parseInt(new String(r.getPayload()));
+            assertEquals(numRead, i);
+            r = reader.readNext(false);
+        }
+        assertEquals(10, numRead);
+        reader.close();
+        dlm.close();
+    }
+
+    /**
+     * Sanity check to make sure both checksum flag values work.
+     */
+    @Test(timeout = 60000)
+    public void testChecksumFlag() throws Exception {
+        String name = "testChecksumFlag";
+        LocalRoutingService routingService = LocalRoutingService.newBuilder().build();
+        routingService.addHost(name, dlServer.getAddress());
+        DistributedLogClientBuilder dlClientBuilder = DistributedLogClientBuilder.newBuilder()
+            .name(name)
+            .clientId(ClientId$.MODULE$.apply("test"))
+            .routingService(routingService)
+            .handshakeWithClientInfo(true)
+            .clientBuilder(ClientBuilder.get()
+                .hostConnectionLimit(1)
+                .connectionTimeout(Duration.fromSeconds(1))
+                .requestTimeout(Duration.fromSeconds(60)))
+            .checksum(false);
+        DistributedLogClient dlClient = dlClientBuilder.build();
+        Await.result(dlClient.write(name, ByteBuffer.wrap(("1").getBytes())));
+        dlClient.close();
+
+        dlClient = dlClientBuilder.checksum(true).build();
+        Await.result(dlClient.write(name, ByteBuffer.wrap(("2").getBytes())));
+        dlClient.close();
+    }
+
+    private void runSimpleBulkWriteTest(int writeCount) throws Exception {
+        String name = String.format("dlserver-bulk-write-%d", writeCount);
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount);
+        for (long i = 1; i <= writeCount; i++) {
+            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
+        }
+
+        logger.debug("Write {} entries to stream {}.", writeCount, name);
+        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
+        assertEquals(futures.size(), writeCount);
+        for (Future<DLSN> future : futures) {
+            // No throw == pass.
+            DLSN dlsn = Await.result(future, Duration.fromSeconds(10));
+        }
+
+        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
+        LogReader reader = dlm.getInputStream(1);
+        int numRead = 0;
+        LogRecord r = reader.readNext(false);
+        while (null != r) {
+            int i = Integer.parseInt(new String(r.getPayload()));
+            assertEquals(numRead + 1, i);
+            ++numRead;
+            r = reader.readNext(false);
+        }
+        assertEquals(writeCount, numRead);
+        reader.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWrite() throws Exception {
+        runSimpleBulkWriteTest(100);
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWriteSingleWrite() throws Exception {
+        runSimpleBulkWriteTest(1);
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWriteEmptyList() throws Exception {
+        String name = String.format("dlserver-bulk-write-%d", 0);
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>();
+        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
+
+        assertEquals(0, futures.size());
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWriteNullArg() throws Exception {
+
+        String name = String.format("dlserver-bulk-write-%s", "null");
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>();
+        writes.add(null);
+
+        try {
+            dlClient.dlClient.writeBulk(name, writes);
+            fail("should not have succeeded");
+        } catch (NullPointerException npe) {
+            // expected
+            logger.info("Expected to catch NullPointException.");
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWriteEmptyBuffer() throws Exception {
+        String name = String.format("dlserver-bulk-write-%s", "empty");
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>();
+        writes.add(ByteBuffer.wrap(("").getBytes()));
+        writes.add(ByteBuffer.wrap(("").getBytes()));
+        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
+        assertEquals(2, futures.size());
+        for (Future<DLSN> future : futures) {
+            // No throw == pass
+            DLSN dlsn = Await.result(future, Duration.fromSeconds(10));
+        }
+    }
+
+    void failDueToWrongException(Exception ex) {
+        logger.info("testBulkWritePartialFailure: ", ex);
+        fail(String.format("failed with wrong exception %s", ex.getClass().getName()));
+    }
+
+    int validateAllFailedAsCancelled(List<Future<DLSN>> futures, int start, int finish) {
+        int failed = 0;
+        for (int i = start; i < finish; i++) {
+            Future<DLSN> future = futures.get(i);
+            try {
+                Await.result(future, Duration.fromSeconds(10));
+                fail("future should have failed!");
+            } catch (DLException cre) {
+                ++failed;
+            } catch (Exception ex) {
+                failDueToWrongException(ex);
+            }
+        }
+        return failed;
+    }
+
+    void validateFailedAsLogRecordTooLong(Future<DLSN> future) {
+        try {
+            Await.result(future, Duration.fromSeconds(10));
+            fail("should have failed");
+        } catch (DLException dle) {
+            assertEquals(StatusCode.TOO_LARGE_RECORD, dle.getCode());
+        } catch (Exception ex) {
+            failDueToWrongException(ex);
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWritePartialFailure() throws Exception {
+        String name = String.format("dlserver-bulk-write-%s", "partial-failure");
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        final int writeCount = 100;
+
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount * 2 + 1);
+        for (long i = 1; i <= writeCount; i++) {
+            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
+        }
+        // Too big, will cause partial failure.
+        ByteBuffer buf = ByteBuffer.allocate(MAX_LOGRECORD_SIZE + 1);
+        writes.add(buf);
+        for (long i = 1; i <= writeCount; i++) {
+            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
+        }
+
+        // Count succeeded.
+        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
+        int succeeded = 0;
+        for (int i = 0; i < writeCount; i++) {
+            Future<DLSN> future = futures.get(i);
+            try {
+                Await.result(future, Duration.fromSeconds(10));
+                ++succeeded;
+            } catch (Exception ex) {
+                failDueToWrongException(ex);
+            }
+        }
+
+        validateFailedAsLogRecordTooLong(futures.get(writeCount));
+        FutureUtils.result(Futures.collect(futures.subList(writeCount + 1, 2 * writeCount + 1)));
+        assertEquals(writeCount, succeeded);
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWriteTotalFailureFirstWriteFailed() throws Exception {
+        String name = String.format("dlserver-bulk-write-%s", "first-write-failed");
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        final int writeCount = 100;
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount + 1);
+        ByteBuffer buf = ByteBuffer.allocate(MAX_LOGRECORD_SIZE + 1);
+        writes.add(buf);
+        for (long i = 1; i <= writeCount; i++) {
+            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
+        }
+
+        List<Future<DLSN>> futures = dlClient.dlClient.writeBulk(name, writes);
+        validateFailedAsLogRecordTooLong(futures.get(0));
+        FutureUtils.result(Futures.collect(futures.subList(1, writeCount + 1)));
+    }
+
+    @Test(timeout = 60000)
+    public void testBulkWriteTotalFailureLostLock() throws Exception {
+        String name = String.format("dlserver-bulk-write-%s", "lost-lock");
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        final int writeCount = 8;
+        List<ByteBuffer> writes = new ArrayList<ByteBuffer>(writeCount + 1);
+        ByteBuffer buf = ByteBuffer.allocate(8);
+        writes.add(buf);
+        for (long i = 1; i <= writeCount; i++) {
+            writes.add(ByteBuffer.wrap(("" + i).getBytes()));
+        }
+        // Warm it up with a write.
+        Await.result(dlClient.dlClient.write(name, ByteBuffer.allocate(8)));
+
+        // Failpoint a lost lock, make sure the failure gets promoted to an operation failure.
+        DistributedLogServiceImpl svcImpl = (DistributedLogServiceImpl) dlServer.dlServer.getLeft();
+        try {
+            FailpointUtils.setFailpoint(
+                FailpointUtils.FailPointName.FP_WriteInternalLostLock,
+                FailpointUtils.FailPointActions.FailPointAction_Default
+            );
+            Future<BulkWriteResponse> futures = svcImpl.writeBulkWithContext(name, writes, new WriteContext());
+            assertEquals(StatusCode.LOCKING_EXCEPTION, Await.result(futures).header.code);
+        } finally {
+            FailpointUtils.removeFailpoint(
+                FailpointUtils.FailPointName.FP_WriteInternalLostLock
+            );
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testHeartbeat() throws Exception {
+        String name = "dlserver-heartbeat";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        for (long i = 1; i <= 10; i++) {
+            logger.debug("Send heartbeat {} to stream {}.", i, name);
+            dlClient.dlClient.check(name).get();
+        }
+
+        logger.debug("Write entry one to stream {}.", name);
+        dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes())).get();
+
+        Thread.sleep(1000);
+
+        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
+        LogReader reader = dlm.getInputStream(DLSN.InitialDLSN);
+        int numRead = 0;
+        // eid=0 => control records
+        // other 9 heartbeats will not trigger writing any control records.
+        // eid=1 => user entry
+        long startEntryId = 1;
+        LogRecordWithDLSN r = reader.readNext(false);
+        while (null != r) {
+            int i = Integer.parseInt(new String(r.getPayload()));
+            assertEquals(numRead + 1, i);
+            assertEquals(r.getDlsn().compareTo(new DLSN(1, startEntryId, 0)), 0);
+            ++numRead;
+            ++startEntryId;
+            r = reader.readNext(false);
+        }
+        assertEquals(1, numRead);
+    }
+
+    @Test(timeout = 60000)
+    public void testFenceWrite() throws Exception {
+        String name = "dlserver-fence-write";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        for (long i = 1; i <= 10; i++) {
+            logger.debug("Write entry {} to stream {}.", i, name);
+            dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())).get();
+        }
+
+        Thread.sleep(1000);
+
+        logger.info("Fencing stream {}.", name);
+        DLMTestUtil.fenceStream(conf, getUri(), name);
+        logger.info("Fenced stream {}.", name);
+
+        for (long i = 11; i <= 20; i++) {
+            logger.debug("Write entry {} to stream {}.", i, name);
+            dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())).get();
+        }
+
+        DistributedLogManager dlm = DLMTestUtil.createNewDLM(name, conf, getUri());
+        LogReader reader = dlm.getInputStream(1);
+        int numRead = 0;
+        LogRecord r = reader.readNext(false);
+        while (null != r) {
+            int i = Integer.parseInt(new String(r.getPayload()));
+            assertEquals(numRead + 1, i);
+            ++numRead;
+            r = reader.readNext(false);
+        }
+        assertEquals(20, numRead);
+        reader.close();
+        dlm.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testDeleteStream() throws Exception {
+        String name = "dlserver-delete-stream";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        long txid = 101;
+        for (long i = 1; i <= 10; i++) {
+            long curTxId = txid++;
+            logger.debug("Write entry {} to stream {}.", curTxId, name);
+            dlClient.dlClient.write(name,
+                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
+        }
+
+        checkStream(1, 1, 1, name, dlServer.getAddress(), true, true);
+
+        dlClient.dlClient.delete(name).get();
+
+        checkStream(0, 0, 0, name, dlServer.getAddress(), false, false);
+
+        Thread.sleep(1000);
+
+        DistributedLogManager dlm101 = DLMTestUtil.createNewDLM(name, conf, getUri());
+        AsyncLogReader reader101 = FutureUtils.result(dlm101.openAsyncLogReader(DLSN.InitialDLSN));
+        try {
+            FutureUtils.result(reader101.readNext());
+            fail("Should fail with LogNotFoundException since the stream is deleted");
+        } catch (LogNotFoundException lnfe) {
+            // expected
+        }
+        FutureUtils.result(reader101.asyncClose());
+        dlm101.close();
+
+        txid = 201;
+        for (long i = 1; i <= 10; i++) {
+            long curTxId = txid++;
+            logger.debug("Write entry {} to stream {}.", curTxId, name);
+            DLSN dlsn = dlClient.dlClient.write(name,
+                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
+        }
+        Thread.sleep(1000);
+
+        DistributedLogManager dlm201 = DLMTestUtil.createNewDLM(name, conf, getUri());
+        LogReader reader201 = dlm201.getInputStream(1);
+        int numRead = 0;
+        int curTxId = 201;
+        LogRecord r = reader201.readNext(false);
+        while (null != r) {
+            int i = Integer.parseInt(new String(r.getPayload()));
+            assertEquals(curTxId++, i);
+            ++numRead;
+            r = reader201.readNext(false);
+        }
+        assertEquals(10, numRead);
+        reader201.close();
+        dlm201.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateStream() throws Exception {
+        try {
+            setupNoAdHocCluster();
+            final String name = "dlserver-create-stream";
+
+            noAdHocClient.routingService.addHost("dlserver-create-stream", noAdHocServer.getAddress());
+            assertFalse(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
+            assertTrue(Await.ready(noAdHocClient.dlClient.create(name)).isReturn());
+
+            long txid = 101;
+            for (long i = 1; i <= 10; i++) {
+                long curTxId = txid++;
+                logger.debug("Write entry {} to stream {}.", curTxId, name);
+                noAdHocClient.dlClient.write(name,
+                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
+            }
+
+            assertTrue(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
+        } finally {
+            tearDownNoAdHocCluster();
+        }
+    }
+
+    /**
+     * This tests that create has touch like behavior in that trying to create the stream twice, simply does nothing.
+     */
+    @Test(timeout = 60000)
+    public void testCreateStreamTwice() throws Exception {
+        try {
+            setupNoAdHocCluster();
+            final String name = "dlserver-create-stream-twice";
+
+            noAdHocClient.routingService.addHost("dlserver-create-stream-twice", noAdHocServer.getAddress());
+            assertFalse(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
+            assertTrue(Await.ready(noAdHocClient.dlClient.create(name)).isReturn());
+
+            long txid = 101;
+            for (long i = 1; i <= 10; i++) {
+                long curTxId = txid++;
+                logger.debug("Write entry {} to stream {}.", curTxId, name);
+                noAdHocClient.dlClient.write(name,
+                    ByteBuffer.wrap(("" + curTxId).getBytes())).get();
+            }
+
+            assertTrue(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
+
+            // create again
+            assertTrue(Await.ready(noAdHocClient.dlClient.create(name)).isReturn());
+            assertTrue(noAdHocServer.dlServer.getKey().getStreamManager().isAcquired(name));
+        } finally {
+            tearDownNoAdHocCluster();
+        }
+    }
+
+
+
+    @Test(timeout = 60000)
+    public void testTruncateStream() throws Exception {
+        String name = "dlserver-truncate-stream";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        long txid = 1;
+        Map<Long, DLSN> txid2DLSN = new HashMap<Long, DLSN>();
+        for (int s = 1; s <= 2; s++) {
+            for (long i = 1; i <= 10; i++) {
+                long curTxId = txid++;
+                logger.debug("Write entry {} to stream {}.", curTxId, name);
+                DLSN dlsn = dlClient.dlClient.write(name,
+                        ByteBuffer.wrap(("" + curTxId).getBytes())).get();
+                txid2DLSN.put(curTxId, dlsn);
+            }
+            if (s == 1) {
+                dlClient.dlClient.release(name).get();
+            }
+        }
+
+        DLSN dlsnToDelete = txid2DLSN.get(11L);
+        dlClient.dlClient.truncate(name, dlsnToDelete).get();
+
+        DistributedLogManager readDLM = DLMTestUtil.createNewDLM(name, conf, getUri());
+        LogReader reader = readDLM.getInputStream(1);
+        int numRead = 0;
+        int curTxId = 11;
+        LogRecord r = reader.readNext(false);
+        while (null != r) {
+            int i = Integer.parseInt(new String(r.getPayload()));
+            assertEquals(curTxId++, i);
+            ++numRead;
+            r = reader.readNext(false);
+        }
+        assertEquals(10, numRead);
+        reader.close();
+        readDLM.close();
+    }
+
+    @Test(timeout = 60000)
+    public void testRequestDenied() throws Exception {
+        String name = "request-denied";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        AccessControlEntry ace = new AccessControlEntry();
+        ace.setDenyWrite(true);
+        ZooKeeperClient zkc = TestZooKeeperClientBuilder
+                .newBuilder()
+                .uri(getUri())
+                .connectionTimeoutMs(60000)
+                .sessionTimeoutMs(60000)
+                .build();
+        DistributedLogNamespace dlNamespace = dlServer.dlServer.getLeft().getDistributedLogNamespace();
+        BKDLConfig bkdlConfig = BKDLConfig.resolveDLConfig(zkc, getUri());
+        String zkPath = getUri().getPath() + "/" + bkdlConfig.getACLRootPath() + "/" + name;
+        ZKAccessControl accessControl = new ZKAccessControl(ace, zkPath);
+        accessControl.create(zkc);
+
+        AccessControlManager acm = dlNamespace.createAccessControlManager();
+        while (acm.allowWrite(name)) {
+            Thread.sleep(100);
+        }
+
+        try {
+            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
+            fail("Should fail with request denied exception");
+        } catch (DLException dle) {
+            assertEquals(StatusCode.REQUEST_DENIED, dle.getCode());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testNoneStreamNameRegex() throws Exception {
+        String streamNamePrefix = "none-stream-name-regex-";
+        int numStreams = 5;
+        Set<String> streams = new HashSet<String>();
+
+        for (int i = 0; i < numStreams; i++) {
+            streams.add(streamNamePrefix + i);
+        }
+        testStreamNameRegex(streams, ".*", streams);
+    }
+
+    @Test(timeout = 60000)
+    public void testStreamNameRegex() throws Exception {
+        String streamNamePrefix = "stream-name-regex-";
+        int numStreams = 5;
+        Set<String> streams = new HashSet<String>();
+        Set<String> expectedStreams = new HashSet<String>();
+        String streamNameRegex = streamNamePrefix + "1";
+
+        for (int i = 0; i < numStreams; i++) {
+            streams.add(streamNamePrefix + i);
+        }
+        expectedStreams.add(streamNamePrefix + "1");
+
+        testStreamNameRegex(streams, streamNameRegex, expectedStreams);
+    }
+
+    private void testStreamNameRegex(Set<String> streams, String streamNameRegex,
+                                     Set<String> expectedStreams)
+            throws Exception {
+        for (String streamName : streams) {
+            dlClient.routingService.addHost(streamName, dlServer.getAddress());
+            Await.result(dlClient.dlClient.write(streamName,
+                    ByteBuffer.wrap(streamName.getBytes(UTF_8))));
+        }
+
+        DLClient client = createDistributedLogClient(
+                "test-stream-name-regex",
+                streamNameRegex,
+                Optional.<String>absent());
+        try {
+            client.routingService.addHost("unknown", dlServer.getAddress());
+            client.handshake();
+            Map<SocketAddress, Set<String>> distribution =
+                    client.dlClient.getStreamOwnershipDistribution();
+            assertEquals(1, distribution.size());
+            Set<String> cachedStreams = distribution.values().iterator().next();
+            assertNotNull(cachedStreams);
+            assertEquals(expectedStreams.size(), cachedStreams.size());
+
+            for (String streamName : cachedStreams) {
+                assertTrue(expectedStreams.contains(streamName));
+            }
+        } finally {
+            client.shutdown();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testReleaseStream() throws Exception {
+        String name = "dlserver-release-stream";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+
+        Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
+        checkStream(1, 1, 1, name, dlServer.getAddress(), true, true);
+
+        // release the stream
+        Await.result(dlClient.dlClient.release(name));
+        checkStream(0, 0, 0, name, dlServer.getAddress(), false, false);
+    }
+
+    protected void checkStream(int expectedNumProxiesInClient, int expectedClientCacheSize, int expectedServerCacheSize,
+                             String name, SocketAddress owner, boolean existedInServer, boolean existedInClient) {
+        Map<SocketAddress, Set<String>> distribution = dlClient.dlClient.getStreamOwnershipDistribution();
+        assertEquals(expectedNumProxiesInClient, distribution.size());
+
+        if (expectedNumProxiesInClient > 0) {
+            Map.Entry<SocketAddress, Set<String>> localEntry =
+                    distribution.entrySet().iterator().next();
+            assertEquals(owner, localEntry.getKey());
+            assertEquals(expectedClientCacheSize, localEntry.getValue().size());
+            assertEquals(existedInClient, localEntry.getValue().contains(name));
+        }
+
+
+        StreamManagerImpl streamManager = (StreamManagerImpl) dlServer.dlServer.getKey().getStreamManager();
+        Set<String> cachedStreams = streamManager.getCachedStreams().keySet();
+        Set<String> acquiredStreams = streamManager.getCachedStreams().keySet();
+
+        assertEquals(expectedServerCacheSize, cachedStreams.size());
+        assertEquals(existedInServer, cachedStreams.contains(name));
+        assertEquals(expectedServerCacheSize, acquiredStreams.size());
+        assertEquals(existedInServer, acquiredStreams.contains(name));
+    }
+
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerClientRouting.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerClientRouting.java
new file mode 100644
index 0000000..c7ae960
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerClientRouting.java
@@ -0,0 +1,58 @@
+/**
+ * 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.service;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.fail;
+
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.util.Await;
+import java.nio.ByteBuffer;
+import org.junit.Test;
+
+/**
+ * Test the server with client side routing.
+ */
+public class TestDistributedLogServerClientRouting extends TestDistributedLogServerBase {
+
+    public TestDistributedLogServerClientRouting() {
+        super(true);
+    }
+
+    @Test(timeout = 60000)
+    public void testAcceptNewStream() throws Exception {
+        String name = "dlserver-accept-new-stream";
+
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+        dlClient.routingService.setAllowRetrySameHost(false);
+
+        Await.result(dlClient.dlClient.setAcceptNewStream(false));
+
+        try {
+            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
+            fail("Should fail because the proxy couldn't accept new stream");
+        } catch (NoBrokersAvailableException nbae) {
+            // expected
+        }
+        checkStream(0, 0, 0, name, dlServer.getAddress(), false, false);
+
+        Await.result(dlServer.dlServer.getLeft().setAcceptNewStream(true));
+        Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
+        checkStream(1, 1, 1, name, dlServer.getAddress(), true, true);
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerServerRouting.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerServerRouting.java
new file mode 100644
index 0000000..12416a3
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogServerServerRouting.java
@@ -0,0 +1,28 @@
+/**
+ * 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.service;
+
+/**
+ * Test the server with client side routing.
+ */
+public class TestDistributedLogServerServerRouting extends TestDistributedLogServerBase {
+
+    public TestDistributedLogServerServerRouting() {
+        super(false);
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java
new file mode 100644
index 0000000..e5d75c2
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestDistributedLogService.java
@@ -0,0 +1,833 @@
+/**
+ * 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.service;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.TestDistributedLogBase;
+import org.apache.distributedlog.acl.DefaultAccessControlManager;
+import org.apache.distributedlog.client.routing.LocalRoutingService;
+import org.apache.distributedlog.exceptions.OwnershipAcquireFailedException;
+import org.apache.distributedlog.exceptions.StreamUnavailableException;
+import org.apache.distributedlog.service.config.NullStreamConfigProvider;
+import org.apache.distributedlog.service.config.ServerConfiguration;
+import org.apache.distributedlog.service.placement.EqualLoadAppraiser;
+import org.apache.distributedlog.service.stream.Stream;
+import org.apache.distributedlog.service.stream.StreamImpl;
+import org.apache.distributedlog.service.stream.StreamImpl.StreamStatus;
+import org.apache.distributedlog.service.stream.StreamManagerImpl;
+import org.apache.distributedlog.service.stream.WriteOp;
+import org.apache.distributedlog.service.streamset.DelimiterStreamPartitionConverter;
+import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import org.apache.distributedlog.thrift.service.HeartbeatOptions;
+import org.apache.distributedlog.thrift.service.StatusCode;
+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 org.apache.distributedlog.util.ProtocolUtils;
+import com.twitter.util.Await;
+import com.twitter.util.Future;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+import org.apache.bookkeeper.feature.SettableFeature;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.util.ReflectionUtils;
+import org.apache.commons.configuration.ConfigurationException;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Case for DistributedLog Service.
+ */
+public class TestDistributedLogService extends TestDistributedLogBase {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestDistributedLogService.class);
+
+    @Rule
+    public TestName testName = new TestName();
+
+    private ServerConfiguration serverConf;
+    private DistributedLogConfiguration dlConf;
+    private URI uri;
+    private final CountDownLatch latch = new CountDownLatch(1);
+    private DistributedLogServiceImpl service;
+
+    @Before
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+        dlConf = new DistributedLogConfiguration();
+        dlConf.addConfiguration(conf);
+        dlConf.setLockTimeout(0)
+                .setOutputBufferSize(0)
+                .setPeriodicFlushFrequencyMilliSeconds(10)
+                .setSchedulerShutdownTimeoutMs(100);
+        serverConf = newLocalServerConf();
+        uri = createDLMURI("/" + testName.getMethodName());
+        ensureURICreated(uri);
+        service = createService(serverConf, dlConf, latch);
+    }
+
+    @After
+    @Override
+    public void teardown() throws Exception {
+        if (null != service) {
+            service.shutdown();
+        }
+        super.teardown();
+    }
+
+    private DistributedLogConfiguration newLocalConf() {
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(dlConf);
+        return confLocal;
+    }
+
+    private ServerConfiguration newLocalServerConf() {
+        ServerConfiguration serverConf = new ServerConfiguration();
+        serverConf.loadConf(dlConf);
+        serverConf.setServerThreads(1);
+        return serverConf;
+    }
+
+    private DistributedLogServiceImpl createService(
+            ServerConfiguration serverConf,
+            DistributedLogConfiguration dlConf) throws Exception {
+        return createService(serverConf, dlConf, new CountDownLatch(1));
+    }
+
+    private DistributedLogServiceImpl createService(
+            ServerConfiguration serverConf,
+            DistributedLogConfiguration dlConf,
+            CountDownLatch latch) throws Exception {
+        // Build the stream partition converter
+        StreamPartitionConverter converter;
+        try {
+            converter = ReflectionUtils.newInstance(serverConf.getStreamPartitionConverterClass());
+        } catch (ConfigurationException e) {
+            logger.warn("Failed to load configured stream-to-partition converter. Fallback to use {}",
+                    IdentityStreamPartitionConverter.class.getName());
+            converter = new IdentityStreamPartitionConverter();
+        }
+        return new DistributedLogServiceImpl(
+            serverConf,
+            dlConf,
+            ConfUtils.getConstDynConf(dlConf),
+            new NullStreamConfigProvider(),
+            uri,
+            converter,
+            new LocalRoutingService(),
+            NullStatsLogger.INSTANCE,
+            NullStatsLogger.INSTANCE,
+            latch,
+            new EqualLoadAppraiser());
+    }
+
+    private StreamImpl createUnstartedStream(DistributedLogServiceImpl service,
+                                             String name) throws Exception {
+        StreamImpl stream = (StreamImpl) service.newStream(name);
+        stream.initialize();
+        return stream;
+    }
+
+    private ByteBuffer createRecord(long txid) {
+        return ByteBuffer.wrap(("record-" + txid).getBytes(UTF_8));
+    }
+
+    private WriteOp createWriteOp(DistributedLogServiceImpl service,
+                                  String streamName,
+                                  long txid) {
+        ByteBuffer data = createRecord(txid);
+        return service.newWriteOp(streamName, data, null);
+    }
+
+    @Test(timeout = 60000)
+    public void testAcquireStreams() throws Exception {
+        String streamName = testName.getMethodName();
+        StreamImpl s0 = createUnstartedStream(service, streamName);
+        ServerConfiguration serverConf1 = new ServerConfiguration();
+        serverConf1.addConfiguration(serverConf);
+        serverConf1.setServerPort(9999);
+        DistributedLogServiceImpl service1 = createService(serverConf1, dlConf);
+        StreamImpl s1 = createUnstartedStream(service1, streamName);
+
+        // create write ops
+        WriteOp op0 = createWriteOp(service, streamName, 0L);
+        s0.submit(op0);
+
+        WriteOp op1 = createWriteOp(service1, streamName, 1L);
+        s1.submit(op1);
+
+        // check pending size
+        assertEquals("Write Op 0 should be pending in service 0",
+                1, s0.numPendingOps());
+        assertEquals("Write Op 1 should be pending in service 1",
+                1, s1.numPendingOps());
+
+        // start acquiring s0
+        s0.start();
+        WriteResponse wr0 = Await.result(op0.result());
+        assertEquals("Op 0 should succeed",
+                StatusCode.SUCCESS, wr0.getHeader().getCode());
+        assertEquals("Service 0 should acquire stream",
+                StreamStatus.INITIALIZED, s0.getStatus());
+        assertNotNull(s0.getManager());
+        assertNotNull(s0.getWriter());
+        assertNull(s0.getLastException());
+
+        // start acquiring s1
+        s1.start();
+        WriteResponse wr1 = Await.result(op1.result());
+        assertEquals("Op 1 should fail",
+                StatusCode.FOUND, wr1.getHeader().getCode());
+        // the stream will be set to ERROR and then be closed.
+        assertTrue("Service 1 should be in unavailable state",
+                StreamStatus.isUnavailable(s1.getStatus()));
+        assertNotNull(s1.getManager());
+        assertNull(s1.getWriter());
+        assertNotNull(s1.getLastException());
+        assertTrue(s1.getLastException() instanceof OwnershipAcquireFailedException);
+
+        service1.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testAcquireStreamsWhenExceedMaxCachedPartitions() throws Exception {
+        String streamName = testName.getMethodName() + "_0000";
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(dlConf);
+        confLocal.setMaxCachedPartitionsPerProxy(1);
+
+        ServerConfiguration serverConfLocal = new ServerConfiguration();
+        serverConfLocal.addConfiguration(serverConf);
+        serverConfLocal.setStreamPartitionConverterClass(DelimiterStreamPartitionConverter.class);
+
+        DistributedLogServiceImpl serviceLocal = createService(serverConfLocal, confLocal);
+        Stream stream = serviceLocal.getLogWriter(streamName);
+
+        // stream is cached
+        assertNotNull(stream);
+        assertEquals(1, serviceLocal.getStreamManager().numCached());
+
+        // create write ops
+        WriteOp op0 = createWriteOp(service, streamName, 0L);
+        stream.submit(op0);
+        WriteResponse wr0 = Await.result(op0.result());
+        assertEquals("Op 0 should succeed",
+                StatusCode.SUCCESS, wr0.getHeader().getCode());
+        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
+
+        // should fail to acquire another partition
+        try {
+            serviceLocal.getLogWriter(testName.getMethodName() + "_0001");
+            fail("Should fail to acquire new streams");
+        } catch (StreamUnavailableException sue) {
+            // expected
+        }
+        assertEquals(1, serviceLocal.getStreamManager().numCached());
+        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
+
+        // should be able to acquire partitions from other streams
+        String anotherStreamName = testName.getMethodName() + "-another_0001";
+        Stream anotherStream = serviceLocal.getLogWriter(anotherStreamName);
+        assertNotNull(anotherStream);
+        assertEquals(2, serviceLocal.getStreamManager().numCached());
+
+        // create write ops
+        WriteOp op1 = createWriteOp(service, anotherStreamName, 0L);
+        anotherStream.submit(op1);
+        WriteResponse wr1 = Await.result(op1.result());
+        assertEquals("Op 1 should succeed",
+                StatusCode.SUCCESS, wr1.getHeader().getCode());
+        assertEquals(2, serviceLocal.getStreamManager().numAcquired());
+    }
+
+    @Test(timeout = 60000)
+    public void testAcquireStreamsWhenExceedMaxAcquiredPartitions() throws Exception {
+        String streamName = testName.getMethodName() + "_0000";
+
+        DistributedLogConfiguration confLocal = new DistributedLogConfiguration();
+        confLocal.addConfiguration(dlConf);
+        confLocal.setMaxCachedPartitionsPerProxy(-1);
+        confLocal.setMaxAcquiredPartitionsPerProxy(1);
+
+        ServerConfiguration serverConfLocal = new ServerConfiguration();
+        serverConfLocal.addConfiguration(serverConf);
+        serverConfLocal.setStreamPartitionConverterClass(DelimiterStreamPartitionConverter.class);
+
+        DistributedLogServiceImpl serviceLocal = createService(serverConfLocal, confLocal);
+        Stream stream = serviceLocal.getLogWriter(streamName);
+
+        // stream is cached
+        assertNotNull(stream);
+        assertEquals(1, serviceLocal.getStreamManager().numCached());
+
+        // create write ops
+        WriteOp op0 = createWriteOp(service, streamName, 0L);
+        stream.submit(op0);
+        WriteResponse wr0 = Await.result(op0.result());
+        assertEquals("Op 0 should succeed",
+                StatusCode.SUCCESS, wr0.getHeader().getCode());
+        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
+
+        // should be able to cache partitions from same stream
+        String anotherStreamName = testName.getMethodName() + "_0001";
+        Stream anotherStream = serviceLocal.getLogWriter(anotherStreamName);
+        assertNotNull(anotherStream);
+        assertEquals(2, serviceLocal.getStreamManager().numCached());
+
+        // create write ops
+        WriteOp op1 = createWriteOp(service, anotherStreamName, 0L);
+        anotherStream.submit(op1);
+        WriteResponse wr1 = Await.result(op1.result());
+        assertEquals("Op 1 should fail",
+                StatusCode.STREAM_UNAVAILABLE, wr1.getHeader().getCode());
+        assertEquals(1, serviceLocal.getStreamManager().numAcquired());
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseShouldErrorOutPendingOps() throws Exception {
+        String streamName = testName.getMethodName();
+        StreamImpl s = createUnstartedStream(service, streamName);
+
+        int numWrites = 10;
+        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>(numWrites);
+        for (int i = 0; i < numWrites; i++) {
+            WriteOp op = createWriteOp(service, streamName, i);
+            s.submit(op);
+            futureList.add(op.result());
+        }
+        assertEquals(numWrites, s.numPendingOps());
+        Await.result(s.requestClose("close stream"));
+        assertEquals("Stream " + streamName + " is set to " + StreamStatus.CLOSED,
+                StreamStatus.CLOSED, s.getStatus());
+        for (int i = 0; i < numWrites; i++) {
+            Future<WriteResponse> future = futureList.get(i);
+            WriteResponse wr = Await.result(future);
+            assertEquals("Pending op should fail with " + StatusCode.STREAM_UNAVAILABLE,
+                    StatusCode.STREAM_UNAVAILABLE, wr.getHeader().getCode());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseTwice() throws Exception {
+        String streamName = testName.getMethodName();
+        StreamImpl s = createUnstartedStream(service, streamName);
+
+        int numWrites = 10;
+        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>(numWrites);
+        for (int i = 0; i < numWrites; i++) {
+            WriteOp op = createWriteOp(service, streamName, i);
+            s.submit(op);
+            futureList.add(op.result());
+        }
+        assertEquals(numWrites, s.numPendingOps());
+
+        Future<Void> closeFuture0 = s.requestClose("close 0");
+        assertTrue("Stream " + streamName + " should be set to " + StreamStatus.CLOSING,
+                StreamStatus.CLOSING == s.getStatus()
+                    || StreamStatus.CLOSED == s.getStatus());
+        Future<Void> closeFuture1 = s.requestClose("close 1");
+        assertTrue("Stream " + streamName + " should be set to " + StreamStatus.CLOSING,
+                StreamStatus.CLOSING == s.getStatus()
+                    || StreamStatus.CLOSED == s.getStatus());
+
+        Await.result(closeFuture0);
+        assertEquals("Stream " + streamName + " should be set to " + StreamStatus.CLOSED,
+                StreamStatus.CLOSED, s.getStatus());
+        Await.result(closeFuture1);
+        assertEquals("Stream " + streamName + " should be set to " + StreamStatus.CLOSED,
+                StreamStatus.CLOSED, s.getStatus());
+
+        for (int i = 0; i < numWrites; i++) {
+            Future<WriteResponse> future = futureList.get(i);
+            WriteResponse wr = Await.result(future);
+            assertEquals("Pending op should fail with " + StatusCode.STREAM_UNAVAILABLE,
+                    StatusCode.STREAM_UNAVAILABLE, wr.getHeader().getCode());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testFailRequestsDuringClosing() throws Exception {
+        String streamName = testName.getMethodName();
+        StreamImpl s = createUnstartedStream(service, streamName);
+
+        Future<Void> closeFuture = s.requestClose("close");
+        assertTrue("Stream " + streamName + " should be set to " + StreamStatus.CLOSING,
+                StreamStatus.CLOSING == s.getStatus()
+                    || StreamStatus.CLOSED == s.getStatus());
+        WriteOp op1 = createWriteOp(service, streamName, 0L);
+        s.submit(op1);
+        WriteResponse response1 = Await.result(op1.result());
+        assertEquals("Op should fail with " + StatusCode.STREAM_UNAVAILABLE + " if it is closing",
+                StatusCode.STREAM_UNAVAILABLE, response1.getHeader().getCode());
+
+        Await.result(closeFuture);
+        assertEquals("Stream " + streamName + " should be set to " + StreamStatus.CLOSED,
+                StreamStatus.CLOSED, s.getStatus());
+        WriteOp op2 = createWriteOp(service, streamName, 1L);
+        s.submit(op2);
+        WriteResponse response2 = Await.result(op2.result());
+        assertEquals("Op should fail with " + StatusCode.STREAM_UNAVAILABLE + " if it is closed",
+                StatusCode.STREAM_UNAVAILABLE, response2.getHeader().getCode());
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceTimeout() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
+                .setImmediateFlushEnabled(false)
+                .setPeriodicFlushFrequencyMilliSeconds(0);
+        ServerConfiguration serverConfLocal = newLocalServerConf();
+        serverConfLocal.addConfiguration(serverConf);
+        serverConfLocal.setServiceTimeoutMs(200)
+                .setStreamProbationTimeoutMs(100);
+        String streamName = testName.getMethodName();
+        // create a new service with 200ms timeout
+        DistributedLogServiceImpl localService = createService(serverConfLocal, confLocal);
+        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
+
+        int numWrites = 10;
+        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>(numWrites);
+        for (int i = 0; i < numWrites; i++) {
+            futureList.add(localService.write(streamName, createRecord(i)));
+        }
+
+        assertTrue("Stream " + streamName + " should be cached",
+                streamManager.getCachedStreams().containsKey(streamName));
+
+        StreamImpl s = (StreamImpl) streamManager.getCachedStreams().get(streamName);
+        // the stream should be set CLOSING
+        while (StreamStatus.CLOSING != s.getStatus()
+            && StreamStatus.CLOSED != s.getStatus()) {
+            TimeUnit.MILLISECONDS.sleep(20);
+        }
+        assertNotNull("Writer should be initialized", s.getWriter());
+        assertNull("No exception should be thrown", s.getLastException());
+        Future<Void> closeFuture = s.getCloseFuture();
+        Await.result(closeFuture);
+        for (int i = 0; i < numWrites; i++) {
+            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,
+                    StatusCode.BK_TRANSMIT_ERROR == response.getHeader().getCode()
+                        || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
+                        || StatusCode.WRITE_CANCELLED_EXCEPTION == response.getHeader().getCode());
+        }
+
+        while (streamManager.getCachedStreams().containsKey(streamName)) {
+            TimeUnit.MILLISECONDS.sleep(20);
+        }
+
+        assertFalse("Stream should be removed from cache",
+                streamManager.getCachedStreams().containsKey(streamName));
+        assertFalse("Stream should be removed from acquired cache",
+                streamManager.getAcquiredStreams().containsKey(streamName));
+
+        localService.shutdown();
+    }
+
+    private DistributedLogServiceImpl createConfiguredLocalService() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setOutputBufferSize(0)
+                .setImmediateFlushEnabled(true)
+                .setPeriodicFlushFrequencyMilliSeconds(0);
+        return createService(serverConf, confLocal);
+    }
+
+    private ByteBuffer getTestDataBuffer() {
+        return ByteBuffer.wrap("test-data".getBytes());
+    }
+
+    @Test(timeout = 60000)
+    public void testNonDurableWrite() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
+                .setImmediateFlushEnabled(false)
+                .setPeriodicFlushFrequencyMilliSeconds(0)
+                .setDurableWriteEnabled(false);
+        ServerConfiguration serverConfLocal = new ServerConfiguration();
+        serverConfLocal.addConfiguration(serverConf);
+        serverConfLocal.enableDurableWrite(false);
+        serverConfLocal.setServiceTimeoutMs(Integer.MAX_VALUE)
+                .setStreamProbationTimeoutMs(Integer.MAX_VALUE);
+        String streamName = testName.getMethodName();
+        DistributedLogServiceImpl localService =
+                createService(serverConfLocal, confLocal);
+        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
+
+        int numWrites = 10;
+        List<Future<WriteResponse>> futureList = new ArrayList<Future<WriteResponse>>();
+        for (int i = 0; i < numWrites; i++) {
+            futureList.add(localService.write(streamName, createRecord(i)));
+        }
+        assertTrue("Stream " + streamName + " should be cached",
+                streamManager.getCachedStreams().containsKey(streamName));
+        List<WriteResponse> resultList = FutureUtils.result(Future.collect(futureList));
+        for (WriteResponse wr : resultList) {
+            assertEquals(DLSN.InvalidDLSN, DLSN.deserialize(wr.getDlsn()));
+        }
+
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteOpNoChecksum() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext();
+        Future<WriteResponse> result = localService.writeWithContext("test", getTestDataBuffer(), ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testTruncateOpNoChecksum() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext();
+        Future<WriteResponse> result = localService.truncate("test", new DLSN(1, 2, 3).serialize(), ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testStreamOpNoChecksum() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext();
+        HeartbeatOptions option = new HeartbeatOptions();
+        option.setSendHeartBeatToReader(true);
+
+        // hearbeat to acquire the stream and then release the stream
+        Future<WriteResponse> result = localService.heartbeatWithOptions("test", ctx, option);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
+        result = localService.release("test", ctx);
+        resp = Await.result(result);
+        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
+
+        // heartbeat to acquire the stream and then delete the stream
+        result = localService.heartbeatWithOptions("test", ctx, option);
+        resp = Await.result(result);
+        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
+        result = localService.delete("test", ctx);
+        resp = Await.result(result);
+        assertEquals(StatusCode.SUCCESS, resp.getHeader().getCode());
+
+        // shutdown the local service
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteOpChecksumBadChecksum() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext().setCrc32(999);
+        Future<WriteResponse> result = localService.writeWithContext("test", getTestDataBuffer(), ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteOpChecksumBadStream() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext().setCrc32(
+            ProtocolUtils.writeOpCRC32("test", getTestDataBuffer().array()));
+        Future<WriteResponse> result = localService.writeWithContext("test1", getTestDataBuffer(), ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testWriteOpChecksumBadData() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        ByteBuffer buffer = getTestDataBuffer();
+        WriteContext ctx = new WriteContext().setCrc32(
+            ProtocolUtils.writeOpCRC32("test", buffer.array()));
+
+        // Overwrite 1 byte to corrupt data.
+        buffer.put(1, (byte) 0xab);
+        Future<WriteResponse> result = localService.writeWithContext("test", buffer, ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testStreamOpChecksumBadChecksum() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext().setCrc32(999);
+        Future<WriteResponse> result = localService.heartbeat("test", ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        result = localService.release("test", ctx);
+        resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        result = localService.delete("test", ctx);
+        resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testTruncateOpChecksumBadChecksum() throws Exception {
+        DistributedLogServiceImpl localService = createConfiguredLocalService();
+        WriteContext ctx = new WriteContext().setCrc32(999);
+        Future<WriteResponse> result = localService.truncate("test", new DLSN(1, 2, 3).serialize(), ctx);
+        WriteResponse resp = Await.result(result);
+        assertEquals(StatusCode.CHECKSUM_FAILED, resp.getHeader().getCode());
+        localService.shutdown();
+    }
+
+    private WriteOp getWriteOp(String name, SettableFeature disabledFeature, Long checksum) {
+        return new WriteOp(name,
+            ByteBuffer.wrap("test".getBytes()),
+            new NullStatsLogger(),
+            new NullStatsLogger(),
+            new IdentityStreamPartitionConverter(),
+            new ServerConfiguration(),
+            (byte) 0,
+            checksum,
+            false,
+            disabledFeature,
+            DefaultAccessControlManager.INSTANCE);
+    }
+
+    @Test(timeout = 60000)
+    public void testStreamOpBadChecksumWithChecksumDisabled() throws Exception {
+        String streamName = testName.getMethodName();
+
+        SettableFeature disabledFeature = new SettableFeature("", 0);
+
+        WriteOp writeOp0 = getWriteOp(streamName, disabledFeature, 919191L);
+        WriteOp writeOp1 = getWriteOp(streamName, disabledFeature, 919191L);
+
+        try {
+            writeOp0.preExecute();
+            fail("should have thrown");
+        } catch (Exception ex) {
+        }
+
+        disabledFeature.set(1);
+        writeOp1.preExecute();
+    }
+
+    @Test(timeout = 60000)
+    public void testStreamOpGoodChecksumWithChecksumDisabled() throws Exception {
+        String streamName = testName.getMethodName();
+
+        SettableFeature disabledFeature = new SettableFeature("", 1);
+        WriteOp writeOp0 = getWriteOp(
+            streamName,
+            disabledFeature,
+            ProtocolUtils.writeOpCRC32(streamName, "test".getBytes()));
+        WriteOp writeOp1 = getWriteOp(
+            streamName,
+            disabledFeature,
+            ProtocolUtils.writeOpCRC32(streamName, "test".getBytes()));
+
+        writeOp0.preExecute();
+        disabledFeature.set(0);
+        writeOp1.preExecute();
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseStreamsShouldFlush() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
+                .setImmediateFlushEnabled(false)
+                .setPeriodicFlushFrequencyMilliSeconds(0);
+
+        String streamNamePrefix = testName.getMethodName();
+        DistributedLogServiceImpl localService = createService(serverConf, confLocal);
+        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
+
+        int numStreams = 10;
+        int numWrites = 10;
+        List<Future<WriteResponse>> futureList =
+                Lists.newArrayListWithExpectedSize(numStreams * numWrites);
+        for (int i = 0; i < numStreams; i++) {
+            String streamName = streamNamePrefix + "-" + i;
+            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));
+            for (int j = 0; j < numWrites; j++) {
+                futureList.add(localService.write(streamName, createRecord(i * numWrites + j)));
+            }
+        }
+
+        assertEquals("There should be " + numStreams + " streams in cache",
+                numStreams, streamManager.getCachedStreams().size());
+        while (streamManager.getAcquiredStreams().size() < numStreams) {
+            TimeUnit.MILLISECONDS.sleep(20);
+        }
+
+        Future<List<Void>> closeResult = localService.closeStreams();
+        List<Void> closedStreams = Await.result(closeResult);
+        assertEquals("There should be " + numStreams + " streams closed",
+                numStreams, closedStreams.size());
+        // all writes should be flushed
+        for (Future<WriteResponse> future : futureList) {
+            WriteResponse response = Await.result(future);
+            assertTrue("Op should succeed or be rejected : " + response.getHeader().getCode(),
+                    StatusCode.SUCCESS == response.getHeader().getCode()
+                        || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
+                        || StatusCode.STREAM_UNAVAILABLE == response.getHeader().getCode());
+        }
+        assertTrue("There should be no streams in the cache",
+                streamManager.getCachedStreams().isEmpty());
+        assertTrue("There should be no streams in the acquired cache",
+                streamManager.getAcquiredStreams().isEmpty());
+
+        localService.shutdown();
+    }
+
+    @Test(timeout = 60000)
+    public void testCloseStreamsShouldAbort() throws Exception {
+        DistributedLogConfiguration confLocal = newLocalConf();
+        confLocal.setOutputBufferSize(Integer.MAX_VALUE)
+                .setImmediateFlushEnabled(false)
+                .setPeriodicFlushFrequencyMilliSeconds(0);
+
+        String streamNamePrefix = testName.getMethodName();
+        DistributedLogServiceImpl localService = createService(serverConf, confLocal);
+        StreamManagerImpl streamManager = (StreamManagerImpl) localService.getStreamManager();
+
+        int numStreams = 10;
+        int numWrites = 10;
+        List<Future<WriteResponse>> futureList =
+                Lists.newArrayListWithExpectedSize(numStreams * numWrites);
+        for (int i = 0; i < numStreams; i++) {
+            String streamName = streamNamePrefix + "-" + i;
+            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));
+            for (int j = 0; j < numWrites; j++) {
+                futureList.add(localService.write(streamName, createRecord(i * numWrites + j)));
+            }
+        }
+
+        assertEquals("There should be " + numStreams + " streams in cache",
+                numStreams, streamManager.getCachedStreams().size());
+        while (streamManager.getAcquiredStreams().size() < numStreams) {
+            TimeUnit.MILLISECONDS.sleep(20);
+        }
+
+        for (Stream s : streamManager.getAcquiredStreams().values()) {
+            StreamImpl stream = (StreamImpl) s;
+            stream.setStatus(StreamStatus.ERROR);
+        }
+
+        Future<List<Void>> closeResult = localService.closeStreams();
+        List<Void> closedStreams = Await.result(closeResult);
+        assertEquals("There should be " + numStreams + " streams closed",
+                numStreams, closedStreams.size());
+        // all writes should be flushed
+        for (Future<WriteResponse> future : futureList) {
+            WriteResponse response = Await.result(future);
+            assertTrue("Op should fail with " + StatusCode.BK_TRANSMIT_ERROR + " or be rejected : "
+                    + response.getHeader().getCode(),
+                    StatusCode.BK_TRANSMIT_ERROR == response.getHeader().getCode()
+                        || StatusCode.WRITE_EXCEPTION == response.getHeader().getCode()
+                        || StatusCode.WRITE_CANCELLED_EXCEPTION == response.getHeader().getCode());
+        }
+        // acquired streams should all been removed after we close them
+        assertTrue("There should be no streams in the acquired cache",
+            streamManager.getAcquiredStreams().isEmpty());
+        localService.shutdown();
+        // cached streams wouldn't be removed immediately after streams are closed
+        // but they should be removed after we shutdown the service
+        assertTrue("There should be no streams in the cache after shutting down the service",
+            streamManager.getCachedStreams().isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testShutdown() throws Exception {
+        service.shutdown();
+        StreamManagerImpl streamManager = (StreamManagerImpl) service.getStreamManager();
+        WriteResponse response =
+                Await.result(service.write(testName.getMethodName(), createRecord(0L)));
+        assertEquals("Write should fail with " + StatusCode.SERVICE_UNAVAILABLE,
+                StatusCode.SERVICE_UNAVAILABLE, response.getHeader().getCode());
+        assertTrue("There should be no streams created after shutdown",
+                streamManager.getCachedStreams().isEmpty());
+        assertTrue("There should be no streams acquired after shutdown",
+                streamManager.getAcquiredStreams().isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testGetOwner() throws Exception {
+        ((LocalRoutingService) service.getRoutingService())
+                .addHost("stream-0", service.getServiceAddress().getSocketAddress())
+                .setAllowRetrySameHost(false);
+
+        service.startPlacementPolicy();
+
+        WriteResponse response = FutureUtils.result(service.getOwner("stream-1", new WriteContext()));
+        assertEquals(StatusCode.FOUND, response.getHeader().getCode());
+        assertEquals(service.getServiceAddress().toString(),
+                response.getHeader().getLocation());
+
+        // service cache "stream-2"
+        StreamImpl stream = (StreamImpl) service.getStreamManager().getOrCreateStream("stream-2", false);
+        // create write ops to stream-2 to make service acquire the stream
+        WriteOp op = createWriteOp(service, "stream-2", 0L);
+        stream.submit(op);
+        stream.start();
+        WriteResponse wr = Await.result(op.result());
+        assertEquals("Op should succeed",
+                StatusCode.SUCCESS, wr.getHeader().getCode());
+        assertEquals("Service should acquire stream",
+                StreamStatus.INITIALIZED, stream.getStatus());
+        assertNotNull(stream.getManager());
+        assertNotNull(stream.getWriter());
+        assertNull(stream.getLastException());
+
+        // the stream is acquired
+        response = FutureUtils.result(service.getOwner("stream-2", new WriteContext()));
+        assertEquals(StatusCode.FOUND, response.getHeader().getCode());
+        assertEquals(service.getServiceAddress().toString(),
+                response.getHeader().getLocation());
+    }
+
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestRegionUnavailable.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestRegionUnavailable.java
new file mode 100644
index 0000000..d0a2f88
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestRegionUnavailable.java
@@ -0,0 +1,140 @@
+/**
+ * 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.service;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.feature.DefaultFeatureProvider;
+import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
+import java.net.SocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import org.apache.bookkeeper.feature.Feature;
+import org.apache.bookkeeper.feature.FeatureProvider;
+import org.apache.bookkeeper.feature.SettableFeature;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link org.apache.distributedlog.exceptions.RegionUnavailableException}.
+ */
+public class TestRegionUnavailable extends DistributedLogServerTestCase {
+
+    /**
+     * A feature provider for testing.
+     */
+    public static class TestFeatureProvider extends DefaultFeatureProvider {
+
+        public TestFeatureProvider(String rootScope,
+                                   DistributedLogConfiguration conf,
+                                   StatsLogger statsLogger) {
+            super(rootScope, conf, statsLogger);
+        }
+
+        @Override
+        protected Feature makeFeature(String featureName) {
+            if (featureName.contains(ServerFeatureKeys.REGION_STOP_ACCEPT_NEW_STREAM.name().toLowerCase())) {
+                return new SettableFeature(featureName, 10000);
+            }
+            return super.makeFeature(featureName);
+        }
+
+        @Override
+        protected FeatureProvider makeProvider(String fullScopeName) {
+            return super.makeProvider(fullScopeName);
+        }
+    }
+
+    private final int numServersPerDC = 3;
+    private final List<DLServer> localCluster;
+    private final List<DLServer> remoteCluster;
+    private TwoRegionDLClient client;
+
+    public TestRegionUnavailable() {
+        super(true);
+        this.localCluster = new ArrayList<DLServer>();
+        this.remoteCluster = new ArrayList<DLServer>();
+    }
+
+    @Before
+    @Override
+    public void setup() throws Exception {
+        DistributedLogConfiguration localConf = new DistributedLogConfiguration();
+        localConf.addConfiguration(conf);
+        localConf.setFeatureProviderClass(TestFeatureProvider.class);
+        DistributedLogConfiguration remoteConf = new DistributedLogConfiguration();
+        remoteConf.addConfiguration(conf);
+        super.setup();
+        int localPort = 9010;
+        int remotePort = 9020;
+        for (int i = 0; i < numServersPerDC; i++) {
+            localCluster.add(createDistributedLogServer(localConf, localPort + i));
+            remoteCluster.add(createDistributedLogServer(remoteConf, remotePort + i));
+        }
+        Map<SocketAddress, String> regionMap = new HashMap<SocketAddress, String>();
+        for (DLServer server : localCluster) {
+            regionMap.put(server.getAddress(), "local");
+        }
+        for (DLServer server : remoteCluster) {
+            regionMap.put(server.getAddress(), "remote");
+        }
+        client = createTwoRegionDLClient("two_regions_client", regionMap);
+
+    }
+
+    private void registerStream(String streamName) {
+        for (DLServer server : localCluster) {
+            client.localRoutingService.addHost(streamName, server.getAddress());
+        }
+        client.remoteRoutingService.addHost(streamName, remoteCluster.get(0).getAddress());
+    }
+
+    @After
+    @Override
+    public void teardown() throws Exception {
+        super.teardown();
+        if (null != client) {
+            client.shutdown();
+        }
+        for (DLServer server : localCluster) {
+            server.shutdown();
+        }
+        for (DLServer server : remoteCluster) {
+            server.shutdown();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testRegionUnavailable() throws Exception {
+        String name = "dlserver-region-unavailable";
+        registerStream(name);
+
+        for (long i = 1; i <= 10; i++) {
+            client.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes())).get();
+        }
+
+        // check local region
+        for (DLServer server : localCluster) {
+            checkStreams(0, server);
+        }
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestStatsFilter.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestStatsFilter.java
new file mode 100644
index 0000000..c8b8bdf
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/TestStatsFilter.java
@@ -0,0 +1,58 @@
+/**
+ * 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.service;
+
+import static org.junit.Assert.assertEquals;
+
+import com.twitter.finagle.Service;
+import com.twitter.finagle.service.ConstantService;
+import com.twitter.util.Await;
+import com.twitter.util.Future;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.bookkeeper.stats.StatsLogger;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link StatsFilter}.
+ */
+public class TestStatsFilter {
+
+    class RuntimeExService<Req, Rep> extends Service<Req, Rep> {
+        public Future<Rep> apply(Req request) {
+            throw new RuntimeException("test");
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceSuccess() throws Exception {
+        StatsLogger stats = new NullStatsLogger();
+        StatsFilter<String, String> filter = new StatsFilter<String, String>(stats);
+        Future<String> result = filter.apply("", new ConstantService<String, String>(Future.value("result")));
+        assertEquals("result", Await.result(result));
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceFailure() throws Exception {
+        StatsLogger stats = new NullStatsLogger();
+        StatsFilter<String, String> filter = new StatsFilter<String, String>(stats);
+        try {
+            filter.apply("", new RuntimeExService<String, String>());
+        } catch (RuntimeException ex) {
+        }
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestBalancerUtils.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestBalancerUtils.java
new file mode 100644
index 0000000..21bebb5
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestBalancerUtils.java
@@ -0,0 +1,65 @@
+/**
+ * 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.service.balancer;
+
+import static org.junit.Assert.assertEquals;
+
+import java.util.HashMap;
+import java.util.Map;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link BalancerUtils}.
+ */
+public class TestBalancerUtils {
+
+    @Test(timeout = 60000)
+    public void testCalculateNumStreamsToRebalance() {
+        String myNode = "mynode";
+
+        // empty load distribution
+        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
+                myNode, new HashMap<String, Integer>(), 0, 10));
+        // my node doesn't exist in load distribution
+        Map<String, Integer> loadDistribution = new HashMap<String, Integer>();
+        loadDistribution.put("node1", 10);
+        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
+                myNode, loadDistribution, 0, 10));
+        // my node doesn't reach rebalance water mark
+        loadDistribution.clear();
+        loadDistribution.put("node1", 1);
+        loadDistribution.put(myNode, 100);
+        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
+                myNode, loadDistribution, 200, 10));
+        // my node is below average in the cluster.
+        loadDistribution.clear();
+        loadDistribution.put(myNode, 1);
+        loadDistribution.put("node1", 99);
+        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
+                myNode, loadDistribution, 0, 10));
+        // my node is above average in the cluster
+        assertEquals(49, BalancerUtils.calculateNumStreamsToRebalance(
+                "node1", loadDistribution, 0, 10));
+        // my node is at the tolerance range
+        loadDistribution.clear();
+        loadDistribution.put(myNode, 55);
+        loadDistribution.put("node1", 45);
+        assertEquals(0, BalancerUtils.calculateNumStreamsToRebalance(
+                myNode, loadDistribution, 0, 10));
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestClusterBalancer.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestClusterBalancer.java
new file mode 100644
index 0000000..fb3fb6e
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestClusterBalancer.java
@@ -0,0 +1,189 @@
+/**
+ * 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.service.balancer;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.fail;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.client.monitor.MonitorServiceClient;
+import org.apache.distributedlog.service.DLSocketAddress;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
+import org.apache.distributedlog.service.DistributedLogServerTestCase;
+import com.twitter.util.Await;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+import org.apache.commons.lang3.tuple.Pair;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Ignore;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Case for {@link ClusterBalancer}.
+ */
+public class TestClusterBalancer extends DistributedLogServerTestCase {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestClusterBalancer.class);
+
+    private final int numServers = 5;
+    private final List<DLServer> cluster;
+    private DLClient client;
+
+    public TestClusterBalancer() {
+        super(true);
+        this.cluster = new ArrayList<DLServer>();
+    }
+
+    @Before
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+        int initPort = 9001;
+        for (int i = 0; i < numServers; i++) {
+            cluster.add(createDistributedLogServer(initPort + i));
+        }
+        client = createDistributedLogClient("cluster_client", Optional.<String>absent());
+    }
+
+    @After
+    @Override
+    public void teardown() throws Exception {
+        super.teardown();
+        if (null != client) {
+            client.shutdown();
+        }
+        for (DLServer server: cluster) {
+            server.shutdown();
+        }
+    }
+
+    private void initStreams(String namePrefix) {
+        logger.info("Init streams with prefix {}", namePrefix);
+        // Stream Distribution: 5, 4, 3, 2, 1
+        initStreams(namePrefix, 5, 1, 0);
+        initStreams(namePrefix, 4, 6, 1);
+        initStreams(namePrefix, 3, 10, 2);
+        initStreams(namePrefix, 2, 13, 3);
+        initStreams(namePrefix, 1, 15, 4);
+    }
+
+    private void initStreams(String namePrefix, int numStreams, int streamId, int proxyId) {
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + (streamId++);
+            client.routingService.addHost(name, cluster.get(proxyId).getAddress());
+        }
+    }
+
+    private void writeStreams(String namePrefix) throws Exception {
+        logger.info("Write streams with prefix {}", namePrefix);
+        writeStreams(namePrefix, 5, 1);
+        writeStreams(namePrefix, 4, 6);
+        writeStreams(namePrefix, 3, 10);
+        writeStreams(namePrefix, 2, 13);
+        writeStreams(namePrefix, 1, 15);
+    }
+
+    private void writeStreams(String namePrefix, int numStreams, int streamId) throws Exception {
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + (streamId++);
+            try {
+                Await.result(client.dlClient.write(name, ByteBuffer.wrap(name.getBytes(UTF_8))));
+            } catch (Exception e) {
+                logger.error("Error writing stream {} : ", name, e);
+                throw e;
+            }
+        }
+    }
+
+    private void validateStreams(String namePrefix) throws Exception {
+        logger.info("Validate streams with prefix {}", namePrefix);
+        validateStreams(namePrefix, 5, 1, 0);
+        validateStreams(namePrefix, 4, 6, 1);
+        validateStreams(namePrefix, 3, 10, 2);
+        validateStreams(namePrefix, 2, 13, 3);
+        validateStreams(namePrefix, 1, 15, 4);
+    }
+
+    private void validateStreams(String namePrefix, int numStreams, int streamId, int proxyIdx) {
+        Set<String> expectedStreams = new HashSet<String>();
+        for (int i = 0; i < numStreams; i++) {
+            expectedStreams.add(namePrefix + (streamId++));
+        }
+        checkStreams(expectedStreams, cluster.get(proxyIdx));
+    }
+
+    @Ignore
+    @Test(timeout = 60000)
+    public void testBalanceAll() throws Exception {
+        String namePrefix = "clusterbalancer-balance-all-";
+
+        initStreams(namePrefix);
+        writeStreams(namePrefix);
+        validateStreams(namePrefix);
+
+        Optional<RateLimiter> rateLimiter = Optional.absent();
+
+        Balancer balancer = new ClusterBalancer(client.dlClientBuilder,
+                Pair.of((DistributedLogClient) client.dlClient, (MonitorServiceClient) client.dlClient));
+        logger.info("Rebalancing from 'unknown' target");
+        try {
+            balancer.balanceAll("unknown", 10, rateLimiter);
+            fail("Should fail on balanceAll from 'unknown' target.");
+        } catch (IllegalArgumentException iae) {
+            // expected
+        }
+        validateStreams(namePrefix);
+
+        logger.info("Rebalancing from 'unexisted' host");
+        String addr = DLSocketAddress.toString(DLSocketAddress.getSocketAddress(9999));
+        balancer.balanceAll(addr, 10, rateLimiter);
+        validateStreams(namePrefix);
+
+        addr = DLSocketAddress.toString(cluster.get(0).getAddress());
+        logger.info("Rebalancing from host {}.", addr);
+        balancer.balanceAll(addr, 10, rateLimiter);
+        checkStreams(0, cluster.get(0));
+        checkStreams(4, cluster.get(1));
+        checkStreams(3, cluster.get(2));
+        checkStreams(4, cluster.get(3));
+        checkStreams(4, cluster.get(4));
+
+        addr = DLSocketAddress.toString(cluster.get(2).getAddress());
+        logger.info("Rebalancing from host {}.", addr);
+        balancer.balanceAll(addr, 10, rateLimiter);
+        checkStreams(3, cluster.get(0));
+        checkStreams(4, cluster.get(1));
+        checkStreams(0, cluster.get(2));
+        checkStreams(4, cluster.get(3));
+        checkStreams(4, cluster.get(4));
+
+        logger.info("Rebalancing the cluster");
+        balancer.balance(0, 0.0f, 10, rateLimiter);
+        for (int i = 0; i < 5; i++) {
+            checkStreams(3, cluster.get(i));
+        }
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestCountBasedStreamChooser.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestCountBasedStreamChooser.java
new file mode 100644
index 0000000..6734083
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestCountBasedStreamChooser.java
@@ -0,0 +1,204 @@
+/**
+ * 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.service.balancer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+import static org.junit.Assert.assertNull;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import com.google.common.collect.Sets;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Map;
+import java.util.Set;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link CountBasedStreamChooser}.
+ */
+public class TestCountBasedStreamChooser {
+
+    @Test(timeout = 60000)
+    public void testEmptyStreamDistribution() {
+        try {
+            new CountBasedStreamChooser(new HashMap<SocketAddress, Set<String>>());
+            fail("Should fail constructing stream chooser if the stream distribution is empty");
+        } catch (IllegalArgumentException iae) {
+            // expected
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testMultipleHostsWithEmptyStreams() {
+        for (int i = 1; i <= 3; i++) {
+            Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
+            int port = 1000;
+            for (int j = 0; j < i; j++) {
+                SocketAddress address = new InetSocketAddress("127.0.0.1", port + j);
+                streamDistribution.put(address, new HashSet<String>());
+            }
+
+            CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
+            for (int k = 0; k < i + 1; k++) {
+                assertNull(chooser.choose());
+            }
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testSingleHostWithStreams() {
+        for (int i = 0; i < 3; i++) {
+            Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
+
+            Set<String> streams = new HashSet<String>();
+            for (int j = 0; j < 3; j++) {
+                streams.add("SingleHostStream-" + j);
+            }
+
+            int port = 1000;
+            SocketAddress address = new InetSocketAddress("127.0.0.1", port);
+            streamDistribution.put(address, streams);
+
+            for (int k = 1; k <= i; k++) {
+                address = new InetSocketAddress("127.0.0.1", port + k);
+                streamDistribution.put(address, new HashSet<String>());
+            }
+
+            Set<String> choosenStreams = new HashSet<String>();
+
+            CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
+            for (int l = 0; l < 3 + i + 1; l++) {
+                String s = chooser.choose();
+                if (null != s) {
+                    choosenStreams.add(s);
+                }
+            }
+
+            assertEquals(streams.size(), choosenStreams.size());
+            assertTrue(Sets.difference(streams, choosenStreams).immutableCopy().isEmpty());
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testHostsHaveSameNumberStreams() {
+        Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
+        Set<String> allStreams = new HashSet<String>();
+
+        int numHosts = 3;
+        int numStreamsPerHost = 3;
+
+        int port = 1000;
+        for (int i = 1; i <= numHosts; i++) {
+            SocketAddress address = new InetSocketAddress("127.0.0.1", port + i);
+            Set<String> streams = new HashSet<String>();
+
+            for (int j = 1; j <= numStreamsPerHost; j++) {
+                String streamName = "HostsHaveSameNumberStreams-" + i + "-" + j;
+                streams.add(streamName);
+                allStreams.add(streamName);
+            }
+
+            streamDistribution.put(address, streams);
+        }
+
+        Set<String> streamsChoosen = new HashSet<String>();
+        CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
+        for (int i = 1; i <= numStreamsPerHost; i++) {
+            for (int j = 1; j <= numHosts; j++) {
+                String s = chooser.choose();
+                assertNotNull(s);
+                streamsChoosen.add(s);
+            }
+            for (int j = 0; j < numHosts; j++) {
+                assertEquals(numStreamsPerHost - i, chooser.streamsDistribution.get(j).getRight().size());
+            }
+        }
+        assertNull(chooser.choose());
+        assertEquals(numHosts * numStreamsPerHost, streamsChoosen.size());
+        assertTrue(Sets.difference(allStreams, streamsChoosen).isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testHostsHaveDifferentNumberStreams() {
+        Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
+        Set<String> allStreams = new HashSet<String>();
+
+        int numHosts = 6;
+        int maxStreamsPerHost = 4;
+
+        int port = 1000;
+        for (int i = 0; i < numHosts; i++) {
+            int group = i / 2;
+            int numStreamsThisGroup = maxStreamsPerHost - group;
+
+            SocketAddress address = new InetSocketAddress("127.0.0.1", port + i);
+            Set<String> streams = new HashSet<String>();
+
+            for (int j = 1; j <= numStreamsThisGroup; j++) {
+                String streamName = "HostsHaveDifferentNumberStreams-" + i + "-" + j;
+                streams.add(streamName);
+                allStreams.add(streamName);
+            }
+
+            streamDistribution.put(address, streams);
+        }
+
+        Set<String> streamsChoosen = new HashSet<String>();
+        CountBasedStreamChooser chooser = new CountBasedStreamChooser(streamDistribution);
+
+        for (int i = 0; i < allStreams.size(); i++) {
+            String s = chooser.choose();
+            assertNotNull(s);
+            streamsChoosen.add(s);
+        }
+        assertNull(chooser.choose());
+        assertEquals(allStreams.size(), streamsChoosen.size());
+        assertTrue(Sets.difference(allStreams, streamsChoosen).isEmpty());
+    }
+
+    @Test(timeout = 60000)
+    public void testLimitedStreamChooser() {
+        Map<SocketAddress, Set<String>> streamDistribution = new HashMap<SocketAddress, Set<String>>();
+
+        Set<String> streams = new HashSet<String>();
+        for (int j = 0; j < 10; j++) {
+            streams.add("SingleHostStream-" + j);
+        }
+
+        int port = 1000;
+        SocketAddress address = new InetSocketAddress("127.0.0.1", port);
+        streamDistribution.put(address, streams);
+
+        Set<String> choosenStreams = new HashSet<String>();
+
+        CountBasedStreamChooser underlying = new CountBasedStreamChooser(streamDistribution);
+        LimitedStreamChooser chooser = LimitedStreamChooser.of(underlying, 1);
+        for (int l = 0; l < 10; l++) {
+            String s = chooser.choose();
+            if (null != s) {
+                choosenStreams.add(s);
+            }
+        }
+
+        assertEquals(1, choosenStreams.size());
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestSimpleBalancer.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestSimpleBalancer.java
new file mode 100644
index 0000000..73fa98a
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestSimpleBalancer.java
@@ -0,0 +1,180 @@
+/**
+ * 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.service.balancer;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
+import org.apache.distributedlog.service.DistributedLogServerTestCase;
+import com.twitter.util.Await;
+import java.nio.ByteBuffer;
+import java.util.Set;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * Test Case for {@link SimpleBalancer}.
+ */
+public class TestSimpleBalancer extends DistributedLogServerTestCase {
+
+    private static final Logger logger = LoggerFactory.getLogger(TestSimpleBalancer.class);
+
+    DLClient targetClient;
+    DLServer targetServer;
+
+    public TestSimpleBalancer() {
+        super(true);
+    }
+
+    @Before
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+        targetServer = createDistributedLogServer(7003);
+        targetClient = createDistributedLogClient("target", Optional.<String>absent());
+    }
+
+    @After
+    @Override
+    public void teardown() throws Exception {
+        super.teardown();
+        if (null != targetClient) {
+            targetClient.shutdown();
+        }
+        if (null != targetServer) {
+            targetServer.shutdown();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testBalanceAll() throws Exception {
+        String namePrefix = "simplebalancer-balance-all-";
+        int numStreams = 10;
+
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            // src client
+            dlClient.routingService.addHost(name, dlServer.getAddress());
+            // target client
+            targetClient.routingService.addHost(name, targetServer.getAddress());
+        }
+
+        // write to multiple streams
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes(UTF_8))));
+        }
+
+        // validation
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            checkStream(name, dlClient, dlServer, 1, numStreams, numStreams, true, true);
+            checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
+        }
+
+        Optional<RateLimiter> rateLimiter = Optional.absent();
+
+        Balancer balancer = new SimpleBalancer("source", dlClient.dlClient, dlClient.dlClient,
+                                               "target", targetClient.dlClient, targetClient.dlClient);
+        logger.info("Rebalancing from 'unknown' target");
+        try {
+            balancer.balanceAll("unknown", 10, rateLimiter);
+            fail("Should fail on balanceAll from 'unknown' target.");
+        } catch (IllegalArgumentException iae) {
+            // expected
+        }
+
+        // nothing to balance from 'target'
+        logger.info("Rebalancing from 'target' target");
+        balancer.balanceAll("target", 1, rateLimiter);
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            checkStream(name, dlClient, dlServer, 1, numStreams, numStreams, true, true);
+            checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
+        }
+
+        // balance all streams from 'source'
+        logger.info("Rebalancing from 'source' target");
+        balancer.balanceAll("source", 10, rateLimiter);
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            checkStream(name, targetClient, targetServer, 1, numStreams, numStreams, true, true);
+            checkStream(name, dlClient, dlServer, 0, 0, 0, false, false);
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testBalanceStreams() throws Exception {
+        String namePrefix = "simplebalancer-balance-streams-";
+        int numStreams = 10;
+
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            // src client
+            dlClient.routingService.addHost(name, dlServer.getAddress());
+            // target client
+            targetClient.routingService.addHost(name, targetServer.getAddress());
+        }
+
+        // write to multiple streams
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            Await.result(dlClient.dlClient.write(name, ByteBuffer.wrap(("" + i).getBytes(UTF_8))));
+        }
+
+        // validation
+        for (int i = 0; i < numStreams; i++) {
+            String name = namePrefix + i;
+            checkStream(name, dlClient, dlServer, 1, numStreams, numStreams, true, true);
+            checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
+        }
+
+        Optional<RateLimiter> rateLimiter = Optional.absent();
+
+        Balancer balancer = new SimpleBalancer("source", dlClient.dlClient, dlClient.dlClient,
+                                               "target", targetClient.dlClient, targetClient.dlClient);
+
+        // balance all streams from 'source'
+        logger.info("Rebalancing streams between targets");
+        balancer.balance(0, 0, 10, rateLimiter);
+
+        Set<String> sourceStreams = getAllStreamsFromDistribution(getStreamOwnershipDistribution(dlClient));
+        Set<String> targetStreams = getAllStreamsFromDistribution(getStreamOwnershipDistribution(targetClient));
+
+        assertEquals(numStreams / 2, sourceStreams.size());
+        assertEquals(numStreams / 2, targetStreams.size());
+
+        for (String name : sourceStreams) {
+            checkStream(name, dlClient, dlServer, 1, numStreams / 2, numStreams / 2, true, true);
+            checkStream(name, targetClient, targetServer, 1, numStreams / 2, numStreams / 2, false, false);
+        }
+
+        for (String name : targetStreams) {
+            checkStream(name, targetClient, targetServer, 1, numStreams / 2, numStreams / 2, true, true);
+            checkStream(name, dlClient, dlServer, 1, numStreams / 2, numStreams / 2, false, false);
+        }
+    }
+
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestStreamMover.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestStreamMover.java
new file mode 100644
index 0000000..ce7b2c1
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/balancer/TestStreamMover.java
@@ -0,0 +1,86 @@
+/**
+ * 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.service.balancer;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.base.Optional;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogCluster.DLServer;
+import org.apache.distributedlog.service.DistributedLogServerTestCase;
+import com.twitter.util.Await;
+import java.nio.ByteBuffer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link StreamMover}.
+ */
+public class TestStreamMover extends DistributedLogServerTestCase {
+
+    DLClient targetClient;
+    DLServer targetServer;
+
+    public TestStreamMover() {
+        super(true);
+    }
+
+    @Before
+    @Override
+    public void setup() throws Exception {
+        super.setup();
+        targetServer = createDistributedLogServer(7003);
+        targetClient = createDistributedLogClient("target", Optional.<String>absent());
+    }
+
+    @After
+    @Override
+    public void teardown() throws Exception {
+        super.teardown();
+        if (null != targetClient) {
+            targetClient.shutdown();
+        }
+        if (null != targetServer) {
+            targetServer.shutdown();
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testMoveStream() throws Exception {
+        String name = "dlserver-move-stream";
+
+        // src client
+        dlClient.routingService.addHost(name, dlServer.getAddress());
+        // target client
+        targetClient.routingService.addHost(name, targetServer.getAddress());
+
+        // src client write a record to that stream
+        Await.result(((DistributedLogClient) dlClient.dlClient).write(name, ByteBuffer.wrap("1".getBytes(UTF_8))));
+        checkStream(name, dlClient, dlServer, 1, 1, 1, true, true);
+        checkStream(name, targetClient, targetServer, 0, 0, 0, false, false);
+
+        StreamMover streamMover = new StreamMoverImpl("source", dlClient.dlClient, dlClient.dlClient,
+                                                      "target", targetClient.dlClient, targetClient.dlClient);
+        assertTrue(streamMover.moveStream(name));
+        checkStream(name, dlClient, dlServer, 0, 0, 0, false, false);
+        checkStream(name, targetClient, targetServer, 1, 1, 1, true, true);
+    }
+
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestServerConfiguration.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestServerConfiguration.java
new file mode 100644
index 0000000..71dfa45
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestServerConfiguration.java
@@ -0,0 +1,68 @@
+/**
+ * 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.service.config;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+/**
+ * Test Case for {@link ServerConfiguration}.
+ */
+public class TestServerConfiguration {
+
+    @Test(timeout = 60000, expected = IllegalArgumentException.class)
+    public void testUnassignedShardId() {
+        new ServerConfiguration().validate();
+    }
+
+    @Test(timeout = 60000)
+    public void testAssignedShardId() {
+        ServerConfiguration conf = new ServerConfiguration();
+        conf.setServerShardId(100);
+        conf.validate();
+        assertEquals(100, conf.getServerShardId());
+    }
+
+    @Test(timeout = 60000, expected = IllegalArgumentException.class)
+    public void testInvalidServerThreads() {
+        ServerConfiguration conf = new ServerConfiguration();
+        conf.setServerShardId(100);
+        conf.setServerThreads(-1);
+        conf.validate();
+    }
+
+    @Test(timeout = 60000, expected = IllegalArgumentException.class)
+    public void testInvalidDlsnVersion() {
+        ServerConfiguration conf = new ServerConfiguration();
+        conf.setServerShardId(100);
+        conf.setDlsnVersion((byte) 9999);
+        conf.validate();
+    }
+
+    @Test(timeout = 60000)
+    public void testUseHostnameAsAllocatorPoolName() {
+        ServerConfiguration conf = new ServerConfiguration();
+        assertFalse("Should not use hostname by default", conf.isUseHostnameAsAllocatorPoolName());
+        conf.setUseHostnameAsAllocatorPoolName(true);
+        assertTrue("Should use hostname now", conf.isUseHostnameAsAllocatorPoolName());
+    }
+
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestStreamConfigProvider.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestStreamConfigProvider.java
new file mode 100644
index 0000000..bdbde11
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/config/TestStreamConfigProvider.java
@@ -0,0 +1,140 @@
+/**
+ * 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.service.config;
+
+import static org.apache.distributedlog.DistributedLogConfiguration.BKDL_RETENTION_PERIOD_IN_HOURS;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import com.google.common.base.Optional;
+import com.google.common.util.concurrent.ThreadFactoryBuilder;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.config.PropertiesWriter;
+import org.apache.distributedlog.service.streamset.IdentityStreamPartitionConverter;
+import org.apache.distributedlog.service.streamset.StreamPartitionConverter;
+import java.util.concurrent.Executors;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.TimeUnit;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link StreamConfigProvider}.
+ */
+public class TestStreamConfigProvider {
+    private static final String DEFAULT_CONFIG_DIR = "conf";
+    private final String defaultConfigPath;
+    private final ScheduledExecutorService configExecutorService;
+
+    public TestStreamConfigProvider() throws Exception {
+        this.configExecutorService = Executors.newScheduledThreadPool(1,
+                new ThreadFactoryBuilder().setNameFormat("DistributedLogService-Dyncfg-%d").build());
+        PropertiesWriter writer = new PropertiesWriter();
+        writer.save();
+        this.defaultConfigPath = writer.getFile().getPath();
+    }
+
+    StreamConfigProvider getServiceProvider(StreamPartitionConverter converter)
+            throws Exception {
+        return getServiceProvider(converter, DEFAULT_CONFIG_DIR);
+    }
+
+    StreamConfigProvider getServiceProvider(
+            StreamPartitionConverter converter,
+            String configPath,
+            String defaultPath) throws Exception {
+        return new ServiceStreamConfigProvider(
+                configPath,
+                defaultPath,
+                converter,
+                configExecutorService,
+                1,
+                TimeUnit.SECONDS);
+    }
+
+    StreamConfigProvider getServiceProvider(
+            StreamPartitionConverter converter,
+            String configPath) throws Exception {
+        return getServiceProvider(converter, configPath, defaultConfigPath);
+    }
+
+    StreamConfigProvider getDefaultProvider(String configFile) throws Exception {
+        return new DefaultStreamConfigProvider(configFile, configExecutorService, 1, TimeUnit.SECONDS);
+    }
+
+    StreamConfigProvider getNullProvider() throws Exception {
+        return new NullStreamConfigProvider();
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceProviderWithConfigRouters() throws Exception {
+        getServiceProvider(new IdentityStreamPartitionConverter());
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceProviderWithMissingConfig() throws Exception {
+        StreamConfigProvider provider = getServiceProvider(new IdentityStreamPartitionConverter());
+        Optional<DynamicDistributedLogConfiguration> config = provider.getDynamicStreamConfig("stream1");
+        assertTrue(config.isPresent());
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceProviderWithDefaultConfigPath() throws Exception {
+        // Default config with property set.
+        PropertiesWriter writer1 = new PropertiesWriter();
+        writer1.setProperty("rpsStreamAcquireServiceLimit", "191919");
+        writer1.save();
+        String fallbackConfPath1 = writer1.getFile().getPath();
+        StreamConfigProvider provider1 = getServiceProvider(new IdentityStreamPartitionConverter(),
+                DEFAULT_CONFIG_DIR, fallbackConfPath1);
+        Optional<DynamicDistributedLogConfiguration> config1 = provider1.getDynamicStreamConfig("stream1");
+
+        // Empty default config.
+        PropertiesWriter writer2 = new PropertiesWriter();
+        writer2.save();
+        String fallbackConfPath2 = writer2.getFile().getPath();
+        StreamConfigProvider provider2 = getServiceProvider(new IdentityStreamPartitionConverter(),
+                DEFAULT_CONFIG_DIR, fallbackConfPath2);
+        Optional<DynamicDistributedLogConfiguration> config2 = provider2.getDynamicStreamConfig("stream1");
+
+        assertEquals(191919, config1.get().getRpsStreamAcquireServiceLimit());
+        assertEquals(-1, config2.get().getRpsStreamAcquireServiceLimit());
+    }
+
+    @Test(timeout = 60000)
+    public void testDefaultProvider() throws Exception {
+        PropertiesWriter writer = new PropertiesWriter();
+        writer.setProperty(BKDL_RETENTION_PERIOD_IN_HOURS, "99");
+        writer.save();
+        StreamConfigProvider provider = getDefaultProvider(writer.getFile().getPath());
+        Optional<DynamicDistributedLogConfiguration> config1 = provider.getDynamicStreamConfig("stream1");
+        Optional<DynamicDistributedLogConfiguration> config2 = provider.getDynamicStreamConfig("stream2");
+        assertTrue(config1.isPresent());
+        assertTrue(config1.get() == config2.get());
+        assertEquals(99, config1.get().getRetentionPeriodHours());
+    }
+
+    @Test(timeout = 60000)
+    public void testNullProvider() throws Exception {
+        StreamConfigProvider provider = getNullProvider();
+        Optional<DynamicDistributedLogConfiguration> config1 = provider.getDynamicStreamConfig("stream1");
+        Optional<DynamicDistributedLogConfiguration> config2 = provider.getDynamicStreamConfig("stream2");
+        assertFalse(config1.isPresent());
+        assertTrue(config1 == config2);
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
new file mode 100644
index 0000000..4f062f8
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestLeastLoadPlacementPolicy.java
@@ -0,0 +1,176 @@
+/**
+ * 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.service.placement;
+
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import static org.mockito.Matchers.anyString;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+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;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.SocketAddress;
+import java.util.LinkedHashSet;
+import java.util.Random;
+import java.util.Set;
+import java.util.TreeSet;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.junit.Test;
+import org.mockito.ArgumentCaptor;
+import org.mockito.invocation.InvocationOnMock;
+import org.mockito.stubbing.Answer;
+
+/**
+ * Test Case for {@link LeastLoadPlacementPolicy}.
+ */
+public class TestLeastLoadPlacementPolicy {
+
+    @Test(timeout = 10000)
+    public void testCalculateBalances() throws Exception {
+        int numSevers = new Random().nextInt(20) + 1;
+        int numStreams = new Random().nextInt(200) + 1;
+        RoutingService mockRoutingService = mock(RoutingService.class);
+        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
+        LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
+            new EqualLoadAppraiser(),
+            mockRoutingService,
+            mockNamespace,
+            null,
+            Duration.fromSeconds(600),
+            new NullStatsLogger());
+        TreeSet<ServerLoad> serverLoads =
+            Await.result(leastLoadPlacementPolicy.calculate(generateServers(numSevers), generateStreams(numStreams)));
+        long lowLoadPerServer = numStreams / numSevers;
+        long highLoadPerServer = lowLoadPerServer + 1;
+        for (ServerLoad serverLoad : serverLoads) {
+            long load = serverLoad.getLoad();
+            assertEquals(load, serverLoad.getStreamLoads().size());
+            assertTrue(String.format("Load %d is not between %d and %d",
+                load, lowLoadPerServer, highLoadPerServer), load == lowLoadPerServer || load == highLoadPerServer);
+        }
+    }
+
+    @Test(timeout = 10000)
+    public void testRefreshAndPlaceStream() throws Exception {
+        int numSevers = new Random().nextInt(20) + 1;
+        int numStreams = new Random().nextInt(200) + 1;
+        RoutingService mockRoutingService = mock(RoutingService.class);
+        when(mockRoutingService.getHosts()).thenReturn(generateSocketAddresses(numSevers));
+        DistributedLogNamespace mockNamespace = mock(DistributedLogNamespace.class);
+        try {
+            when(mockNamespace.getLogs()).thenReturn(generateStreams(numStreams).iterator());
+        } catch (IOException e) {
+            fail();
+        }
+        PlacementStateManager mockPlacementStateManager = mock(PlacementStateManager.class);
+        LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
+            new EqualLoadAppraiser(),
+            mockRoutingService,
+            mockNamespace,
+            mockPlacementStateManager,
+            Duration.fromSeconds(600),
+            new NullStatsLogger());
+        leastLoadPlacementPolicy.refresh();
+
+        final ArgumentCaptor<TreeSet> captor = ArgumentCaptor.forClass(TreeSet.class);
+        verify(mockPlacementStateManager).saveOwnership(captor.capture());
+        TreeSet<ServerLoad> serverLoads = (TreeSet<ServerLoad>) captor.getValue();
+        ServerLoad next = serverLoads.first();
+        String serverPlacement = Await.result(leastLoadPlacementPolicy.placeStream("newstream1"));
+        assertEquals(next.getServer(), serverPlacement);
+    }
+
+    @Test(timeout = 10000)
+    public void testCalculateUnequalWeight() throws Exception {
+        int numSevers = new Random().nextInt(20) + 1;
+        int numStreams = new Random().nextInt(200) + 1;
+    /* 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);
+        LoadAppraiser mockLoadAppraiser = mock(LoadAppraiser.class);
+        when(mockLoadAppraiser.getStreamLoad(anyString())).then(new Answer<Future<StreamLoad>>() {
+            @Override
+            public Future<StreamLoad> answer(InvocationOnMock invocationOnMock) throws Throwable {
+                int load = new Random().nextInt(100000);
+                if (load > maxLoad.get()) {
+                    maxLoad.set(load);
+                }
+                return Future.value(new StreamLoad(invocationOnMock.getArguments()[0].toString(), load));
+            }
+        });
+        LeastLoadPlacementPolicy leastLoadPlacementPolicy = new LeastLoadPlacementPolicy(
+            mockLoadAppraiser,
+            mockRoutingService,
+            mockNamespace,
+            null,
+            Duration.fromSeconds(600),
+            new NullStatsLogger());
+        TreeSet<ServerLoad> serverLoads =
+            Await.result(leastLoadPlacementPolicy.calculate(generateServers(numSevers), generateStreams(numStreams)));
+        long highestLoadSeen = Long.MIN_VALUE;
+        long lowestLoadSeen = Long.MAX_VALUE;
+        for (ServerLoad serverLoad : serverLoads) {
+            long load = serverLoad.getLoad();
+            if (load < lowestLoadSeen) {
+                lowestLoadSeen = load;
+            }
+            if (load > highestLoadSeen) {
+                highestLoadSeen = load;
+            }
+        }
+        assertTrue("Unexpected placement for " + numStreams + " streams to "
+                + numSevers + " servers : highest load = " + highestLoadSeen
+                + ", lowest load = " + lowestLoadSeen + ", max stream load = " + maxLoad.get(),
+            highestLoadSeen - lowestLoadSeen < maxLoad.get());
+    }
+
+    private Set<SocketAddress> generateSocketAddresses(int num) {
+        LinkedHashSet<SocketAddress> socketAddresses = new LinkedHashSet<SocketAddress>();
+        for (int i = 0; i < num; i++) {
+            socketAddresses.add(new InetSocketAddress(i));
+        }
+        return socketAddresses;
+    }
+
+    private Set<String> generateStreams(int num) {
+        LinkedHashSet<String> streams = new LinkedHashSet<String>();
+        for (int i = 0; i < num; i++) {
+            streams.add("stream_" + i);
+        }
+        return streams;
+    }
+
+    private Set<String> generateServers(int num) {
+        LinkedHashSet<String> servers = new LinkedHashSet<String>();
+        for (int i = 0; i < num; i++) {
+            servers.add("server_" + i);
+        }
+        return servers;
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestServerLoad.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestServerLoad.java
new file mode 100644
index 0000000..5bd234f
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestServerLoad.java
@@ -0,0 +1,50 @@
+/**
+ * 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.service.placement;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link ServerLoad}.
+ */
+public class TestServerLoad {
+
+    @Test(timeout = 60000)
+    public void testSerializeDeserialize() throws IOException {
+        final ServerLoad serverLoad = new ServerLoad("th1s1s@s3rv3rn@m3");
+        for (int i = 0; i < 20; i++) {
+            serverLoad.addStream(new StreamLoad("stream-" + i, i));
+        }
+        assertEquals(serverLoad, ServerLoad.deserialize(serverLoad.serialize()));
+    }
+
+    @Test(timeout = 60000)
+    public void testGetLoad() throws IOException {
+        final ServerLoad serverLoad = new ServerLoad("th1s1s@s3rv3rn@m3");
+        assertEquals(0, serverLoad.getLoad());
+        serverLoad.addStream(new StreamLoad("stream-" + 1, 3));
+        assertEquals(3, serverLoad.getLoad());
+        serverLoad.addStream(new StreamLoad("stream-" + 2, 7));
+        assertEquals(10, serverLoad.getLoad());
+        serverLoad.addStream(new StreamLoad("stream-" + 3, 1));
+        assertEquals(11, serverLoad.getLoad());
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestStreamLoad.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestStreamLoad.java
new file mode 100644
index 0000000..36a6fed
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestStreamLoad.java
@@ -0,0 +1,37 @@
+/**
+ * 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.service.placement;
+
+import static org.junit.Assert.assertEquals;
+
+import java.io.IOException;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link StreamLoad}.
+ */
+public class TestStreamLoad {
+
+    @Test(timeout = 10000)
+    public void testSerializeDeserialize() throws IOException {
+        final String streamName = "aHellaRandomStreamName";
+        final int load = 1337;
+        final StreamLoad streamLoad = new StreamLoad(streamName, load);
+        assertEquals(streamLoad, StreamLoad.deserialize(streamLoad.serialize()));
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestZKPlacementStateManager.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestZKPlacementStateManager.java
new file mode 100644
index 0000000..07ec5a5
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/placement/TestZKPlacementStateManager.java
@@ -0,0 +1,136 @@
+/**
+ * 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.service.placement;
+
+import static org.apache.distributedlog.LocalDLMEmulator.DLOG_NAMESPACE;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import java.io.IOException;
+import java.net.URI;
+import java.util.SortedSet;
+import java.util.TreeSet;
+import java.util.concurrent.LinkedBlockingQueue;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.apache.curator.test.TestingServer;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link ZKPlacementStateManager}.
+ */
+public class TestZKPlacementStateManager {
+    private TestingServer zkTestServer;
+    private String zkServers;
+    private URI uri;
+    private ZKPlacementStateManager zkPlacementStateManager;
+
+    @Before
+    public void startZookeeper() throws Exception {
+        zkTestServer = new TestingServer(2181);
+        zkServers = "127.0.0.1:2181";
+        uri = new URI("distributedlog-bk://" + zkServers + DLOG_NAMESPACE + "/bknamespace");
+        zkPlacementStateManager =
+            new ZKPlacementStateManager(uri, new DistributedLogConfiguration(), NullStatsLogger.INSTANCE);
+    }
+
+    @Test(timeout = 60000)
+    public void testSaveLoad() throws Exception {
+        TreeSet<ServerLoad> ownerships = new TreeSet<ServerLoad>();
+        zkPlacementStateManager.saveOwnership(ownerships);
+        SortedSet<ServerLoad> loadedOwnerships = zkPlacementStateManager.loadOwnership();
+        assertEquals(ownerships, loadedOwnerships);
+
+        ownerships.add(new ServerLoad("emptyServer"));
+        zkPlacementStateManager.saveOwnership(ownerships);
+        loadedOwnerships = zkPlacementStateManager.loadOwnership();
+        assertEquals(ownerships, loadedOwnerships);
+
+        ServerLoad sl1 = new ServerLoad("server1");
+        sl1.addStream(new StreamLoad("stream1", 3));
+        sl1.addStream(new StreamLoad("stream2", 4));
+        ServerLoad sl2 = new ServerLoad("server2");
+        sl2.addStream(new StreamLoad("stream3", 1));
+        ownerships.add(sl1);
+        ownerships.add(sl2);
+        zkPlacementStateManager.saveOwnership(ownerships);
+        loadedOwnerships = zkPlacementStateManager.loadOwnership();
+        assertEquals(ownerships, loadedOwnerships);
+
+        loadedOwnerships.remove(sl1);
+        zkPlacementStateManager.saveOwnership(ownerships);
+        loadedOwnerships = zkPlacementStateManager.loadOwnership();
+        assertEquals(ownerships, loadedOwnerships);
+    }
+
+    private TreeSet<ServerLoad> waitForServerLoadsNotificationAsc(
+        LinkedBlockingQueue<TreeSet<ServerLoad>> notificationQueue,
+        int expectedNumServerLoads) throws InterruptedException {
+        TreeSet<ServerLoad> notification = notificationQueue.take();
+        assertNotNull(notification);
+        while (notification.size() < expectedNumServerLoads) {
+            notification = notificationQueue.take();
+        }
+        assertEquals(expectedNumServerLoads, notification.size());
+        return notification;
+    }
+
+    @Test(timeout = 60000)
+    public void testWatchIndefinitely() throws Exception {
+        TreeSet<ServerLoad> ownerships = new TreeSet<ServerLoad>();
+        ownerships.add(new ServerLoad("server1"));
+        final LinkedBlockingQueue<TreeSet<ServerLoad>> serverLoadNotifications =
+            new LinkedBlockingQueue<TreeSet<ServerLoad>>();
+        PlacementStateManager.PlacementCallback callback = new PlacementStateManager.PlacementCallback() {
+            @Override
+            public void callback(TreeSet<ServerLoad> serverLoads) {
+                serverLoadNotifications.add(serverLoads);
+            }
+        };
+        zkPlacementStateManager.saveOwnership(ownerships); // need to initialize the zk path before watching
+        zkPlacementStateManager.watch(callback);
+        // cannot verify the callback here as it may call before the verify is called
+
+        zkPlacementStateManager.saveOwnership(ownerships);
+        assertEquals(ownerships, waitForServerLoadsNotificationAsc(serverLoadNotifications, 1));
+
+        ServerLoad server2 = new ServerLoad("server2");
+        server2.addStream(new StreamLoad("hella-important-stream", 415));
+        ownerships.add(server2);
+        zkPlacementStateManager.saveOwnership(ownerships);
+        assertEquals(ownerships, waitForServerLoadsNotificationAsc(serverLoadNotifications, 2));
+    }
+
+    @Test(timeout = 60000)
+    public void testZkFormatting() throws Exception {
+        final String server = "host/10.0.0.0:31351";
+        final String zkFormattedServer = "host--10.0.0.0:31351";
+        URI uri = new URI("distributedlog-bk://" + zkServers + DLOG_NAMESPACE + "/bknamespace");
+        ZKPlacementStateManager zkPlacementStateManager =
+            new ZKPlacementStateManager(uri, new DistributedLogConfiguration(), NullStatsLogger.INSTANCE);
+        assertEquals(zkFormattedServer, zkPlacementStateManager.serverToZkFormat(server));
+        assertEquals(server, zkPlacementStateManager.zkFormatToServer(zkFormattedServer));
+    }
+
+    @After
+    public void stopZookeeper() throws IOException {
+        zkTestServer.stop();
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java
new file mode 100644
index 0000000..56e9483
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamManager.java
@@ -0,0 +1,135 @@
+/**
+ * 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.service.stream;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.verify;
+import static org.mockito.Mockito.when;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+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;
+import com.twitter.util.Await;
+import java.util.concurrent.ScheduledExecutorService;
+import java.util.concurrent.ScheduledThreadPoolExecutor;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Test Case for StreamManager.
+ */
+public class TestStreamManager {
+
+    @Rule
+    public TestName testName = new TestName();
+
+    ScheduledExecutorService mockExecutorService = mock(ScheduledExecutorService.class);
+
+    @Test(timeout = 60000)
+    public void testCollectionMethods() throws Exception {
+        Stream mockStream = mock(Stream.class);
+        when(mockStream.getStreamName()).thenReturn("stream1");
+        when(mockStream.getPartition()).thenReturn(new Partition("stream1", 0));
+        StreamFactory mockStreamFactory = mock(StreamFactory.class);
+        StreamPartitionConverter mockPartitionConverter = mock(StreamPartitionConverter.class);
+        StreamConfigProvider mockStreamConfigProvider = mock(StreamConfigProvider.class);
+        when(mockStreamFactory.create(
+                (String) any(),
+                (DynamicDistributedLogConfiguration) any(),
+                (StreamManager) any())).thenReturn(mockStream);
+        StreamManager streamManager = new StreamManagerImpl(
+                "",
+                new DistributedLogConfiguration(),
+                mockExecutorService,
+                mockStreamFactory,
+                mockPartitionConverter,
+                mockStreamConfigProvider,
+                mock(DistributedLogNamespace.class));
+
+        assertFalse(streamManager.isAcquired("stream1"));
+        assertEquals(0, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+
+        streamManager.notifyAcquired(mockStream);
+        assertTrue(streamManager.isAcquired("stream1"));
+        assertEquals(1, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+
+        streamManager.notifyReleased(mockStream);
+        assertFalse(streamManager.isAcquired("stream1"));
+        assertEquals(0, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+
+        streamManager.notifyAcquired(mockStream);
+        assertTrue(streamManager.isAcquired("stream1"));
+        assertEquals(1, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+
+        streamManager.notifyAcquired(mockStream);
+        assertTrue(streamManager.isAcquired("stream1"));
+        assertEquals(1, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+
+        streamManager.notifyReleased(mockStream);
+        assertFalse(streamManager.isAcquired("stream1"));
+        assertEquals(0, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+
+        streamManager.notifyReleased(mockStream);
+        assertFalse(streamManager.isAcquired("stream1"));
+        assertEquals(0, streamManager.numAcquired());
+        assertEquals(0, streamManager.numCached());
+    }
+
+    @Test(timeout = 60000)
+    public void testCreateStream() throws Exception {
+        Stream mockStream = mock(Stream.class);
+        final String streamName = "stream1";
+        when(mockStream.getStreamName()).thenReturn(streamName);
+        StreamFactory mockStreamFactory = mock(StreamFactory.class);
+        StreamPartitionConverter mockPartitionConverter = mock(StreamPartitionConverter.class);
+        StreamConfigProvider mockStreamConfigProvider = mock(StreamConfigProvider.class);
+        when(mockStreamFactory.create(
+            (String) any(),
+            (DynamicDistributedLogConfiguration) any(),
+            (StreamManager) any())
+        ).thenReturn(mockStream);
+        DistributedLogNamespace dlNamespace = mock(DistributedLogNamespace.class);
+        ScheduledExecutorService executorService = new ScheduledThreadPoolExecutor(1);
+
+        StreamManager streamManager = new StreamManagerImpl(
+                "",
+                new DistributedLogConfiguration(),
+                executorService,
+                mockStreamFactory,
+                mockPartitionConverter,
+                mockStreamConfigProvider,
+                dlNamespace);
+
+        assertTrue(Await.ready(streamManager.createStreamAsync(streamName)).isReturn());
+        verify(dlNamespace).createLog(streamName);
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.java
new file mode 100644
index 0000000..a18fda1
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/TestStreamOp.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.service.stream;
+
+import static org.junit.Assert.assertEquals;
+import static org.mockito.Mockito.any;
+import static org.mockito.Mockito.mock;
+import static org.mockito.Mockito.when;
+
+import org.apache.distributedlog.AsyncLogWriter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecord;
+import org.apache.distributedlog.acl.DefaultAccessControlManager;
+import org.apache.distributedlog.exceptions.InternalServerException;
+import org.apache.distributedlog.service.ResponseUtils;
+import org.apache.distributedlog.service.config.ServerConfiguration;
+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 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;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TestName;
+
+/**
+ * Test Case for StreamOps.
+ */
+public class TestStreamOp {
+
+    @Rule
+    public TestName testName = new TestName();
+
+    private WriteOp getWriteOp() {
+        SettableFeature disabledFeature = new SettableFeature("", 0);
+        return new WriteOp("test",
+            ByteBuffer.wrap("test".getBytes()),
+            new NullStatsLogger(),
+            new NullStatsLogger(),
+            new IdentityStreamPartitionConverter(),
+            new ServerConfiguration(),
+            (byte) 0,
+            null,
+            false,
+            disabledFeature,
+            DefaultAccessControlManager.INSTANCE);
+    }
+
+    @Test(timeout = 60000)
+    public void testResponseFailedTwice() throws Exception {
+        WriteOp writeOp = getWriteOp();
+        writeOp.fail(new InternalServerException("test1"));
+        writeOp.fail(new InternalServerException("test2"));
+
+        WriteResponse response = Await.result(writeOp.result());
+        assertEquals(StatusCode.INTERNAL_SERVER_ERROR, response.getHeader().getCode());
+        assertEquals(ResponseUtils.exceptionToHeader(new InternalServerException("test1")), response.getHeader());
+    }
+
+    @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.getStreamName()).thenReturn("test");
+        WriteOp writeOp = getWriteOp();
+        writeOp.execute(writer, new Sequencer() {
+            public long nextId() {
+                return 0;
+            }
+        }, new Object());
+        writeOp.fail(new InternalServerException("test2"));
+
+        WriteResponse response = Await.result(writeOp.result());
+        assertEquals(StatusCode.SUCCESS, response.getHeader().getCode());
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
new file mode 100644
index 0000000..431bfa4
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/stream/limiter/TestServiceRequestLimiter.java
@@ -0,0 +1,301 @@
+/**
+ * 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.service.stream.limiter;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.config.ConcurrentConstConfiguration;
+import org.apache.distributedlog.config.DynamicDistributedLogConfiguration;
+import org.apache.distributedlog.exceptions.OverCapacityException;
+import org.apache.distributedlog.limiter.ChainedRequestLimiter;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter.CostFunction;
+import org.apache.distributedlog.limiter.ComposableRequestLimiter.OverlimitFunction;
+import org.apache.distributedlog.limiter.GuavaRateLimiter;
+import org.apache.distributedlog.limiter.RateLimiter;
+import org.apache.distributedlog.limiter.RequestLimiter;
+import java.util.concurrent.atomic.AtomicInteger;
+import org.apache.bookkeeper.feature.SettableFeature;
+import org.apache.bookkeeper.stats.NullStatsLogger;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link ServiceRequestLimiter}.
+ */
+public class TestServiceRequestLimiter {
+
+    /**
+     * Mock Request.
+     */
+    class MockRequest {
+        int size;
+        MockRequest() {
+            this(1);
+        }
+        MockRequest(int size) {
+            this.size = size;
+        }
+        int getSize() {
+            return size;
+        }
+    }
+
+    /**
+     * Mock request limiter.
+     */
+    class MockRequestLimiter implements RequestLimiter<MockRequest> {
+        public void apply(MockRequest request) {
+        }
+    }
+
+    /**
+     * Counter based limiter.
+     */
+    static class CounterLimiter implements RateLimiter {
+        final int limit;
+        int count;
+
+        public CounterLimiter(int limit) {
+            this.limit = limit;
+            this.count = 0;
+        }
+
+        @Override
+        public boolean acquire(int permits) {
+            if (++count > limit) {
+                return false;
+            }
+            return true;
+        }
+    }
+
+    /**
+     * Mock hard request limiter.
+     */
+    class MockHardRequestLimiter implements RequestLimiter<MockRequest> {
+
+        RequestLimiter<MockRequest> limiter;
+        int limitHitCount;
+
+        MockHardRequestLimiter(int limit) {
+            this(GuavaRateLimiter.of(limit));
+        }
+
+        MockHardRequestLimiter(RateLimiter limiter) {
+            this.limiter = new ComposableRequestLimiter<MockRequest>(
+                limiter,
+                new OverlimitFunction<MockRequest>() {
+                    public void apply(MockRequest request) throws OverCapacityException {
+                        limitHitCount++;
+                        throw new OverCapacityException("Limit exceeded");
+                    }
+                },
+                new CostFunction<MockRequest>() {
+                    public int apply(MockRequest request) {
+                        return request.getSize();
+                    }
+                },
+                NullStatsLogger.INSTANCE);
+        }
+
+        @Override
+        public void apply(MockRequest op) throws OverCapacityException {
+            limiter.apply(op);
+        }
+
+        public int getLimitHitCount() {
+            return limitHitCount;
+        }
+    }
+
+    /**
+     * Mock soft request limiter.
+     */
+    class MockSoftRequestLimiter implements RequestLimiter<MockRequest> {
+
+        RequestLimiter<MockRequest> limiter;
+        int limitHitCount;
+
+        MockSoftRequestLimiter(int limit) {
+            this(GuavaRateLimiter.of(limit));
+        }
+
+        MockSoftRequestLimiter(RateLimiter limiter) {
+            this.limiter = new ComposableRequestLimiter<MockRequest>(
+                limiter,
+                new OverlimitFunction<MockRequest>() {
+                    public void apply(MockRequest request) throws OverCapacityException {
+                        limitHitCount++;
+                    }
+                },
+                new CostFunction<MockRequest>() {
+                    public int apply(MockRequest request) {
+                        return request.getSize();
+                    }
+                },
+                NullStatsLogger.INSTANCE);
+        }
+
+        @Override
+        public void apply(MockRequest op) throws OverCapacityException {
+            limiter.apply(op);
+        }
+
+        public int getLimitHitCount() {
+            return limitHitCount;
+        }
+    }
+
+    @Test(timeout = 60000)
+    public void testDynamicLimiter() throws Exception {
+        final AtomicInteger id = new AtomicInteger(0);
+        final DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(
+                new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
+        DynamicRequestLimiter<MockRequest> limiter = new DynamicRequestLimiter<MockRequest>(
+                dynConf, NullStatsLogger.INSTANCE, new SettableFeature("", 0)) {
+            @Override
+            public RequestLimiter<MockRequest> build() {
+                id.getAndIncrement();
+                return new MockRequestLimiter();
+            }
+        };
+        limiter.initialize();
+        assertEquals(1, id.get());
+        dynConf.setProperty("test1", 1);
+        assertEquals(2, id.get());
+        dynConf.setProperty("test2", 2);
+        assertEquals(3, id.get());
+    }
+
+    @Test(timeout = 60000)
+    public void testDynamicLimiterWithDisabledFeature() throws Exception {
+        final DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(
+                new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
+        final MockSoftRequestLimiter rateLimiter = new MockSoftRequestLimiter(0);
+        final SettableFeature disabledFeature = new SettableFeature("", 0);
+        DynamicRequestLimiter<MockRequest> limiter = new DynamicRequestLimiter<MockRequest>(
+                dynConf, NullStatsLogger.INSTANCE, disabledFeature) {
+            @Override
+            public RequestLimiter<MockRequest> build() {
+                return rateLimiter;
+            }
+        };
+        limiter.initialize();
+        assertEquals(0, rateLimiter.getLimitHitCount());
+
+        // Not disabled, rate limiter was invoked
+        limiter.apply(new MockRequest(Integer.MAX_VALUE));
+        assertEquals(1, rateLimiter.getLimitHitCount());
+
+        // Disabled, rate limiter not invoked
+        disabledFeature.set(1);
+        limiter.apply(new MockRequest(Integer.MAX_VALUE));
+        assertEquals(1, rateLimiter.getLimitHitCount());
+    }
+
+    @Test(timeout = 60000)
+    public void testDynamicLimiterWithException() throws Exception {
+        final AtomicInteger id = new AtomicInteger(0);
+        final DynamicDistributedLogConfiguration dynConf = new DynamicDistributedLogConfiguration(
+                new ConcurrentConstConfiguration(new DistributedLogConfiguration()));
+        DynamicRequestLimiter<MockRequest> limiter = new DynamicRequestLimiter<MockRequest>(
+                dynConf, NullStatsLogger.INSTANCE, new SettableFeature("", 0)) {
+            @Override
+            public RequestLimiter<MockRequest> build() {
+                if (id.incrementAndGet() >= 2) {
+                    throw new RuntimeException("exception in dynamic limiter build()");
+                }
+                return new MockRequestLimiter();
+            }
+        };
+        limiter.initialize();
+        assertEquals(1, id.get());
+        try {
+            dynConf.setProperty("test1", 1);
+            fail("should have thrown on config failure");
+        } catch (RuntimeException ex) {
+        }
+        assertEquals(2, id.get());
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceRequestLimiter() throws Exception {
+        MockHardRequestLimiter limiter = new MockHardRequestLimiter(new CounterLimiter(1));
+        limiter.apply(new MockRequest());
+        try {
+            limiter.apply(new MockRequest());
+        } catch (OverCapacityException ex) {
+        }
+        assertEquals(1, limiter.getLimitHitCount());
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceRequestLimiterWithDefaultRate() throws Exception {
+        MockHardRequestLimiter limiter = new MockHardRequestLimiter(-1);
+        limiter.apply(new MockRequest(Integer.MAX_VALUE));
+        limiter.apply(new MockRequest(Integer.MAX_VALUE));
+        assertEquals(0, limiter.getLimitHitCount());
+    }
+
+    @Test(timeout = 60000)
+    public void testServiceRequestLimiterWithZeroRate() throws Exception {
+        MockHardRequestLimiter limiter = new MockHardRequestLimiter(0);
+        try {
+            limiter.apply(new MockRequest(1));
+            fail("should have failed with overcap");
+        } catch (OverCapacityException ex) {
+        }
+        assertEquals(1, limiter.getLimitHitCount());
+    }
+
+    @Test(timeout = 60000)
+    public void testChainedServiceRequestLimiter() throws Exception {
+        MockSoftRequestLimiter softLimiter = new MockSoftRequestLimiter(new CounterLimiter(1));
+        MockHardRequestLimiter hardLimiter = new MockHardRequestLimiter(new CounterLimiter(3));
+
+        RequestLimiter<MockRequest> limiter =
+                new ChainedRequestLimiter.Builder<MockRequest>()
+                .addLimiter(softLimiter)
+                .addLimiter(hardLimiter)
+                .build();
+
+        assertEquals(0, softLimiter.getLimitHitCount());
+        assertEquals(0, hardLimiter.getLimitHitCount());
+
+        limiter.apply(new MockRequest());
+        assertEquals(0, softLimiter.getLimitHitCount());
+        assertEquals(0, hardLimiter.getLimitHitCount());
+
+        limiter.apply(new MockRequest());
+        assertEquals(1, softLimiter.getLimitHitCount());
+        assertEquals(0, hardLimiter.getLimitHitCount());
+
+        limiter.apply(new MockRequest());
+        assertEquals(2, softLimiter.getLimitHitCount());
+        assertEquals(0, hardLimiter.getLimitHitCount());
+
+        try {
+            limiter.apply(new MockRequest());
+        } catch (OverCapacityException ex) {
+        }
+        assertEquals(3, softLimiter.getLimitHitCount());
+        assertEquals(1, hardLimiter.getLimitHitCount());
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java
new file mode 100644
index 0000000..15a0753
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestDelimiterStreamPartitionConverter.java
@@ -0,0 +1,52 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.distributedlog.service.streamset;
+
+import static org.junit.Assert.assertEquals;
+
+import org.junit.Test;
+
+/**
+ * Test Cases for {@link DelimiterStreamPartitionConverter}.
+ */
+public class TestDelimiterStreamPartitionConverter {
+
+    @Test(timeout = 20000)
+    public void testNormalStream() throws Exception {
+        StreamPartitionConverter converter = new DelimiterStreamPartitionConverter();
+        assertEquals(new Partition("distributedlog-smoketest", 1),
+                converter.convert("distributedlog-smoketest_1"));
+        assertEquals(new Partition("distributedlog-smoketest-", 1),
+                converter.convert("distributedlog-smoketest-_1"));
+        assertEquals(new Partition("distributedlog-smoketest", 1),
+                converter.convert("distributedlog-smoketest_000001"));
+    }
+
+    private void assertIdentify(String streamName, StreamPartitionConverter converter) {
+        assertEquals(new Partition(streamName, 0), converter.convert(streamName));
+    }
+
+    @Test(timeout = 20000)
+    public void testUnknownStream() throws Exception {
+        StreamPartitionConverter converter = new DelimiterStreamPartitionConverter();
+        assertIdentify("test1", converter);
+        assertIdentify("test1-000001", converter);
+        assertIdentify("test1_test1_000001", converter);
+        assertIdentify("test1_test1", converter);
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.java
new file mode 100644
index 0000000..1a5d8d3
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestIdentityStreamPartitionConverter.java
@@ -0,0 +1,43 @@
+/**
+ * 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.service.streamset;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+/**
+ * Test Case for {@link IdentityStreamPartitionConverter}.
+ */
+public class TestIdentityStreamPartitionConverter {
+
+    @Test(timeout = 20000)
+    public void testIdentityConverter() {
+        String streamName = "test-identity-converter";
+
+        IdentityStreamPartitionConverter converter =
+                new IdentityStreamPartitionConverter();
+
+        Partition p0 = converter.convert(streamName);
+        assertEquals(new Partition(streamName, 0), p0);
+
+        Partition p1 = converter.convert(streamName);
+        assertTrue(p0 == p1);
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestPartitionMap.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestPartitionMap.java
new file mode 100644
index 0000000..b6e5ff3
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/streamset/TestPartitionMap.java
@@ -0,0 +1,68 @@
+/**
+ * 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.service.streamset;
+
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.junit.Test;
+
+/**
+ * Test {@link PartitionMap}.
+ */
+public class TestPartitionMap {
+
+    @Test(timeout = 20000)
+    public void testAddPartitionNegativeMax() {
+        PartitionMap map = new PartitionMap();
+        for (int i = 0; i < 20; i++) {
+            assertTrue(map.addPartition(new Partition("test", i), -1));
+        }
+    }
+
+    @Test(timeout = 20000)
+    public void testAddPartitionMultipleTimes() {
+        PartitionMap map = new PartitionMap();
+        for (int i = 0; i < 20; i++) {
+            assertTrue(map.addPartition(new Partition("test", 0), 3));
+        }
+    }
+
+    @Test(timeout = 20000)
+    public void testAddPartition() {
+        PartitionMap map = new PartitionMap();
+        for (int i = 0; i < 3; i++) {
+            assertTrue(map.addPartition(new Partition("test", i), 3));
+        }
+        for (int i = 3; i < 20; i++) {
+            assertFalse(map.addPartition(new Partition("test", i), 3));
+        }
+    }
+
+    @Test(timeout = 20000)
+    public void testRemovePartition() {
+        PartitionMap map = new PartitionMap();
+        for (int i = 0; i < 3; i++) {
+            assertTrue(map.addPartition(new Partition("test", i), 3));
+        }
+        assertFalse(map.addPartition(new Partition("test", 3), 3));
+        assertFalse(map.removePartition(new Partition("test", 3)));
+        assertTrue(map.removePartition(new Partition("test", 0)));
+        assertTrue(map.addPartition(new Partition("test", 3), 3));
+    }
+}
diff --git a/distributedlog-service/src/test/java/org/apache/distributedlog/service/utils/TestServerUtils.java b/distributedlog-service/src/test/java/org/apache/distributedlog/service/utils/TestServerUtils.java
new file mode 100644
index 0000000..2853df1
--- /dev/null
+++ b/distributedlog-service/src/test/java/org/apache/distributedlog/service/utils/TestServerUtils.java
@@ -0,0 +1,41 @@
+/**
+ * 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.service.utils;
+
+import static org.junit.Assert.assertEquals;
+
+import java.net.InetAddress;
+import org.junit.Test;
+
+/**
+ * Test Case for {@link ServerUtils}.
+ */
+public class TestServerUtils {
+
+    @Test(timeout = 6000)
+    public void testGetLedgerAllocatorPoolName() throws Exception {
+        int region = 123;
+        int shard = 999;
+        String hostname = InetAddress.getLocalHost().getHostAddress();
+        assertEquals("allocator_0123_0000000999",
+            ServerUtils.getLedgerAllocatorPoolName(region, shard, false));
+        assertEquals("allocator_0123_" + hostname,
+            ServerUtils.getLedgerAllocatorPoolName(region, shard, true));
+    }
+
+}
diff --git a/distributedlog-tutorials/distributedlog-basic/basic-1.md b/distributedlog-tutorials/distributedlog-basic/basic-1.md
index d5c4fc5..86ec9ed 100644
--- a/distributedlog-tutorials/distributedlog-basic/basic-1.md
+++ b/distributedlog-tutorials/distributedlog-basic/basic-1.md
@@ -102,15 +102,15 @@
 3.  Tailing the stream using `TailReader` to wait for new records.
     ```
         // Tailing Stream `basic-stream-1`
-        // runner run com.twitter.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-1
+        // runner run org.apache.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-1
     ```
 
 4.  Run the example to write records to the stream in a console.
     ```
         // Write Records into Stream `basic-stream-1`
-        // runner run com.twitter.distributedlog.basic.ConsoleWriter ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.ConsoleWriter distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-1
+        // runner run org.apache.distributedlog.basic.ConsoleWriter ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.ConsoleWriter distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-1
     ```
 
 5.  Example output from `ConsoleWriter` and `TailReader`.
@@ -135,9 +135,9 @@
     `ConsoleWriter` is still holding lock on writing to stream `basic-stream-1`.
     ```
         Opening log stream basic-stream-1
-        Exception in thread "main" com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException: LockPath - /messaging/distributedlog/basic-stream-1/<default>/lock: Lock acquisition failed, the current owner is console-writer
-            at com.twitter.distributedlog.lock.ZKSessionLock$8.apply(ZKSessionLock.java:570)
-            at com.twitter.distributedlog.lock.ZKSessionLock$8.apply(ZKSessionLock.java:567)
+        Exception in thread "main" org.apache.distributedlog.exceptions.OwnershipAcquireFailedException: LockPath - /messaging/distributedlog/basic-stream-1/<default>/lock: Lock acquisition failed, the current owner is console-writer
+            at org.apache.distributedlog.lock.ZKSessionLock$8.apply(ZKSessionLock.java:570)
+            at org.apache.distributedlog.lock.ZKSessionLock$8.apply(ZKSessionLock.java:567)
             at com.twitter.util.Future$$anonfun$map$1$$anonfun$apply$8.apply(Future.scala:1041)
             at com.twitter.util.Try$.apply(Try.scala:13)
             at com.twitter.util.Future$.apply(Future.scala:132)
@@ -156,7 +156,7 @@
             at com.twitter.util.Promise.updateIfEmpty(Promise.scala:716)
             at com.twitter.util.Promise.update(Promise.scala:694)
             at com.twitter.util.Promise.setValue(Promise.scala:670)
-            at com.twitter.distributedlog.lock.ZKSessionLock$9.safeRun(ZKSessionLock.java:622)
+            at org.apache.distributedlog.lock.ZKSessionLock$9.safeRun(ZKSessionLock.java:622)
             at org.apache.bookkeeper.util.SafeRunnable.run(SafeRunnable.java:31)
             at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
             at java.util.concurrent.FutureTask.run(FutureTask.java:262)
diff --git a/distributedlog-tutorials/distributedlog-basic/basic-2.md b/distributedlog-tutorials/distributedlog-basic/basic-2.md
index cf443fb..0459bbf 100644
--- a/distributedlog-tutorials/distributedlog-basic/basic-2.md
+++ b/distributedlog-tutorials/distributedlog-basic/basic-2.md
@@ -74,7 +74,7 @@
 2.  Start the write proxy, listening on port 8000.
     ```
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
     ```
 
 3.  Create the stream under the distributedlog uri.
@@ -88,15 +88,15 @@
 4.  Tailing the stream using `TailReader` to wait for new records.
     ```
         // Tailing Stream `basic-stream-2`
-        // runner run com.twitter.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-2
+        // runner run org.apache.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-2
     ```
 
 6.  Run the example to write records to the stream in a console.
     ```
         // Write Records into Stream `basic-stream-2`
-        // runner run com.twitter.distributedlog.basic.ConsoleProxyWriter ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.ConsoleProxyWriter 'inet!127.0.0.1:8000' basic-stream-2
+        // runner run org.apache.distributedlog.basic.ConsoleProxyWriter ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.ConsoleProxyWriter 'inet!127.0.0.1:8000' basic-stream-2
     ```
 
 7.  Example output from `ConsoleProxyWriter` and `TailReader`.
diff --git a/distributedlog-tutorials/distributedlog-basic/basic-3.md b/distributedlog-tutorials/distributedlog-basic/basic-3.md
index 60f29d1..56972bf 100644
--- a/distributedlog-tutorials/distributedlog-basic/basic-3.md
+++ b/distributedlog-tutorials/distributedlog-basic/basic-3.md
@@ -107,7 +107,7 @@
 2.  Start the write proxy, listening on port 8000.
     ```
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
     ```
 
 3.  Create multi streams under the distributedlog uri.
@@ -121,15 +121,15 @@
 4.  Tailing multi streams using `MultiReader` to wait for new records.
     ```
         // Tailing Stream `basic-stream-{3-7}`
-        // runner run com.twitter.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[,${stream}]
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-3,basic-stream-4,basic-stream-5,basic-stream-6,basic-stream-7
+        // runner run org.apache.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[,${stream}]
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-3,basic-stream-4,basic-stream-5,basic-stream-6,basic-stream-7
     ```
 
 6.  Run the example to write records to the multi streams in a console.
     ```
         // Write Records into Stream `basic-stream-{3-7}`
-        // runner run com.twitter.distributedlog.basic.ConsoleProxyMultiWriter ${distributedlog-uri} ${stream}[,${stream}]
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.ConsoleProxyMultiWriter 'inet!127.0.0.1:8000' basic-stream-3,basic-stream-4,basic-stream-5,basic-stream-6,basic-stream-7
+        // runner run org.apache.distributedlog.basic.ConsoleProxyMultiWriter ${distributedlog-uri} ${stream}[,${stream}]
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.ConsoleProxyMultiWriter 'inet!127.0.0.1:8000' basic-stream-3,basic-stream-4,basic-stream-5,basic-stream-6,basic-stream-7
     ```
 
 7.  Example output from `ConsoleProxyMultiWriter` and `MultiReader`.
diff --git a/distributedlog-tutorials/distributedlog-basic/basic-4.md b/distributedlog-tutorials/distributedlog-basic/basic-4.md
index f394640..684bdeb 100644
--- a/distributedlog-tutorials/distributedlog-basic/basic-4.md
+++ b/distributedlog-tutorials/distributedlog-basic/basic-4.md
@@ -92,7 +92,7 @@
 2.  Start the write proxy, listening on port 8000.
     ```
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
     ```
 
 3.  Create the stream under the distributedlog uri.
@@ -106,15 +106,15 @@
 4.  Tailing the stream using `TailReader` to wait for new records.
     ```
         // Tailing Stream `basic-stream-8`
-        // runner run com.twitter.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-8
+        // runner run org.apache.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-8
     ```
 
 6.  Run the example to write multiple records to the stream.
     ```
         // Write Records into Stream `basic-stream-8`
-        // runner run com.twitter.distributedlog.basic.AtomicWriter ${distributedlog-uri} ${stream} ${message}[, ${message}]
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.AtomicWriter 'inet!127.0.0.1:8000' basic-stream-8 "message-1" "message-2" "message-3" "message-4" "message-5"
+        // runner run org.apache.distributedlog.basic.AtomicWriter ${distributedlog-uri} ${stream} ${message}[, ${message}]
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.AtomicWriter 'inet!127.0.0.1:8000' basic-stream-8 "message-1" "message-2" "message-3" "message-4" "message-5"
     ```
 
 7.  Example output from `AtomicWriter` and `TailReader`.
diff --git a/distributedlog-tutorials/distributedlog-basic/basic-5.md b/distributedlog-tutorials/distributedlog-basic/basic-5.md
index 726ce76..684f228 100644
--- a/distributedlog-tutorials/distributedlog-basic/basic-5.md
+++ b/distributedlog-tutorials/distributedlog-basic/basic-5.md
@@ -90,7 +90,7 @@
 2.  Start the write proxy, listening on port 8000.
     ```
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
     ```
 
 3.  Create the stream under the distributedlog uri.
@@ -104,15 +104,15 @@
 4.  Tailing the stream using `TailReader` to wait for new records.
     ```
         // Tailing Stream `basic-stream-9`
-        // runner run com.twitter.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-9
+        // runner run org.apache.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-9
     ```
 
 6.  Run the example to write records to the stream in a console.
     ```
         // Write Records into Stream `basic-stream-9`
-        // runner run com.twitter.distributedlog.basic.ConsoleProxyWriter ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.ConsoleProxyWriter 'inet!127.0.0.1:8000' basic-stream-9
+        // runner run org.apache.distributedlog.basic.ConsoleProxyWriter ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.ConsoleProxyWriter 'inet!127.0.0.1:8000' basic-stream-9
     ```
 
 7.  Example output from `ConsoleProxyWriter` and `TailReader`.
diff --git a/distributedlog-tutorials/distributedlog-basic/basic-6.md b/distributedlog-tutorials/distributedlog-basic/basic-6.md
index a7f140d..3edd596 100644
--- a/distributedlog-tutorials/distributedlog-basic/basic-6.md
+++ b/distributedlog-tutorials/distributedlog-basic/basic-6.md
@@ -88,7 +88,7 @@
 2.  Start the write proxy, listening on port 8000.
     ```
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
     ```
 
 3.  Create the stream under the distributedlog uri.
@@ -102,15 +102,15 @@
 4.  Run the `RecordGenerator` to generate records.
     ```
         // Write Records into Stream `basic-stream-10` in 1 requests/second
-        // runner run com.twitter.distributedlog.basic.RecordGenerator ${distributedlog-uri} ${stream} ${rate}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.RecordGenerator 'inet!127.0.0.1:8000' basic-stream-10 1
+        // runner run org.apache.distributedlog.basic.RecordGenerator ${distributedlog-uri} ${stream} ${rate}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.RecordGenerator 'inet!127.0.0.1:8000' basic-stream-10 1
     ```
 
 5.  Rewind the stream using `StreamRewinder` to read records from 30 seconds ago
     ```
         // Rewind `basic-stream-10`
-        // runner run com.twitter.distributedlog.basic.StreamRewinder ${distributedlog-uri} ${stream} ${seconds-to-rewind}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.StreamRewinder distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-10  30
+        // runner run org.apache.distributedlog.basic.StreamRewinder ${distributedlog-uri} ${stream} ${seconds-to-rewind}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.StreamRewinder distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-10  30
     ```
 
 6.  Example output from `StreamRewinder`.
diff --git a/distributedlog-tutorials/distributedlog-basic/conf/log4j.properties b/distributedlog-tutorials/distributedlog-basic/conf/log4j.properties
index cc6486b..f3d3ad1 100644
--- a/distributedlog-tutorials/distributedlog-basic/conf/log4j.properties
+++ b/distributedlog-tutorials/distributedlog-basic/conf/log4j.properties
@@ -32,11 +32,11 @@
 log4j.logger.org.apache.bookkeeper=INFO
 
 # redirect executor output to executors.log since slow op warnings can be quite verbose
-log4j.logger.com.twitter.distributedlog.util.MonitoredFuturePool=INFO, Executors
-log4j.logger.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=INFO, Executors
+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.com.twitter.distributedlog.util.MonitoredFuturePool=false
-log4j.additivity.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=false
+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/com/twitter/distributedlog/basic/AtomicWriter.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/AtomicWriter.java
deleted file mode 100644
index 1f9cc9e..0000000
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/AtomicWriter.java
+++ /dev/null
@@ -1,107 +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 com.twitter.distributedlog.basic;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecordSet;
-import com.twitter.distributedlog.io.CompressionCodec.Type;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.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;
-
-/**
- * Write multiple record atomically
- */
-public class AtomicWriter {
-
-    private final static String HELP = "AtomicWriter <finagle-name> <stream> <message>[,<message>]";
-
-    public static void main(String[] args) throws Exception {
-        if (args.length < 3) {
-            System.out.println(HELP);
-            return;
-        }
-
-        String finagleNameStr = args[0];
-        String streamName = args[1];
-        String[] messages = new String[args.length - 2];
-        System.arraycopy(args, 2, messages, 0, messages.length);
-
-        DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
-                .clientId(ClientId$.MODULE$.apply("atomic-writer"))
-                .name("atomic-writer")
-                .thriftmux(true)
-                .finagleNameStr(finagleNameStr)
-                .build();
-
-        final LogRecordSet.Writer recordSetWriter = LogRecordSet.newWriter(16 * 1024, Type.NONE);
-        List<Future<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>() {
-                @Override
-                public void onFailure(Throwable cause) {
-                    System.out.println("Encountered error on writing data");
-                    cause.printStackTrace(System.err);
-                    Runtime.getRuntime().exit(0);
-                }
-
-                @Override
-                public void onSuccess(DLSN dlsn) {
-                    System.out.println("Write '" + message + "' as record " + dlsn);
-                }
-            });
-            recordSetWriter.writeRecord(msgBuf, writeFuture);
-            writeFutures.add(writeFuture);
-        }
-        FutureUtils.result(
-            client.writeRecordSet(streamName, recordSetWriter)
-                .addEventListener(new FutureEventListener<DLSN>() {
-                    @Override
-                    public void onFailure(Throwable cause) {
-                        recordSetWriter.abortTransmit(cause);
-                        System.out.println("Encountered error on writing data");
-                        cause.printStackTrace(System.err);
-                        Runtime.getRuntime().exit(0);
-                    }
-
-                    @Override
-                    public void onSuccess(DLSN dlsn) {
-                        recordSetWriter.completeTransmit(
-                                dlsn.getLogSegmentSequenceNo(),
-                                dlsn.getEntryId(),
-                                dlsn.getSlotId());
-                    }
-                })
-        );
-        FutureUtils.result(Future.collect(writeFutures));
-        client.close();
-    }
-}
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/ConsoleProxyMultiWriter.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/ConsoleProxyMultiWriter.java
deleted file mode 100644
index e5a52d0..0000000
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/ConsoleProxyMultiWriter.java
+++ /dev/null
@@ -1,91 +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 com.twitter.distributedlog.basic;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.client.DistributedLogMultiStreamWriter;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.FutureEventListener;
-import jline.ConsoleReader;
-import org.apache.commons.lang.StringUtils;
-
-import java.nio.ByteBuffer;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Writer write records from console
- */
-public class ConsoleProxyMultiWriter {
-
-    private final static String HELP = "ConsoleProxyWriter <finagle-name> <stream-1>[,<stream-2>,...,<stream-n>]";
-    private final static String PROMPT_MESSAGE = "[dlog] > ";
-
-    public static void main(String[] args) throws Exception {
-        if (2 != args.length) {
-            System.out.println(HELP);
-            return;
-        }
-
-        String finagleNameStr = args[0];
-        final String streamList = args[1];
-
-        DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
-                .clientId(ClientId$.MODULE$.apply("console-proxy-writer"))
-                .name("console-proxy-writer")
-                .thriftmux(true)
-                .finagleNameStr(finagleNameStr)
-                .build();
-        String[] streamNameList = StringUtils.split(streamList, ',');
-        DistributedLogMultiStreamWriter multiStreamWriter = DistributedLogMultiStreamWriter.newBuilder()
-                .streams(Lists.newArrayList(streamNameList))
-                .bufferSize(0)
-                .client(client)
-                .flushIntervalMs(0)
-                .firstSpeculativeTimeoutMs(10000)
-                .maxSpeculativeTimeoutMs(20000)
-                .requestTimeoutMs(50000)
-                .build();
-
-        ConsoleReader reader = new ConsoleReader();
-        String line;
-        while ((line = reader.readLine(PROMPT_MESSAGE)) != null) {
-            multiStreamWriter.write(ByteBuffer.wrap(line.getBytes(UTF_8)))
-                    .addEventListener(new FutureEventListener<DLSN>() {
-                        @Override
-                        public void onFailure(Throwable cause) {
-                            System.out.println("Encountered error on writing data");
-                            cause.printStackTrace(System.err);
-                            Runtime.getRuntime().exit(0);
-                        }
-
-                        @Override
-                        public void onSuccess(DLSN value) {
-                            // done
-                        }
-                    });
-        }
-
-        multiStreamWriter.close();
-        client.close();
-    }
-
-}
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/ConsoleProxyWriter.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/ConsoleProxyWriter.java
deleted file mode 100644
index d719015..0000000
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/ConsoleProxyWriter.java
+++ /dev/null
@@ -1,77 +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 com.twitter.distributedlog.basic;
-
-import com.twitter.distributedlog.*;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.FutureEventListener;
-import jline.ConsoleReader;
-
-import java.nio.ByteBuffer;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Writer write records from console
- */
-public class ConsoleProxyWriter {
-
-    private final static String HELP = "ConsoleProxyWriter <finagle-name> <string>";
-    private final static String PROMPT_MESSAGE = "[dlog] > ";
-
-    public static void main(String[] args) throws Exception {
-        if (2 != args.length) {
-            System.out.println(HELP);
-            return;
-        }
-
-        String finagleNameStr = args[0];
-        final String streamName = args[1];
-
-        DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
-                .clientId(ClientId$.MODULE$.apply("console-proxy-writer"))
-                .name("console-proxy-writer")
-                .thriftmux(true)
-                .finagleNameStr(finagleNameStr)
-                .build();
-
-        ConsoleReader reader = new ConsoleReader();
-        String line;
-        while ((line = reader.readLine(PROMPT_MESSAGE)) != null) {
-            client.write(streamName, ByteBuffer.wrap(line.getBytes(UTF_8)))
-                    .addEventListener(new FutureEventListener<DLSN>() {
-                        @Override
-                        public void onFailure(Throwable cause) {
-                            System.out.println("Encountered error on writing data");
-                            cause.printStackTrace(System.err);
-                            Runtime.getRuntime().exit(0);
-                        }
-
-                        @Override
-                        public void onSuccess(DLSN value) {
-                            // done
-                        }
-                    });
-        }
-
-        client.close();
-    }
-
-}
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/ConsoleWriter.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/ConsoleWriter.java
deleted file mode 100644
index 549e84a..0000000
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/ConsoleWriter.java
+++ /dev/null
@@ -1,101 +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 com.twitter.distributedlog.basic;
-
-import com.twitter.distributedlog.*;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
-import jline.ConsoleReader;
-
-import java.net.URI;
-import java.util.concurrent.TimeUnit;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Writer write records from console
- */
-public class ConsoleWriter {
-
-    private final static String HELP = "ConsoleWriter <uri> <string>";
-    private final static String PROMPT_MESSAGE = "[dlog] > ";
-
-    public static void main(String[] args) throws Exception {
-        if (2 != args.length) {
-            System.out.println(HELP);
-            return;
-        }
-
-        String dlUriStr = args[0];
-        final String streamName = args[1];
-
-        URI uri = URI.create(dlUriStr);
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.setImmediateFlushEnabled(true);
-        conf.setOutputBufferSize(0);
-        conf.setPeriodicFlushFrequencyMilliSeconds(0);
-        conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE);
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(uri)
-                .regionId(DistributedLogConstants.LOCAL_REGION_ID)
-                .clientId("console-writer")
-                .build();
-
-        // open the dlm
-        System.out.println("Opening log stream " + streamName);
-        DistributedLogManager dlm = namespace.openLog(streamName);
-
-        try {
-            AsyncLogWriter writer = null;
-            try {
-                writer = FutureUtils.result(dlm.openAsyncLogWriter());
-
-                ConsoleReader reader = new ConsoleReader();
-                String line;
-                while ((line = reader.readLine(PROMPT_MESSAGE)) != null) {
-                    writer.write(new LogRecord(System.currentTimeMillis(), line.getBytes(UTF_8)))
-                            .addEventListener(new FutureEventListener<DLSN>() {
-                                @Override
-                                public void onFailure(Throwable cause) {
-                                    System.out.println("Encountered error on writing data");
-                                    cause.printStackTrace(System.err);
-                                    Runtime.getRuntime().exit(0);
-                                }
-
-                                @Override
-                                public void onSuccess(DLSN value) {
-                                    // done
-                                }
-                            });
-                }
-            } finally {
-                if (null != writer) {
-                    FutureUtils.result(writer.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
-                }
-            }
-        } finally {
-            dlm.close();
-            namespace.close();
-        }
-    }
-
-}
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/MultiReader.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/MultiReader.java
deleted file mode 100644
index 33471bb..0000000
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/MultiReader.java
+++ /dev/null
@@ -1,140 +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 com.twitter.distributedlog.basic;
-
-import com.twitter.distributedlog.*;
-import com.twitter.distributedlog.exceptions.LogEmptyException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.util.FutureEventListener;
-import org.apache.commons.lang.StringUtils;
-
-import java.net.URI;
-import java.util.concurrent.CountDownLatch;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * A reader is tailing multiple streams
- */
-public class MultiReader {
-
-    private final static String HELP = "TailReader <uri> <stream-1>[,<stream-2>,...,<stream-n>]";
-
-    public static void main(String[] args) throws Exception {
-        if (2 != args.length) {
-            System.out.println(HELP);
-            return;
-        }
-
-        String dlUriStr = args[0];
-        final String streamList = args[1];
-
-        URI uri = URI.create(dlUriStr);
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(uri)
-                .build();
-
-        String[] streamNameList = StringUtils.split(streamList, ',');
-        DistributedLogManager[] managers = new DistributedLogManager[streamNameList.length];
-
-        for (int i = 0; i < managers.length; i++) {
-            String streamName = streamNameList[i];
-            // open the dlm
-            System.out.println("Opening log stream " + streamName);
-            managers[i] = namespace.openLog(streamName);
-        }
-
-        final CountDownLatch keepAliveLatch = new CountDownLatch(1);
-
-        for (DistributedLogManager dlm : managers) {
-            final DistributedLogManager manager = dlm;
-            dlm.getLastLogRecordAsync().addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
-                @Override
-                public void onFailure(Throwable cause) {
-                    if (cause instanceof LogNotFoundException) {
-                        System.err.println("Log stream " + manager.getStreamName() + " is not found. Please create it first.");
-                        keepAliveLatch.countDown();
-                    } else if (cause instanceof LogEmptyException) {
-                        System.err.println("Log stream " + manager.getStreamName() + " is empty.");
-                        readLoop(manager, DLSN.InitialDLSN, keepAliveLatch);
-                    } else {
-                        System.err.println("Encountered exception on process stream " + manager.getStreamName());
-                        keepAliveLatch.countDown();
-                    }
-                }
-
-                @Override
-                public void onSuccess(LogRecordWithDLSN record) {
-                    readLoop(manager, record.getDlsn(), keepAliveLatch);
-                }
-            });
-        }
-        keepAliveLatch.await();
-        for (DistributedLogManager dlm : managers) {
-            dlm.close();
-        }
-        namespace.close();
-    }
-
-    private static void readLoop(final DistributedLogManager dlm,
-                                 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>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                System.err.println("Encountered error on reading records from stream " + dlm.getStreamName());
-                cause.printStackTrace(System.err);
-                keepAliveLatch.countDown();
-            }
-
-            @Override
-            public void onSuccess(AsyncLogReader reader) {
-                System.out.println("Open reader to read records from stream " + reader.getStreamName());
-                readLoop(reader, keepAliveLatch);
-            }
-        });
-    }
-
-    private static void readLoop(final AsyncLogReader reader,
-                                 final CountDownLatch keepAliveLatch) {
-        final FutureEventListener<LogRecordWithDLSN> readListener = new FutureEventListener<LogRecordWithDLSN>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                System.err.println("Encountered error on reading records from stream " + reader.getStreamName());
-                cause.printStackTrace(System.err);
-                keepAliveLatch.countDown();
-            }
-
-            @Override
-            public void onSuccess(LogRecordWithDLSN record) {
-                System.out.println("Received record " + record.getDlsn() + " from stream " + reader.getStreamName());
-                System.out.println("\"\"\"");
-                System.out.println(new String(record.getPayload(), UTF_8));
-                System.out.println("\"\"\"");
-                reader.readNext().addEventListener(this);
-            }
-        };
-        reader.readNext().addEventListener(readListener);
-    }
-
-}
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/RecordGenerator.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/RecordGenerator.java
deleted file mode 100644
index 63b03f5..0000000
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/RecordGenerator.java
+++ /dev/null
@@ -1,90 +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 com.twitter.distributedlog.basic;
-
-import com.google.common.util.concurrent.RateLimiter;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.FutureEventListener;
-
-import java.nio.ByteBuffer;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Generate records in a given rate
- */
-public class RecordGenerator {
-
-    private final static String HELP = "RecordGenerator <finagle-name> <stream> <rate>";
-
-    public static void main(String[] args) throws Exception {
-        if (3 != args.length) {
-            System.out.println(HELP);
-            return;
-        }
-
-        String finagleNameStr = args[0];
-        final String streamName = args[1];
-        double rate = Double.parseDouble(args[2]);
-        RateLimiter limiter = RateLimiter.create(rate);
-
-        DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
-                .clientId(ClientId$.MODULE$.apply("record-generator"))
-                .name("record-generator")
-                .thriftmux(true)
-                .finagleNameStr(finagleNameStr)
-                .build();
-
-        final CountDownLatch keepAliveLatch = new CountDownLatch(1);
-        final AtomicLong numWrites = new AtomicLong(0);
-        final AtomicBoolean running = new AtomicBoolean(true);
-
-        while (running.get()) {
-            limiter.acquire();
-            String record = "record-" + System.currentTimeMillis();
-            client.write(streamName, ByteBuffer.wrap(record.getBytes(UTF_8)))
-                    .addEventListener(new FutureEventListener<DLSN>() {
-                        @Override
-                        public void onFailure(Throwable cause) {
-                            System.out.println("Encountered error on writing data");
-                            cause.printStackTrace(System.err);
-                            running.set(false);
-                            keepAliveLatch.countDown();
-                        }
-
-                        @Override
-                        public void onSuccess(DLSN value) {
-                            long numSuccesses = numWrites.incrementAndGet();
-                            if (numSuccesses % 100 == 0) {
-                                System.out.println("Write " + numSuccesses + " records.");
-                            }
-                        }
-                    });
-        }
-
-        keepAliveLatch.await();
-        client.close();
-    }
-
-}
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/StreamRewinder.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/StreamRewinder.java
deleted file mode 100644
index 2c1eaaf..0000000
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/StreamRewinder.java
+++ /dev/null
@@ -1,113 +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 com.twitter.distributedlog.basic;
-
-import com.twitter.distributedlog.*;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.CountDownLatch;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
-
-import java.net.URI;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.atomic.AtomicBoolean;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Rewind a stream to read data back in a while
- */
-public class StreamRewinder {
-
-    private final static String HELP = "StreamRewinder <uri> <string> <seconds>";
-
-    public static void main(String[] args) throws Exception {
-        if (3 != args.length) {
-            System.out.println(HELP);
-            return;
-        }
-
-        String dlUriStr = args[0];
-        final String streamName = args[1];
-        final int rewindSeconds = Integer.parseInt(args[2]);
-
-        URI uri = URI.create(dlUriStr);
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(uri)
-                .build();
-
-        // open the dlm
-        System.out.println("Opening log stream " + streamName);
-        DistributedLogManager dlm = namespace.openLog(streamName);
-
-        try {
-            readLoop(dlm, rewindSeconds);
-        } finally {
-            dlm.close();
-            namespace.close();
-        }
-    }
-
-    private static void readLoop(final DistributedLogManager dlm,
-                                 final int rewindSeconds) throws Exception {
-
-        final CountDownLatch keepAliveLatch = new CountDownLatch(1);
-
-        long rewindToTxId = System.currentTimeMillis() -
-                TimeUnit.MILLISECONDS.convert(rewindSeconds, TimeUnit.SECONDS);
-
-        System.out.println("Record records starting from " + rewindToTxId
-                + " which is " + rewindSeconds + " seconds ago");
-        final AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(rewindToTxId));
-        final AtomicBoolean caughtup = new AtomicBoolean(false);
-        final FutureEventListener<LogRecordWithDLSN> readListener = new FutureEventListener<LogRecordWithDLSN>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                System.err.println("Encountered error on reading records from stream " + dlm.getStreamName());
-                cause.printStackTrace(System.err);
-                keepAliveLatch.countDown();
-            }
-
-            @Override
-            public void onSuccess(LogRecordWithDLSN record) {
-                System.out.println("Received record " + record.getDlsn());
-                System.out.println("\"\"\"");
-                System.out.println(new String(record.getPayload(), UTF_8));
-                System.out.println("\"\"\"");
-
-                long diffInMilliseconds = System.currentTimeMillis() - record.getTransactionId();
-
-                if (!caughtup.get() && diffInMilliseconds < 2000) {
-                    System.out.println("Reader caught with latest data");
-                    caughtup.set(true);
-                }
-
-                reader.readNext().addEventListener(this);
-            }
-        };
-        reader.readNext().addEventListener(readListener);
-
-        keepAliveLatch.await();
-        FutureUtils.result(reader.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
-    }
-
-}
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/TailReader.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/TailReader.java
deleted file mode 100644
index e8a4a1e..0000000
--- a/distributedlog-tutorials/distributedlog-basic/src/main/java/com/twitter/distributedlog/basic/TailReader.java
+++ /dev/null
@@ -1,112 +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 com.twitter.distributedlog.basic;
-
-import com.twitter.distributedlog.*;
-import com.twitter.distributedlog.exceptions.LogEmptyException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
-
-import java.net.URI;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * A reader is tailing a log
- */
-public class TailReader {
-
-    private final static String HELP = "TailReader <uri> <string>";
-
-    public static void main(String[] args) throws Exception {
-        if (2 != args.length) {
-            System.out.println(HELP);
-            return;
-        }
-
-        String dlUriStr = args[0];
-        final String streamName = args[1];
-
-        URI uri = URI.create(dlUriStr);
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(uri)
-                .build();
-
-        // open the dlm
-        System.out.println("Opening log stream " + streamName);
-        DistributedLogManager dlm = namespace.openLog(streamName);
-
-        // get the last record
-        LogRecordWithDLSN lastRecord;
-        DLSN dlsn;
-        try {
-            lastRecord = dlm.getLastLogRecord();
-            dlsn = lastRecord.getDlsn();
-            readLoop(dlm, dlsn);
-        } catch (LogNotFoundException lnfe) {
-            System.err.println("Log stream " + streamName + " is not found. Please create it first.");
-            return;
-        } catch (LogEmptyException lee) {
-            System.err.println("Log stream " + streamName + " is empty.");
-            dlsn = DLSN.InitialDLSN;
-            readLoop(dlm, dlsn);
-        } finally {
-            dlm.close();
-            namespace.close();
-        }
-    }
-
-    private static void readLoop(final DistributedLogManager dlm,
-                                 final DLSN dlsn) throws Exception {
-
-        final CountDownLatch keepAliveLatch = new CountDownLatch(1);
-
-        System.out.println("Wait for records starting from " + dlsn);
-        final AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(dlsn));
-        final FutureEventListener<LogRecordWithDLSN> readListener = new FutureEventListener<LogRecordWithDLSN>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                System.err.println("Encountered error on reading records from stream " + dlm.getStreamName());
-                cause.printStackTrace(System.err);
-                keepAliveLatch.countDown();
-            }
-
-            @Override
-            public void onSuccess(LogRecordWithDLSN record) {
-                System.out.println("Received record " + record.getDlsn());
-                System.out.println("\"\"\"");
-                System.out.println(new String(record.getPayload(), UTF_8));
-                System.out.println("\"\"\"");
-                reader.readNext().addEventListener(this);
-            }
-        };
-        reader.readNext().addEventListener(readListener);
-
-        keepAliveLatch.await();
-        FutureUtils.result(reader.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
-    }
-
-}
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
new file mode 100644
index 0000000..8ef5c46
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/AtomicWriter.java
@@ -0,0 +1,107 @@
+/**
+ * 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.basic;
+
+import com.google.common.collect.Lists;
+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;
+
+/**
+ * Write multiple record atomically
+ */
+public class AtomicWriter {
+
+    private final static String HELP = "AtomicWriter <finagle-name> <stream> <message>[,<message>]";
+
+    public static void main(String[] args) throws Exception {
+        if (args.length < 3) {
+            System.out.println(HELP);
+            return;
+        }
+
+        String finagleNameStr = args[0];
+        String streamName = args[1];
+        String[] messages = new String[args.length - 2];
+        System.arraycopy(args, 2, messages, 0, messages.length);
+
+        DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
+                .clientId(ClientId$.MODULE$.apply("atomic-writer"))
+                .name("atomic-writer")
+                .thriftmux(true)
+                .finagleNameStr(finagleNameStr)
+                .build();
+
+        final LogRecordSet.Writer recordSetWriter = LogRecordSet.newWriter(16 * 1024, Type.NONE);
+        List<Future<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>() {
+                @Override
+                public void onFailure(Throwable cause) {
+                    System.out.println("Encountered error on writing data");
+                    cause.printStackTrace(System.err);
+                    Runtime.getRuntime().exit(0);
+                }
+
+                @Override
+                public void onSuccess(DLSN dlsn) {
+                    System.out.println("Write '" + message + "' as record " + dlsn);
+                }
+            });
+            recordSetWriter.writeRecord(msgBuf, writeFuture);
+            writeFutures.add(writeFuture);
+        }
+        FutureUtils.result(
+            client.writeRecordSet(streamName, recordSetWriter)
+                .addEventListener(new FutureEventListener<DLSN>() {
+                    @Override
+                    public void onFailure(Throwable cause) {
+                        recordSetWriter.abortTransmit(cause);
+                        System.out.println("Encountered error on writing data");
+                        cause.printStackTrace(System.err);
+                        Runtime.getRuntime().exit(0);
+                    }
+
+                    @Override
+                    public void onSuccess(DLSN dlsn) {
+                        recordSetWriter.completeTransmit(
+                                dlsn.getLogSegmentSequenceNo(),
+                                dlsn.getEntryId(),
+                                dlsn.getSlotId());
+                    }
+                })
+        );
+        FutureUtils.result(Future.collect(writeFutures));
+        client.close();
+    }
+}
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/ConsoleProxyMultiWriter.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/ConsoleProxyMultiWriter.java
new file mode 100644
index 0000000..655fa15
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/ConsoleProxyMultiWriter.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.basic;
+
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.client.DistributedLogMultiStreamWriter;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.FutureEventListener;
+import jline.ConsoleReader;
+import org.apache.commons.lang.StringUtils;
+
+import java.nio.ByteBuffer;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Writer write records from console
+ */
+public class ConsoleProxyMultiWriter {
+
+    private final static String HELP = "ConsoleProxyWriter <finagle-name> <stream-1>[,<stream-2>,...,<stream-n>]";
+    private final static String PROMPT_MESSAGE = "[dlog] > ";
+
+    public static void main(String[] args) throws Exception {
+        if (2 != args.length) {
+            System.out.println(HELP);
+            return;
+        }
+
+        String finagleNameStr = args[0];
+        final String streamList = args[1];
+
+        DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
+                .clientId(ClientId$.MODULE$.apply("console-proxy-writer"))
+                .name("console-proxy-writer")
+                .thriftmux(true)
+                .finagleNameStr(finagleNameStr)
+                .build();
+        String[] streamNameList = StringUtils.split(streamList, ',');
+        DistributedLogMultiStreamWriter multiStreamWriter = DistributedLogMultiStreamWriter.newBuilder()
+                .streams(Lists.newArrayList(streamNameList))
+                .bufferSize(0)
+                .client(client)
+                .flushIntervalMs(0)
+                .firstSpeculativeTimeoutMs(10000)
+                .maxSpeculativeTimeoutMs(20000)
+                .requestTimeoutMs(50000)
+                .build();
+
+        ConsoleReader reader = new ConsoleReader();
+        String line;
+        while ((line = reader.readLine(PROMPT_MESSAGE)) != null) {
+            multiStreamWriter.write(ByteBuffer.wrap(line.getBytes(UTF_8)))
+                    .addEventListener(new FutureEventListener<DLSN>() {
+                        @Override
+                        public void onFailure(Throwable cause) {
+                            System.out.println("Encountered error on writing data");
+                            cause.printStackTrace(System.err);
+                            Runtime.getRuntime().exit(0);
+                        }
+
+                        @Override
+                        public void onSuccess(DLSN value) {
+                            // done
+                        }
+                    });
+        }
+
+        multiStreamWriter.close();
+        client.close();
+    }
+
+}
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/ConsoleProxyWriter.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/ConsoleProxyWriter.java
new file mode 100644
index 0000000..027b0cd
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/ConsoleProxyWriter.java
@@ -0,0 +1,77 @@
+/**
+ * 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.basic;
+
+import org.apache.distributedlog.*;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.FutureEventListener;
+import jline.ConsoleReader;
+
+import java.nio.ByteBuffer;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Writer write records from console
+ */
+public class ConsoleProxyWriter {
+
+    private final static String HELP = "ConsoleProxyWriter <finagle-name> <string>";
+    private final static String PROMPT_MESSAGE = "[dlog] > ";
+
+    public static void main(String[] args) throws Exception {
+        if (2 != args.length) {
+            System.out.println(HELP);
+            return;
+        }
+
+        String finagleNameStr = args[0];
+        final String streamName = args[1];
+
+        DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
+                .clientId(ClientId$.MODULE$.apply("console-proxy-writer"))
+                .name("console-proxy-writer")
+                .thriftmux(true)
+                .finagleNameStr(finagleNameStr)
+                .build();
+
+        ConsoleReader reader = new ConsoleReader();
+        String line;
+        while ((line = reader.readLine(PROMPT_MESSAGE)) != null) {
+            client.write(streamName, ByteBuffer.wrap(line.getBytes(UTF_8)))
+                    .addEventListener(new FutureEventListener<DLSN>() {
+                        @Override
+                        public void onFailure(Throwable cause) {
+                            System.out.println("Encountered error on writing data");
+                            cause.printStackTrace(System.err);
+                            Runtime.getRuntime().exit(0);
+                        }
+
+                        @Override
+                        public void onSuccess(DLSN value) {
+                            // done
+                        }
+                    });
+        }
+
+        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
new file mode 100644
index 0000000..4322224
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/ConsoleWriter.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.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 java.net.URI;
+import java.util.concurrent.TimeUnit;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Writer write records from console
+ */
+public class ConsoleWriter {
+
+    private final static String HELP = "ConsoleWriter <uri> <string>";
+    private final static String PROMPT_MESSAGE = "[dlog] > ";
+
+    public static void main(String[] args) throws Exception {
+        if (2 != args.length) {
+            System.out.println(HELP);
+            return;
+        }
+
+        String dlUriStr = args[0];
+        final String streamName = args[1];
+
+        URI uri = URI.create(dlUriStr);
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.setImmediateFlushEnabled(true);
+        conf.setOutputBufferSize(0);
+        conf.setPeriodicFlushFrequencyMilliSeconds(0);
+        conf.setLockTimeout(DistributedLogConstants.LOCK_IMMEDIATE);
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .regionId(DistributedLogConstants.LOCAL_REGION_ID)
+                .clientId("console-writer")
+                .build();
+
+        // open the dlm
+        System.out.println("Opening log stream " + streamName);
+        DistributedLogManager dlm = namespace.openLog(streamName);
+
+        try {
+            AsyncLogWriter writer = null;
+            try {
+                writer = FutureUtils.result(dlm.openAsyncLogWriter());
+
+                ConsoleReader reader = new ConsoleReader();
+                String line;
+                while ((line = reader.readLine(PROMPT_MESSAGE)) != null) {
+                    writer.write(new LogRecord(System.currentTimeMillis(), line.getBytes(UTF_8)))
+                            .addEventListener(new FutureEventListener<DLSN>() {
+                                @Override
+                                public void onFailure(Throwable cause) {
+                                    System.out.println("Encountered error on writing data");
+                                    cause.printStackTrace(System.err);
+                                    Runtime.getRuntime().exit(0);
+                                }
+
+                                @Override
+                                public void onSuccess(DLSN value) {
+                                    // done
+                                }
+                            });
+                }
+            } finally {
+                if (null != writer) {
+                    FutureUtils.result(writer.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
+                }
+            }
+        } finally {
+            dlm.close();
+            namespace.close();
+        }
+    }
+
+}
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
new file mode 100644
index 0000000..9fe2013
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/MultiReader.java
@@ -0,0 +1,140 @@
+/**
+ * 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.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 com.twitter.util.FutureEventListener;
+import org.apache.commons.lang.StringUtils;
+
+import java.net.URI;
+import java.util.concurrent.CountDownLatch;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * A reader is tailing multiple streams
+ */
+public class MultiReader {
+
+    private final static String HELP = "TailReader <uri> <stream-1>[,<stream-2>,...,<stream-n>]";
+
+    public static void main(String[] args) throws Exception {
+        if (2 != args.length) {
+            System.out.println(HELP);
+            return;
+        }
+
+        String dlUriStr = args[0];
+        final String streamList = args[1];
+
+        URI uri = URI.create(dlUriStr);
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build();
+
+        String[] streamNameList = StringUtils.split(streamList, ',');
+        DistributedLogManager[] managers = new DistributedLogManager[streamNameList.length];
+
+        for (int i = 0; i < managers.length; i++) {
+            String streamName = streamNameList[i];
+            // open the dlm
+            System.out.println("Opening log stream " + streamName);
+            managers[i] = namespace.openLog(streamName);
+        }
+
+        final CountDownLatch keepAliveLatch = new CountDownLatch(1);
+
+        for (DistributedLogManager dlm : managers) {
+            final DistributedLogManager manager = dlm;
+            dlm.getLastLogRecordAsync().addEventListener(new FutureEventListener<LogRecordWithDLSN>() {
+                @Override
+                public void onFailure(Throwable cause) {
+                    if (cause instanceof LogNotFoundException) {
+                        System.err.println("Log stream " + manager.getStreamName() + " is not found. Please create it first.");
+                        keepAliveLatch.countDown();
+                    } else if (cause instanceof LogEmptyException) {
+                        System.err.println("Log stream " + manager.getStreamName() + " is empty.");
+                        readLoop(manager, DLSN.InitialDLSN, keepAliveLatch);
+                    } else {
+                        System.err.println("Encountered exception on process stream " + manager.getStreamName());
+                        keepAliveLatch.countDown();
+                    }
+                }
+
+                @Override
+                public void onSuccess(LogRecordWithDLSN record) {
+                    readLoop(manager, record.getDlsn(), keepAliveLatch);
+                }
+            });
+        }
+        keepAliveLatch.await();
+        for (DistributedLogManager dlm : managers) {
+            dlm.close();
+        }
+        namespace.close();
+    }
+
+    private static void readLoop(final DistributedLogManager dlm,
+                                 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>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                System.err.println("Encountered error on reading records from stream " + dlm.getStreamName());
+                cause.printStackTrace(System.err);
+                keepAliveLatch.countDown();
+            }
+
+            @Override
+            public void onSuccess(AsyncLogReader reader) {
+                System.out.println("Open reader to read records from stream " + reader.getStreamName());
+                readLoop(reader, keepAliveLatch);
+            }
+        });
+    }
+
+    private static void readLoop(final AsyncLogReader reader,
+                                 final CountDownLatch keepAliveLatch) {
+        final FutureEventListener<LogRecordWithDLSN> readListener = new FutureEventListener<LogRecordWithDLSN>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                System.err.println("Encountered error on reading records from stream " + reader.getStreamName());
+                cause.printStackTrace(System.err);
+                keepAliveLatch.countDown();
+            }
+
+            @Override
+            public void onSuccess(LogRecordWithDLSN record) {
+                System.out.println("Received record " + record.getDlsn() + " from stream " + reader.getStreamName());
+                System.out.println("\"\"\"");
+                System.out.println(new String(record.getPayload(), UTF_8));
+                System.out.println("\"\"\"");
+                reader.readNext().addEventListener(this);
+            }
+        };
+        reader.readNext().addEventListener(readListener);
+    }
+
+}
diff --git a/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/RecordGenerator.java b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/RecordGenerator.java
new file mode 100644
index 0000000..3c730ef
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/RecordGenerator.java
@@ -0,0 +1,90 @@
+/**
+ * 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.basic;
+
+import com.google.common.util.concurrent.RateLimiter;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.FutureEventListener;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+import java.util.concurrent.atomic.AtomicLong;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Generate records in a given rate
+ */
+public class RecordGenerator {
+
+    private final static String HELP = "RecordGenerator <finagle-name> <stream> <rate>";
+
+    public static void main(String[] args) throws Exception {
+        if (3 != args.length) {
+            System.out.println(HELP);
+            return;
+        }
+
+        String finagleNameStr = args[0];
+        final String streamName = args[1];
+        double rate = Double.parseDouble(args[2]);
+        RateLimiter limiter = RateLimiter.create(rate);
+
+        DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
+                .clientId(ClientId$.MODULE$.apply("record-generator"))
+                .name("record-generator")
+                .thriftmux(true)
+                .finagleNameStr(finagleNameStr)
+                .build();
+
+        final CountDownLatch keepAliveLatch = new CountDownLatch(1);
+        final AtomicLong numWrites = new AtomicLong(0);
+        final AtomicBoolean running = new AtomicBoolean(true);
+
+        while (running.get()) {
+            limiter.acquire();
+            String record = "record-" + System.currentTimeMillis();
+            client.write(streamName, ByteBuffer.wrap(record.getBytes(UTF_8)))
+                    .addEventListener(new FutureEventListener<DLSN>() {
+                        @Override
+                        public void onFailure(Throwable cause) {
+                            System.out.println("Encountered error on writing data");
+                            cause.printStackTrace(System.err);
+                            running.set(false);
+                            keepAliveLatch.countDown();
+                        }
+
+                        @Override
+                        public void onSuccess(DLSN value) {
+                            long numSuccesses = numWrites.incrementAndGet();
+                            if (numSuccesses % 100 == 0) {
+                                System.out.println("Write " + numSuccesses + " records.");
+                            }
+                        }
+                    });
+        }
+
+        keepAliveLatch.await();
+        client.close();
+    }
+
+}
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
new file mode 100644
index 0000000..50a456d
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/StreamRewinder.java
@@ -0,0 +1,113 @@
+/**
+ * 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.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 java.net.URI;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Rewind a stream to read data back in a while
+ */
+public class StreamRewinder {
+
+    private final static String HELP = "StreamRewinder <uri> <string> <seconds>";
+
+    public static void main(String[] args) throws Exception {
+        if (3 != args.length) {
+            System.out.println(HELP);
+            return;
+        }
+
+        String dlUriStr = args[0];
+        final String streamName = args[1];
+        final int rewindSeconds = Integer.parseInt(args[2]);
+
+        URI uri = URI.create(dlUriStr);
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build();
+
+        // open the dlm
+        System.out.println("Opening log stream " + streamName);
+        DistributedLogManager dlm = namespace.openLog(streamName);
+
+        try {
+            readLoop(dlm, rewindSeconds);
+        } finally {
+            dlm.close();
+            namespace.close();
+        }
+    }
+
+    private static void readLoop(final DistributedLogManager dlm,
+                                 final int rewindSeconds) throws Exception {
+
+        final CountDownLatch keepAliveLatch = new CountDownLatch(1);
+
+        long rewindToTxId = System.currentTimeMillis() -
+                TimeUnit.MILLISECONDS.convert(rewindSeconds, TimeUnit.SECONDS);
+
+        System.out.println("Record records starting from " + rewindToTxId
+                + " which is " + rewindSeconds + " seconds ago");
+        final AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(rewindToTxId));
+        final AtomicBoolean caughtup = new AtomicBoolean(false);
+        final FutureEventListener<LogRecordWithDLSN> readListener = new FutureEventListener<LogRecordWithDLSN>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                System.err.println("Encountered error on reading records from stream " + dlm.getStreamName());
+                cause.printStackTrace(System.err);
+                keepAliveLatch.countDown();
+            }
+
+            @Override
+            public void onSuccess(LogRecordWithDLSN record) {
+                System.out.println("Received record " + record.getDlsn());
+                System.out.println("\"\"\"");
+                System.out.println(new String(record.getPayload(), UTF_8));
+                System.out.println("\"\"\"");
+
+                long diffInMilliseconds = System.currentTimeMillis() - record.getTransactionId();
+
+                if (!caughtup.get() && diffInMilliseconds < 2000) {
+                    System.out.println("Reader caught with latest data");
+                    caughtup.set(true);
+                }
+
+                reader.readNext().addEventListener(this);
+            }
+        };
+        reader.readNext().addEventListener(readListener);
+
+        keepAliveLatch.await();
+        FutureUtils.result(reader.asyncClose(), Duration.apply(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
new file mode 100644
index 0000000..8b43b45
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-basic/src/main/java/org/apache/distributedlog/basic/TailReader.java
@@ -0,0 +1,112 @@
+/**
+ * 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.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 java.net.URI;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * A reader is tailing a log
+ */
+public class TailReader {
+
+    private final static String HELP = "TailReader <uri> <string>";
+
+    public static void main(String[] args) throws Exception {
+        if (2 != args.length) {
+            System.out.println(HELP);
+            return;
+        }
+
+        String dlUriStr = args[0];
+        final String streamName = args[1];
+
+        URI uri = URI.create(dlUriStr);
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build();
+
+        // open the dlm
+        System.out.println("Opening log stream " + streamName);
+        DistributedLogManager dlm = namespace.openLog(streamName);
+
+        // get the last record
+        LogRecordWithDLSN lastRecord;
+        DLSN dlsn;
+        try {
+            lastRecord = dlm.getLastLogRecord();
+            dlsn = lastRecord.getDlsn();
+            readLoop(dlm, dlsn);
+        } catch (LogNotFoundException lnfe) {
+            System.err.println("Log stream " + streamName + " is not found. Please create it first.");
+            return;
+        } catch (LogEmptyException lee) {
+            System.err.println("Log stream " + streamName + " is empty.");
+            dlsn = DLSN.InitialDLSN;
+            readLoop(dlm, dlsn);
+        } finally {
+            dlm.close();
+            namespace.close();
+        }
+    }
+
+    private static void readLoop(final DistributedLogManager dlm,
+                                 final DLSN dlsn) throws Exception {
+
+        final CountDownLatch keepAliveLatch = new CountDownLatch(1);
+
+        System.out.println("Wait for records starting from " + dlsn);
+        final AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(dlsn));
+        final FutureEventListener<LogRecordWithDLSN> readListener = new FutureEventListener<LogRecordWithDLSN>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                System.err.println("Encountered error on reading records from stream " + dlm.getStreamName());
+                cause.printStackTrace(System.err);
+                keepAliveLatch.countDown();
+            }
+
+            @Override
+            public void onSuccess(LogRecordWithDLSN record) {
+                System.out.println("Received record " + record.getDlsn());
+                System.out.println("\"\"\"");
+                System.out.println(new String(record.getPayload(), UTF_8));
+                System.out.println("\"\"\"");
+                reader.readNext().addEventListener(this);
+            }
+        };
+        reader.readNext().addEventListener(readListener);
+
+        keepAliveLatch.await();
+        FutureUtils.result(reader.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
+    }
+
+}
diff --git a/distributedlog-tutorials/distributedlog-kafka/conf/log4j.properties b/distributedlog-tutorials/distributedlog-kafka/conf/log4j.properties
index cc6486b..f3d3ad1 100644
--- a/distributedlog-tutorials/distributedlog-kafka/conf/log4j.properties
+++ b/distributedlog-tutorials/distributedlog-kafka/conf/log4j.properties
@@ -32,11 +32,11 @@
 log4j.logger.org.apache.bookkeeper=INFO
 
 # redirect executor output to executors.log since slow op warnings can be quite verbose
-log4j.logger.com.twitter.distributedlog.util.MonitoredFuturePool=INFO, Executors
-log4j.logger.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=INFO, Executors
+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.com.twitter.distributedlog.util.MonitoredFuturePool=false
-log4j.additivity.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=false
+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/com/twitter/distributedlog/kafka/DLFutureRecordMetadata.java b/distributedlog-tutorials/distributedlog-kafka/src/main/java/com/twitter/distributedlog/kafka/DLFutureRecordMetadata.java
deleted file mode 100644
index 1c6e432..0000000
--- a/distributedlog-tutorials/distributedlog-kafka/src/main/java/com/twitter/distributedlog/kafka/DLFutureRecordMetadata.java
+++ /dev/null
@@ -1,104 +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 com.twitter.distributedlog.kafka;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.exceptions.DLInterruptedException;
-import com.twitter.distributedlog.util.FutureUtils;
-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;
-
-class DLFutureRecordMetadata implements Future<RecordMetadata> {
-
-    private final String topic;
-    private final com.twitter.util.Future<DLSN> dlsnFuture;
-    private final Callback callback;
-
-    DLFutureRecordMetadata(final String topic,
-                           com.twitter.util.Future<DLSN> dlsnFuture,
-                           final Callback callback) {
-        this.topic = topic;
-        this.dlsnFuture = dlsnFuture;
-        this.callback = callback;
-
-        this.dlsnFuture.addEventListener(new FutureEventListener<DLSN>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                callback.onCompletion(null, new IOException(cause));
-            }
-
-            @Override
-            public void onSuccess(DLSN value) {
-                callback.onCompletion(new RecordMetadata(new TopicPartition(topic, 0), -1L, -1L), null);
-            }
-        });
-    }
-
-    @Override
-    public boolean cancel(boolean mayInterruptIfRunning) {
-        dlsnFuture.cancel();
-        // it is hard to figure out if it is successful or not.
-        // so return false here.
-        return false;
-    }
-
-    @Override
-    public boolean isCancelled() {
-        return false;
-    }
-
-    @Override
-    public boolean isDone() {
-        return dlsnFuture.isDefined();
-    }
-
-    @Override
-    public RecordMetadata get() throws InterruptedException, ExecutionException {
-        try {
-            FutureUtils.result(dlsnFuture);
-            // TODO: align the DLSN concepts with kafka concepts
-            return new RecordMetadata(new TopicPartition(topic, 0), -1L, -1L);
-        } catch (DLInterruptedException e) {
-            throw new InterruptedException("Interrupted on waiting for response");
-        } catch (IOException e) {
-            throw new ExecutionException("Error on waiting for response", e);
-        }
-    }
-
-    @Override
-    public RecordMetadata get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
-        try {
-            FutureUtils.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) {
-            throw new InterruptedException("Interrupted on waiting for response");
-        } catch (IOException e) {
-            throw new ExecutionException("Error on waiting for response", e);
-        }
-    }
-}
diff --git a/distributedlog-tutorials/distributedlog-kafka/src/main/java/com/twitter/distributedlog/kafka/KafkaDistributedLogProducer.java b/distributedlog-tutorials/distributedlog-kafka/src/main/java/com/twitter/distributedlog/kafka/KafkaDistributedLogProducer.java
deleted file mode 100644
index 11423f9..0000000
--- a/distributedlog-tutorials/distributedlog-kafka/src/main/java/com/twitter/distributedlog/kafka/KafkaDistributedLogProducer.java
+++ /dev/null
@@ -1,158 +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 com.twitter.distributedlog.kafka;
-
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.messaging.PartitionedMultiWriter;
-import com.twitter.distributedlog.messaging.Partitioner;
-import com.twitter.distributedlog.messaging.RRMultiWriter;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import org.apache.kafka.clients.producer.Callback;
-import org.apache.kafka.clients.producer.Producer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.apache.kafka.common.Metric;
-import org.apache.kafka.common.MetricName;
-import org.apache.kafka.common.PartitionInfo;
-
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.ConcurrentHashMap;
-import java.util.concurrent.ConcurrentMap;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-/**
- * It is a kafka producer that uses dl streams
- */
-public class KafkaDistributedLogProducer<K, V> implements Producer<K, V>, Partitioner<K> {
-
-    private final DistributedLogClient client;
-    private final int numPartitions;
-    private final ConcurrentMap<String, PartitionedMultiWriter<K, V>> partitionedWriters;
-    private final ConcurrentMap<String, RRMultiWriter<K, V>> unpartitionedWriters;
-
-    // Assume all streams have same partitions
-    public KafkaDistributedLogProducer(DistributedLogClient client,
-                                       int numPartitions) {
-        this.client = client;
-        this.numPartitions = numPartitions;
-        this.partitionedWriters = new ConcurrentHashMap<String, PartitionedMultiWriter<K, V>>();
-        this.unpartitionedWriters = new ConcurrentHashMap<String, RRMultiWriter<K, V>>();
-    }
-
-    @Override
-    public int partition(K k, int totalPartitions) {
-        if (null != k) {
-            return k.hashCode() % totalPartitions;
-        }
-        return -1;
-    }
-
-    private String[] getStreamsForTopic(String topic) {
-        String[] streams = new String[numPartitions];
-        for (int i = 0; i < numPartitions; i++) {
-            streams[i] = String.format("%s-%d", topic, i);
-        }
-        return streams;
-    }
-
-    private PartitionedMultiWriter<K, V> getPartitionedMultiWriter(String topic) {
-        PartitionedMultiWriter<K, V> writer = partitionedWriters.get(topic);
-        if (null == writer) {
-            PartitionedMultiWriter<K, V> newWriter = new PartitionedMultiWriter<K, V>(
-                    getStreamsForTopic(topic), this, client);
-            PartitionedMultiWriter<K, V> oldWriter = partitionedWriters.putIfAbsent(topic, newWriter);
-            if (null != oldWriter) {
-                writer = oldWriter;
-            } else {
-                writer = newWriter;
-            }
-        }
-        return writer;
-    }
-
-    private RRMultiWriter<K, V> getUnpartitionedMultiWriter(String topic) {
-        RRMultiWriter<K, V> writer = unpartitionedWriters.get(topic);
-        if (null == writer) {
-            RRMultiWriter<K, V> newWriter = new RRMultiWriter<K, V>(
-                    getStreamsForTopic(topic), client);
-            RRMultiWriter<K, V> oldWriter = unpartitionedWriters.putIfAbsent(topic, newWriter);
-            if (null != oldWriter) {
-                writer = oldWriter;
-            } else {
-                writer = newWriter;
-            }
-        }
-        return writer;
-    }
-
-    @Override
-    public Future<RecordMetadata> send(ProducerRecord<K, V> producerRecord) {
-        return send(producerRecord, null);
-    }
-
-    @Override
-    public Future<RecordMetadata> send(ProducerRecord<K, V> producerRecord, Callback callback) {
-        com.twitter.util.Future<DLSN> dlsnFuture;
-        if (null == producerRecord.key()) {
-            dlsnFuture = getUnpartitionedMultiWriter(producerRecord.topic()).write(producerRecord.value());
-        } else {
-            // TODO: be able to publish to a specific partition
-            dlsnFuture = getPartitionedMultiWriter(producerRecord.topic()).write(producerRecord.key(),
-                    producerRecord.value());
-        }
-        return new DLFutureRecordMetadata(producerRecord.topic(), dlsnFuture, callback);
-    }
-
-    @Override
-    public void flush() {
-        // no-op
-    }
-
-    @Override
-    public List<PartitionInfo> partitionsFor(String s) {
-        String[] streams = getStreamsForTopic(s);
-        List<PartitionInfo> partitions = Lists.newArrayListWithExpectedSize(streams.length);
-        for (int i = 0; i < streams.length; i++) {
-            // TODO: maybe add getOwner from dl write proxy to return the owner of the partition
-            partitions.add(new PartitionInfo(s, i, null, null, null));
-        }
-        return partitions;
-    }
-
-    @Override
-    public Map<MetricName, ? extends Metric> metrics() {
-        // no-op
-        return Maps.newHashMap();
-    }
-
-    @Override
-    public void close() {
-        partitionedWriters.clear();
-        unpartitionedWriters.clear();
-    }
-
-    @Override
-    public void close(long l, TimeUnit timeUnit) {
-        partitionedWriters.clear();
-        unpartitionedWriters.clear();
-    }
-}
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
new file mode 100644
index 0000000..f1490d4
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-kafka/src/main/java/org/apache/distributedlog/kafka/DLFutureRecordMetadata.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.kafka;
+
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.exceptions.DLInterruptedException;
+import org.apache.distributedlog.util.FutureUtils;
+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;
+
+class DLFutureRecordMetadata implements Future<RecordMetadata> {
+
+    private final String topic;
+    private final com.twitter.util.Future<DLSN> dlsnFuture;
+    private final Callback callback;
+
+    DLFutureRecordMetadata(final String topic,
+                           com.twitter.util.Future<DLSN> dlsnFuture,
+                           final Callback callback) {
+        this.topic = topic;
+        this.dlsnFuture = dlsnFuture;
+        this.callback = callback;
+
+        this.dlsnFuture.addEventListener(new FutureEventListener<DLSN>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                callback.onCompletion(null, new IOException(cause));
+            }
+
+            @Override
+            public void onSuccess(DLSN value) {
+                callback.onCompletion(new RecordMetadata(new TopicPartition(topic, 0), -1L, -1L), null);
+            }
+        });
+    }
+
+    @Override
+    public boolean cancel(boolean mayInterruptIfRunning) {
+        dlsnFuture.cancel();
+        // it is hard to figure out if it is successful or not.
+        // so return false here.
+        return false;
+    }
+
+    @Override
+    public boolean isCancelled() {
+        return false;
+    }
+
+    @Override
+    public boolean isDone() {
+        return dlsnFuture.isDefined();
+    }
+
+    @Override
+    public RecordMetadata get() throws InterruptedException, ExecutionException {
+        try {
+            FutureUtils.result(dlsnFuture);
+            // TODO: align the DLSN concepts with kafka concepts
+            return new RecordMetadata(new TopicPartition(topic, 0), -1L, -1L);
+        } catch (DLInterruptedException e) {
+            throw new InterruptedException("Interrupted on waiting for response");
+        } catch (IOException e) {
+            throw new ExecutionException("Error on waiting for response", e);
+        }
+    }
+
+    @Override
+    public RecordMetadata get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
+        try {
+            FutureUtils.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) {
+            throw new InterruptedException("Interrupted on waiting for response");
+        } catch (IOException e) {
+            throw new ExecutionException("Error on waiting for response", e);
+        }
+    }
+}
diff --git a/distributedlog-tutorials/distributedlog-kafka/src/main/java/org/apache/distributedlog/kafka/KafkaDistributedLogProducer.java b/distributedlog-tutorials/distributedlog-kafka/src/main/java/org/apache/distributedlog/kafka/KafkaDistributedLogProducer.java
new file mode 100644
index 0000000..c9eac81
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-kafka/src/main/java/org/apache/distributedlog/kafka/KafkaDistributedLogProducer.java
@@ -0,0 +1,158 @@
+/**
+ * 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.kafka;
+
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.messaging.PartitionedMultiWriter;
+import org.apache.distributedlog.messaging.Partitioner;
+import org.apache.distributedlog.messaging.RRMultiWriter;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.MetricName;
+import org.apache.kafka.common.PartitionInfo;
+
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+import java.util.concurrent.Future;
+import java.util.concurrent.TimeUnit;
+
+/**
+ * It is a kafka producer that uses dl streams
+ */
+public class KafkaDistributedLogProducer<K, V> implements Producer<K, V>, Partitioner<K> {
+
+    private final DistributedLogClient client;
+    private final int numPartitions;
+    private final ConcurrentMap<String, PartitionedMultiWriter<K, V>> partitionedWriters;
+    private final ConcurrentMap<String, RRMultiWriter<K, V>> unpartitionedWriters;
+
+    // Assume all streams have same partitions
+    public KafkaDistributedLogProducer(DistributedLogClient client,
+                                       int numPartitions) {
+        this.client = client;
+        this.numPartitions = numPartitions;
+        this.partitionedWriters = new ConcurrentHashMap<String, PartitionedMultiWriter<K, V>>();
+        this.unpartitionedWriters = new ConcurrentHashMap<String, RRMultiWriter<K, V>>();
+    }
+
+    @Override
+    public int partition(K k, int totalPartitions) {
+        if (null != k) {
+            return k.hashCode() % totalPartitions;
+        }
+        return -1;
+    }
+
+    private String[] getStreamsForTopic(String topic) {
+        String[] streams = new String[numPartitions];
+        for (int i = 0; i < numPartitions; i++) {
+            streams[i] = String.format("%s-%d", topic, i);
+        }
+        return streams;
+    }
+
+    private PartitionedMultiWriter<K, V> getPartitionedMultiWriter(String topic) {
+        PartitionedMultiWriter<K, V> writer = partitionedWriters.get(topic);
+        if (null == writer) {
+            PartitionedMultiWriter<K, V> newWriter = new PartitionedMultiWriter<K, V>(
+                    getStreamsForTopic(topic), this, client);
+            PartitionedMultiWriter<K, V> oldWriter = partitionedWriters.putIfAbsent(topic, newWriter);
+            if (null != oldWriter) {
+                writer = oldWriter;
+            } else {
+                writer = newWriter;
+            }
+        }
+        return writer;
+    }
+
+    private RRMultiWriter<K, V> getUnpartitionedMultiWriter(String topic) {
+        RRMultiWriter<K, V> writer = unpartitionedWriters.get(topic);
+        if (null == writer) {
+            RRMultiWriter<K, V> newWriter = new RRMultiWriter<K, V>(
+                    getStreamsForTopic(topic), client);
+            RRMultiWriter<K, V> oldWriter = unpartitionedWriters.putIfAbsent(topic, newWriter);
+            if (null != oldWriter) {
+                writer = oldWriter;
+            } else {
+                writer = newWriter;
+            }
+        }
+        return writer;
+    }
+
+    @Override
+    public Future<RecordMetadata> send(ProducerRecord<K, V> producerRecord) {
+        return send(producerRecord, null);
+    }
+
+    @Override
+    public Future<RecordMetadata> send(ProducerRecord<K, V> producerRecord, Callback callback) {
+        com.twitter.util.Future<DLSN> dlsnFuture;
+        if (null == producerRecord.key()) {
+            dlsnFuture = getUnpartitionedMultiWriter(producerRecord.topic()).write(producerRecord.value());
+        } else {
+            // TODO: be able to publish to a specific partition
+            dlsnFuture = getPartitionedMultiWriter(producerRecord.topic()).write(producerRecord.key(),
+                    producerRecord.value());
+        }
+        return new DLFutureRecordMetadata(producerRecord.topic(), dlsnFuture, callback);
+    }
+
+    @Override
+    public void flush() {
+        // no-op
+    }
+
+    @Override
+    public List<PartitionInfo> partitionsFor(String s) {
+        String[] streams = getStreamsForTopic(s);
+        List<PartitionInfo> partitions = Lists.newArrayListWithExpectedSize(streams.length);
+        for (int i = 0; i < streams.length; i++) {
+            // TODO: maybe add getOwner from dl write proxy to return the owner of the partition
+            partitions.add(new PartitionInfo(s, i, null, null, null));
+        }
+        return partitions;
+    }
+
+    @Override
+    public Map<MetricName, ? extends Metric> metrics() {
+        // no-op
+        return Maps.newHashMap();
+    }
+
+    @Override
+    public void close() {
+        partitionedWriters.clear();
+        unpartitionedWriters.clear();
+    }
+
+    @Override
+    public void close(long l, TimeUnit timeUnit) {
+        partitionedWriters.clear();
+        unpartitionedWriters.clear();
+    }
+}
diff --git a/distributedlog-tutorials/distributedlog-mapreduce/README.md b/distributedlog-tutorials/distributedlog-mapreduce/README.md
index fa63c34..ab50d28 100644
--- a/distributedlog-tutorials/distributedlog-mapreduce/README.md
+++ b/distributedlog-tutorials/distributedlog-mapreduce/README.md
Binary files differ
diff --git a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/DistributedLogInputFormat.java b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/DistributedLogInputFormat.java
deleted file mode 100644
index 87ddec0..0000000
--- a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/DistributedLogInputFormat.java
+++ /dev/null
@@ -1,129 +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 com.twitter.distributedlog.mapreduce;
-
-import com.google.common.collect.Lists;
-import com.twitter.distributedlog.DistributedLogConfiguration;
-import com.twitter.distributedlog.DistributedLogManager;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.LogRecordWithDLSN;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import com.twitter.distributedlog.impl.BKNamespaceDriver;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.BookKeeperAccessor;
-import org.apache.bookkeeper.client.LedgerMetadata;
-import org.apache.bookkeeper.meta.LedgerManager;
-import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
-import org.apache.hadoop.conf.Configurable;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.mapreduce.InputFormat;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.JobContext;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.List;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.atomic.AtomicInteger;
-import java.util.concurrent.atomic.AtomicReference;
-
-/**
- * InputFormat to read data from a distributedlog stream.
- */
-public class DistributedLogInputFormat
-        extends InputFormat<DLSN, LogRecordWithDLSN> implements Configurable {
-
-    private static final String DL_URI = "distributedlog.uri";
-    private static final String DL_STREAM = "distributedlog.stream";
-
-    protected Configuration conf;
-    protected DistributedLogConfiguration dlConf;
-    protected URI dlUri;
-    protected DistributedLogNamespace namespace;
-    protected String streamName;
-    protected DistributedLogManager dlm;
-
-    /** {@inheritDoc} */
-    @Override
-    public void setConf(Configuration configuration) {
-        this.conf = configuration;
-        dlConf = new DistributedLogConfiguration();
-        dlUri = URI.create(configuration.get(DL_URI, ""));
-        streamName = configuration.get(DL_STREAM, "");
-        try {
-            namespace = DistributedLogNamespaceBuilder.newBuilder()
-                    .conf(dlConf)
-                    .uri(dlUri)
-                    .build();
-            dlm = namespace.openLog(streamName);
-        } catch (IOException e) {
-            throw new RuntimeException(e);
-        }
-    }
-
-    @Override
-    public Configuration getConf() {
-        return conf;
-    }
-
-    @Override
-    public List<InputSplit> getSplits(JobContext jobContext)
-            throws IOException, InterruptedException {
-        List<LogSegmentMetadata> segments = dlm.getLogSegments();
-        List<InputSplit> inputSplits = Lists.newArrayListWithCapacity(segments.size());
-        BookKeeper bk = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get();
-        LedgerManager lm = BookKeeperAccessor.getLedgerManager(bk);
-        final AtomicInteger rcHolder = new AtomicInteger(0);
-        final AtomicReference<LedgerMetadata> metadataHolder = new AtomicReference<LedgerMetadata>(null);
-        for (LogSegmentMetadata segment : segments) {
-            final CountDownLatch latch = new CountDownLatch(1);
-            lm.readLedgerMetadata(segment.getLogSegmentId(),
-                    new BookkeeperInternalCallbacks.GenericCallback<LedgerMetadata>() {
-                @Override
-                public void operationComplete(int rc, LedgerMetadata ledgerMetadata) {
-                    metadataHolder.set(ledgerMetadata);
-                    rcHolder.set(rc);
-                    latch.countDown();
-                }
-            });
-            latch.await();
-            if (BKException.Code.OK != rcHolder.get()) {
-                throw new IOException("Faild to get log segment metadata for " + segment + " : "
-                        + BKException.getMessage(rcHolder.get()));
-            }
-            inputSplits.add(new LogSegmentSplit(segment, metadataHolder.get()));
-        }
-        return inputSplits;
-    }
-
-    @Override
-    public RecordReader<DLSN, LogRecordWithDLSN> createRecordReader(
-            InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
-            throws IOException, InterruptedException {
-        return new LogSegmentReader(
-                streamName,
-                dlConf,
-                ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get(),
-                (LogSegmentSplit) inputSplit);
-    }
-}
diff --git a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/LogSegmentReader.java b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/LogSegmentReader.java
deleted file mode 100644
index f8b98f7..0000000
--- a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/LogSegmentReader.java
+++ /dev/null
@@ -1,148 +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 com.twitter.distributedlog.mapreduce;
-
-import com.twitter.distributedlog.*;
-import org.apache.bookkeeper.client.BKException;
-import org.apache.bookkeeper.client.BookKeeper;
-import org.apache.bookkeeper.client.LedgerEntry;
-import org.apache.bookkeeper.client.LedgerHandle;
-import org.apache.hadoop.mapreduce.InputSplit;
-import org.apache.hadoop.mapreduce.RecordReader;
-import org.apache.hadoop.mapreduce.TaskAttemptContext;
-
-import java.io.IOException;
-import java.util.Enumeration;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Record Reader to read from a log segment split
- */
-class LogSegmentReader extends RecordReader<DLSN, LogRecordWithDLSN> {
-
-    final String streamName;
-    final BookKeeper bk;
-    final LedgerHandle lh;
-    final LogSegmentMetadata metadata;
-
-    long entryId = -1L;
-    Entry.Reader reader = null;
-    LogRecordWithDLSN currentRecord = null;
-    int readPos = 0;
-
-    LogSegmentReader(String streamName,
-                     DistributedLogConfiguration conf,
-                     BookKeeper bk,
-                     LogSegmentSplit split)
-            throws IOException {
-        this.streamName = streamName;
-        this.bk = bk;
-        this.metadata = split.getMetadata();
-        try {
-            this.lh = bk.openLedgerNoRecovery(
-                    split.getLogSegmentId(),
-                    BookKeeper.DigestType.CRC32,
-                    conf.getBKDigestPW().getBytes(UTF_8));
-        } catch (BKException e) {
-            throw new IOException(e);
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw new IOException(e);
-        }
-    }
-
-    @Override
-    public void initialize(InputSplit split, TaskAttemptContext context)
-            throws IOException, InterruptedException {
-        // do nothing
-    }
-
-    @Override
-    public boolean nextKeyValue()
-            throws IOException, InterruptedException {
-        LogRecordWithDLSN record;
-        currentRecord = null;
-        if (null != reader) {
-            record = reader.nextRecord();
-            if (null != record) {
-                currentRecord = record;
-                readPos = record.getPositionWithinLogSegment();
-                return true;
-            } else {
-                return false;
-            }
-        }
-        ++entryId;
-        if (entryId > lh.getLastAddConfirmed()) {
-            return false;
-        }
-        try {
-            Enumeration<LedgerEntry> entries =
-                    lh.readEntries(entryId, entryId);
-            if (entries.hasMoreElements()) {
-                LedgerEntry entry = entries.nextElement();
-                reader = Entry.newBuilder()
-                        .setLogSegmentInfo(metadata.getLogSegmentSequenceNumber(),
-                                metadata.getStartSequenceId())
-                        .setEntryId(entry.getEntryId())
-                        .setEnvelopeEntry(
-                                LogSegmentMetadata.supportsEnvelopedEntries(metadata.getVersion()))
-                        .deserializeRecordSet(true)
-                        .setInputStream(entry.getEntryInputStream())
-                        .buildReader();
-            }
-            return nextKeyValue();
-        } catch (BKException e) {
-            throw new IOException(e);
-        }
-    }
-
-    @Override
-    public DLSN getCurrentKey()
-            throws IOException, InterruptedException {
-        return currentRecord.getDlsn();
-    }
-
-    @Override
-    public LogRecordWithDLSN getCurrentValue()
-            throws IOException, InterruptedException {
-        return currentRecord;
-    }
-
-    @Override
-    public float getProgress()
-            throws IOException, InterruptedException {
-        if (metadata.getRecordCount() > 0) {
-            return ((float) (readPos + 1)) / metadata.getRecordCount();
-        }
-        return 1;
-    }
-
-    @Override
-    public void close() throws IOException {
-        try {
-            lh.close();
-        } catch (InterruptedException e) {
-            Thread.currentThread().interrupt();
-            throw new IOException(e);
-        } catch (BKException e) {
-            throw new IOException(e);
-        }
-    }
-}
diff --git a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/LogSegmentSplit.java b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/LogSegmentSplit.java
deleted file mode 100644
index 89e9d44..0000000
--- a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/com/twitter/distributedlog/mapreduce/LogSegmentSplit.java
+++ /dev/null
@@ -1,95 +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 com.twitter.distributedlog.mapreduce;
-
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.LogSegmentMetadata;
-import org.apache.bookkeeper.client.LedgerMetadata;
-import org.apache.bookkeeper.net.BookieSocketAddress;
-import org.apache.bookkeeper.versioning.Version;
-import org.apache.hadoop.io.Writable;
-import org.apache.hadoop.mapreduce.InputSplit;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Set;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * A input split that reads from a log segment.
- */
-public class LogSegmentSplit extends InputSplit implements Writable {
-
-    private LogSegmentMetadata logSegmentMetadata;
-    private LedgerMetadata ledgerMetadata;
-
-    public LogSegmentSplit() {}
-
-    public LogSegmentSplit(LogSegmentMetadata logSegmentMetadata,
-                           LedgerMetadata ledgerMetadata) {
-        this.logSegmentMetadata = logSegmentMetadata;
-        this.ledgerMetadata = ledgerMetadata;
-    }
-
-    public LogSegmentMetadata getMetadata() {
-        return logSegmentMetadata;
-    }
-
-    public long getLogSegmentId() {
-        return logSegmentMetadata.getLogSegmentId();
-    }
-
-    @Override
-    public long getLength()
-            throws IOException, InterruptedException {
-        return logSegmentMetadata.getRecordCount();
-    }
-
-    @Override
-    public String[] getLocations()
-            throws IOException, InterruptedException {
-        Set<String> locations = Sets.newHashSet();
-        for (ArrayList<BookieSocketAddress> ensemble : ledgerMetadata.getEnsembles().values()) {
-            for (BookieSocketAddress host : ensemble) {
-                locations.add(host.getHostName());
-            }
-        }
-        return locations.toArray(new String[locations.size()]);
-    }
-
-    @Override
-    public void write(DataOutput dataOutput) throws IOException {
-        String lsMetadataStr = logSegmentMetadata.getFinalisedData();
-        dataOutput.writeUTF(lsMetadataStr);
-        String lhMetadataStr = new String(ledgerMetadata.serialize(), UTF_8);
-        dataOutput.writeUTF(lhMetadataStr);
-    }
-
-    @Override
-    public void readFields(DataInput dataInput) throws IOException {
-        String lsMetadataStr = dataInput.readUTF();
-        logSegmentMetadata = LogSegmentMetadata.parseData("",
-                lsMetadataStr.getBytes(UTF_8));
-        String lhMetadataStr = dataInput.readUTF();
-        ledgerMetadata = LedgerMetadata.parseConfig(lhMetadataStr.getBytes(UTF_8),
-                Version.ANY);
-    }
-}
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
new file mode 100644
index 0000000..94a53d4
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/org/apache/distributedlog/mapreduce/DistributedLogInputFormat.java
@@ -0,0 +1,129 @@
+/**
+ * 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.mapreduce;
+
+import com.google.common.collect.Lists;
+import org.apache.distributedlog.DistributedLogConfiguration;
+import org.apache.distributedlog.DistributedLogManager;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.LogRecordWithDLSN;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.distributedlog.impl.BKNamespaceDriver;
+import org.apache.distributedlog.namespace.DistributedLogNamespace;
+import org.apache.distributedlog.namespace.DistributedLogNamespaceBuilder;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.BookKeeperAccessor;
+import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.meta.LedgerManager;
+import org.apache.bookkeeper.proto.BookkeeperInternalCallbacks;
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputFormat;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.JobContext;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import java.io.IOException;
+import java.net.URI;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicInteger;
+import java.util.concurrent.atomic.AtomicReference;
+
+/**
+ * InputFormat to read data from a distributedlog stream.
+ */
+public class DistributedLogInputFormat
+        extends InputFormat<DLSN, LogRecordWithDLSN> implements Configurable {
+
+    private static final String DL_URI = "distributedlog.uri";
+    private static final String DL_STREAM = "distributedlog.stream";
+
+    protected Configuration conf;
+    protected DistributedLogConfiguration dlConf;
+    protected URI dlUri;
+    protected DistributedLogNamespace namespace;
+    protected String streamName;
+    protected DistributedLogManager dlm;
+
+    /** {@inheritDoc} */
+    @Override
+    public void setConf(Configuration configuration) {
+        this.conf = configuration;
+        dlConf = new DistributedLogConfiguration();
+        dlUri = URI.create(configuration.get(DL_URI, ""));
+        streamName = configuration.get(DL_STREAM, "");
+        try {
+            namespace = DistributedLogNamespaceBuilder.newBuilder()
+                    .conf(dlConf)
+                    .uri(dlUri)
+                    .build();
+            dlm = namespace.openLog(streamName);
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    @Override
+    public Configuration getConf() {
+        return conf;
+    }
+
+    @Override
+    public List<InputSplit> getSplits(JobContext jobContext)
+            throws IOException, InterruptedException {
+        List<LogSegmentMetadata> segments = dlm.getLogSegments();
+        List<InputSplit> inputSplits = Lists.newArrayListWithCapacity(segments.size());
+        BookKeeper bk = ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get();
+        LedgerManager lm = BookKeeperAccessor.getLedgerManager(bk);
+        final AtomicInteger rcHolder = new AtomicInteger(0);
+        final AtomicReference<LedgerMetadata> metadataHolder = new AtomicReference<LedgerMetadata>(null);
+        for (LogSegmentMetadata segment : segments) {
+            final CountDownLatch latch = new CountDownLatch(1);
+            lm.readLedgerMetadata(segment.getLogSegmentId(),
+                    new BookkeeperInternalCallbacks.GenericCallback<LedgerMetadata>() {
+                @Override
+                public void operationComplete(int rc, LedgerMetadata ledgerMetadata) {
+                    metadataHolder.set(ledgerMetadata);
+                    rcHolder.set(rc);
+                    latch.countDown();
+                }
+            });
+            latch.await();
+            if (BKException.Code.OK != rcHolder.get()) {
+                throw new IOException("Faild to get log segment metadata for " + segment + " : "
+                        + BKException.getMessage(rcHolder.get()));
+            }
+            inputSplits.add(new LogSegmentSplit(segment, metadataHolder.get()));
+        }
+        return inputSplits;
+    }
+
+    @Override
+    public RecordReader<DLSN, LogRecordWithDLSN> createRecordReader(
+            InputSplit inputSplit, TaskAttemptContext taskAttemptContext)
+            throws IOException, InterruptedException {
+        return new LogSegmentReader(
+                streamName,
+                dlConf,
+                ((BKNamespaceDriver) namespace.getNamespaceDriver()).getReaderBKC().get(),
+                (LogSegmentSplit) inputSplit);
+    }
+}
diff --git a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/org/apache/distributedlog/mapreduce/LogSegmentReader.java b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/org/apache/distributedlog/mapreduce/LogSegmentReader.java
new file mode 100644
index 0000000..541db3b
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/org/apache/distributedlog/mapreduce/LogSegmentReader.java
@@ -0,0 +1,148 @@
+/**
+ * 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.mapreduce;
+
+import org.apache.distributedlog.*;
+import org.apache.bookkeeper.client.BKException;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.LedgerEntry;
+import org.apache.bookkeeper.client.LedgerHandle;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.hadoop.mapreduce.RecordReader;
+import org.apache.hadoop.mapreduce.TaskAttemptContext;
+
+import java.io.IOException;
+import java.util.Enumeration;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Record Reader to read from a log segment split
+ */
+class LogSegmentReader extends RecordReader<DLSN, LogRecordWithDLSN> {
+
+    final String streamName;
+    final BookKeeper bk;
+    final LedgerHandle lh;
+    final LogSegmentMetadata metadata;
+
+    long entryId = -1L;
+    Entry.Reader reader = null;
+    LogRecordWithDLSN currentRecord = null;
+    int readPos = 0;
+
+    LogSegmentReader(String streamName,
+                     DistributedLogConfiguration conf,
+                     BookKeeper bk,
+                     LogSegmentSplit split)
+            throws IOException {
+        this.streamName = streamName;
+        this.bk = bk;
+        this.metadata = split.getMetadata();
+        try {
+            this.lh = bk.openLedgerNoRecovery(
+                    split.getLogSegmentId(),
+                    BookKeeper.DigestType.CRC32,
+                    conf.getBKDigestPW().getBytes(UTF_8));
+        } catch (BKException e) {
+            throw new IOException(e);
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new IOException(e);
+        }
+    }
+
+    @Override
+    public void initialize(InputSplit split, TaskAttemptContext context)
+            throws IOException, InterruptedException {
+        // do nothing
+    }
+
+    @Override
+    public boolean nextKeyValue()
+            throws IOException, InterruptedException {
+        LogRecordWithDLSN record;
+        currentRecord = null;
+        if (null != reader) {
+            record = reader.nextRecord();
+            if (null != record) {
+                currentRecord = record;
+                readPos = record.getPositionWithinLogSegment();
+                return true;
+            } else {
+                return false;
+            }
+        }
+        ++entryId;
+        if (entryId > lh.getLastAddConfirmed()) {
+            return false;
+        }
+        try {
+            Enumeration<LedgerEntry> entries =
+                    lh.readEntries(entryId, entryId);
+            if (entries.hasMoreElements()) {
+                LedgerEntry entry = entries.nextElement();
+                reader = Entry.newBuilder()
+                        .setLogSegmentInfo(metadata.getLogSegmentSequenceNumber(),
+                                metadata.getStartSequenceId())
+                        .setEntryId(entry.getEntryId())
+                        .setEnvelopeEntry(
+                                LogSegmentMetadata.supportsEnvelopedEntries(metadata.getVersion()))
+                        .deserializeRecordSet(true)
+                        .setInputStream(entry.getEntryInputStream())
+                        .buildReader();
+            }
+            return nextKeyValue();
+        } catch (BKException e) {
+            throw new IOException(e);
+        }
+    }
+
+    @Override
+    public DLSN getCurrentKey()
+            throws IOException, InterruptedException {
+        return currentRecord.getDlsn();
+    }
+
+    @Override
+    public LogRecordWithDLSN getCurrentValue()
+            throws IOException, InterruptedException {
+        return currentRecord;
+    }
+
+    @Override
+    public float getProgress()
+            throws IOException, InterruptedException {
+        if (metadata.getRecordCount() > 0) {
+            return ((float) (readPos + 1)) / metadata.getRecordCount();
+        }
+        return 1;
+    }
+
+    @Override
+    public void close() throws IOException {
+        try {
+            lh.close();
+        } catch (InterruptedException e) {
+            Thread.currentThread().interrupt();
+            throw new IOException(e);
+        } catch (BKException e) {
+            throw new IOException(e);
+        }
+    }
+}
diff --git a/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/org/apache/distributedlog/mapreduce/LogSegmentSplit.java b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/org/apache/distributedlog/mapreduce/LogSegmentSplit.java
new file mode 100644
index 0000000..132e24d
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-mapreduce/src/main/java/org/apache/distributedlog/mapreduce/LogSegmentSplit.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.mapreduce;
+
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.LogSegmentMetadata;
+import org.apache.bookkeeper.client.LedgerMetadata;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.versioning.Version;
+import org.apache.hadoop.io.Writable;
+import org.apache.hadoop.mapreduce.InputSplit;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Set;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * A input split that reads from a log segment.
+ */
+public class LogSegmentSplit extends InputSplit implements Writable {
+
+    private LogSegmentMetadata logSegmentMetadata;
+    private LedgerMetadata ledgerMetadata;
+
+    public LogSegmentSplit() {}
+
+    public LogSegmentSplit(LogSegmentMetadata logSegmentMetadata,
+                           LedgerMetadata ledgerMetadata) {
+        this.logSegmentMetadata = logSegmentMetadata;
+        this.ledgerMetadata = ledgerMetadata;
+    }
+
+    public LogSegmentMetadata getMetadata() {
+        return logSegmentMetadata;
+    }
+
+    public long getLogSegmentId() {
+        return logSegmentMetadata.getLogSegmentId();
+    }
+
+    @Override
+    public long getLength()
+            throws IOException, InterruptedException {
+        return logSegmentMetadata.getRecordCount();
+    }
+
+    @Override
+    public String[] getLocations()
+            throws IOException, InterruptedException {
+        Set<String> locations = Sets.newHashSet();
+        for (ArrayList<BookieSocketAddress> ensemble : ledgerMetadata.getEnsembles().values()) {
+            for (BookieSocketAddress host : ensemble) {
+                locations.add(host.getHostName());
+            }
+        }
+        return locations.toArray(new String[locations.size()]);
+    }
+
+    @Override
+    public void write(DataOutput dataOutput) throws IOException {
+        String lsMetadataStr = logSegmentMetadata.getFinalisedData();
+        dataOutput.writeUTF(lsMetadataStr);
+        String lhMetadataStr = new String(ledgerMetadata.serialize(), UTF_8);
+        dataOutput.writeUTF(lhMetadataStr);
+    }
+
+    @Override
+    public void readFields(DataInput dataInput) throws IOException {
+        String lsMetadataStr = dataInput.readUTF();
+        logSegmentMetadata = LogSegmentMetadata.parseData("",
+                lsMetadataStr.getBytes(UTF_8));
+        String lhMetadataStr = dataInput.readUTF();
+        ledgerMetadata = LedgerMetadata.parseConfig(lhMetadataStr.getBytes(UTF_8),
+                Version.ANY);
+    }
+}
diff --git a/distributedlog-tutorials/distributedlog-messaging/conf/log4j.properties b/distributedlog-tutorials/distributedlog-messaging/conf/log4j.properties
index cc6486b..f3d3ad1 100644
--- a/distributedlog-tutorials/distributedlog-messaging/conf/log4j.properties
+++ b/distributedlog-tutorials/distributedlog-messaging/conf/log4j.properties
@@ -32,11 +32,11 @@
 log4j.logger.org.apache.bookkeeper=INFO
 
 # redirect executor output to executors.log since slow op warnings can be quite verbose
-log4j.logger.com.twitter.distributedlog.util.MonitoredFuturePool=INFO, Executors
-log4j.logger.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=INFO, Executors
+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.com.twitter.distributedlog.util.MonitoredFuturePool=false
-log4j.additivity.com.twitter.distributedlog.util.MonitoredScheduledThreadPoolExecutor=false
+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/messaging-1.md b/distributedlog-tutorials/distributedlog-messaging/messaging-1.md
index 2da540d..d2c2d7f 100644
--- a/distributedlog-tutorials/distributedlog-messaging/messaging-1.md
+++ b/distributedlog-tutorials/distributedlog-messaging/messaging-1.md
@@ -44,7 +44,7 @@
 2.  Start the write proxy, listening on port 8000.
     ```
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
     ```
 
 3.  Create the stream under the distributedlog uri.
@@ -58,15 +58,15 @@
 4.  Tailing the stream using `MultiReader` to read from multiple streams.
     ```
         // Tailing Stream `messaging-stream-{1,5}`
-        // runner run com.twitter.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[, ${stream}]
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+        // runner run org.apache.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[, ${stream}]
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
     ```
 
 6.  Run the example to write records to multiple stream in a console - the record should be in the form of `KEY:VALUE`.
     ```
         // Write Records into Stream `messaging-stream-{1,5}`
-        // runner run com.twitter.distributedlog.messaging.ConsoleProxyPartitionedMultiWriter ${distributedlog-uri} ${stream}[, ${stream}]
-        ./distributedlog-tutorials/distributedlog-messaging/bin/runner run com.twitter.distributedlog.messaging.ConsoleProxyPartitionedMultiWriter 'inet!127.0.0.1:8000' messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+        // runner run org.apache.distributedlog.messaging.ConsoleProxyPartitionedMultiWriter ${distributedlog-uri} ${stream}[, ${stream}]
+        ./distributedlog-tutorials/distributedlog-messaging/bin/runner run org.apache.distributedlog.messaging.ConsoleProxyPartitionedMultiWriter 'inet!127.0.0.1:8000' messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
     ```
 
 7.  Example output from `ConsoleProxyPartitionedMultiWriter` and `MultiReader`.
diff --git a/distributedlog-tutorials/distributedlog-messaging/messaging-2.md b/distributedlog-tutorials/distributedlog-messaging/messaging-2.md
index 3dc25c5..5622888 100644
--- a/distributedlog-tutorials/distributedlog-messaging/messaging-2.md
+++ b/distributedlog-tutorials/distributedlog-messaging/messaging-2.md
@@ -76,7 +76,7 @@
 2.  Start the write proxy, listening on port 8000.
     ```
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
     ```
 
 3.  Create the stream under the distributedlog uri.
@@ -90,15 +90,15 @@
 4.  Tailing the stream using `MultiReader` to read from multiple streams.
     ```
         // Tailing Stream `messaging-stream-{1,5}`
-        // runner run com.twitter.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[, ${stream}]
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+        // runner run org.apache.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[, ${stream}]
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
     ```
 
 6.  Run the example to write records to multiple stream in a console.
     ```
         // Write Records into Stream `messaging-stream-{1,5}`
-        // runner run com.twitter.distributedlog.messaging.ConsoleProxyRRMultiWriter ${distributedlog-uri} ${stream}[, ${stream}]
-        ./distributedlog-tutorials/distributedlog-messaging/bin/runner run com.twitter.distributedlog.messaging.ConsoleProxyRRMultiWriter 'inet!127.0.0.1:8000' messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+        // runner run org.apache.distributedlog.messaging.ConsoleProxyRRMultiWriter ${distributedlog-uri} ${stream}[, ${stream}]
+        ./distributedlog-tutorials/distributedlog-messaging/bin/runner run org.apache.distributedlog.messaging.ConsoleProxyRRMultiWriter 'inet!127.0.0.1:8000' messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
     ```
 
 7.  Example output from `ConsoleProxyRRMultiWriter` and `MultiReader`.
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/ConsoleProxyPartitionedMultiWriter.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/ConsoleProxyPartitionedMultiWriter.java
deleted file mode 100644
index da804b6..0000000
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/ConsoleProxyPartitionedMultiWriter.java
+++ /dev/null
@@ -1,121 +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 com.twitter.distributedlog.messaging;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.Future;
-import com.twitter.util.FutureEventListener;
-import jline.ConsoleReader;
-import jline.Terminal;
-import org.apache.commons.lang.StringUtils;
-
-import java.nio.ByteBuffer;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Writer write records from console
- */
-public class ConsoleProxyPartitionedMultiWriter {
-
-    private final static String HELP = "ConsoleProxyPartitionedMultiWriter <finagle-name> <stream-1>[,<stream-2>,...,<stream-n>]";
-    private final static String PROMPT_MESSAGE = "[dlog] > ";
-
-    static class PartitionedWriter<KEY, VALUE> {
-
-        private final String[] streams;
-        private final Partitioner<KEY> partitioner;
-        private final DistributedLogClient client;
-
-        PartitionedWriter(String[] streams,
-                          Partitioner<KEY> partitioner,
-                          DistributedLogClient client) {
-            this.streams = streams;
-            this.partitioner = partitioner;
-            this.client = client;
-        }
-
-        Future<DLSN> write(KEY key, VALUE value) {
-            int pid = partitioner.partition(key, streams.length);
-            return client.write(streams[pid], ByteBuffer.wrap(value.toString().getBytes(UTF_8)));
-        }
-
-    }
-
-    public static void main(String[] args) throws Exception {
-        if (2 != args.length) {
-            System.out.println(HELP);
-            return;
-        }
-
-        String finagleNameStr = args[0];
-        final String streamList = args[1];
-
-        DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
-                .clientId(ClientId$.MODULE$.apply("console-proxy-writer"))
-                .name("console-proxy-writer")
-                .thriftmux(true)
-                .finagleNameStr(finagleNameStr)
-                .build();
-        String[] streamNameList = StringUtils.split(streamList, ',');
-        PartitionedWriter<Integer, String> partitionedWriter =
-                new PartitionedWriter<Integer, String>(
-                        streamNameList,
-                        new IntPartitioner(),
-                        client);
-
-        ConsoleReader reader = new ConsoleReader();
-        String line;
-        while ((line = reader.readLine(PROMPT_MESSAGE)) != null) {
-            String[] parts = StringUtils.split(line, ':');
-            if (parts.length != 2) {
-                System.out.println("Invalid input. Needs 'KEY:VALUE'");
-                continue;
-            }
-            int key;
-            try {
-                key = Integer.parseInt(parts[0]);
-            } catch (NumberFormatException nfe) {
-                System.out.println("Invalid input. Needs 'KEY:VALUE'");
-                continue;
-            }
-            String value = parts[1];
-
-            partitionedWriter.write(key, value)
-                    .addEventListener(new FutureEventListener<DLSN>() {
-                        @Override
-                        public void onFailure(Throwable cause) {
-                            System.out.println("Encountered error on writing data");
-                            cause.printStackTrace(System.err);
-                            Runtime.getRuntime().exit(0);
-                        }
-
-                        @Override
-                        public void onSuccess(DLSN value) {
-                            // done
-                        }
-                    });
-        }
-
-        client.close();
-    }
-
-}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/ConsoleProxyRRMultiWriter.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/ConsoleProxyRRMultiWriter.java
deleted file mode 100644
index 142aec2..0000000
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/ConsoleProxyRRMultiWriter.java
+++ /dev/null
@@ -1,77 +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 com.twitter.distributedlog.messaging;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.distributedlog.service.DistributedLogClientBuilder;
-import com.twitter.finagle.thrift.ClientId$;
-import com.twitter.util.FutureEventListener;
-import jline.ConsoleReader;
-import jline.Terminal;
-import org.apache.commons.lang.StringUtils;
-
-/**
- * Writer write records into multiple streams
- */
-public class ConsoleProxyRRMultiWriter {
-
-    private final static String HELP = "ConsoleProxyRRMultiWriter <finagle-name> <stream-1>[,<stream-2>,...,<stream-n>]";
-    private final static String PROMPT_MESSAGE = "[dlog] > ";
-
-    public static void main(String[] args) throws Exception {
-        if (2 != args.length) {
-            System.out.println(HELP);
-            return;
-        }
-
-        String finagleNameStr = args[0];
-        final String streamList = args[1];
-
-        DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
-                .clientId(ClientId$.MODULE$.apply("console-proxy-writer"))
-                .name("console-proxy-writer")
-                .thriftmux(true)
-                .finagleNameStr(finagleNameStr)
-                .build();
-        String[] streamNameList = StringUtils.split(streamList, ',');
-        RRMultiWriter<Integer, String> writer = new RRMultiWriter(streamNameList, client);
-
-        ConsoleReader reader = new ConsoleReader();
-        String line;
-        while ((line = reader.readLine(PROMPT_MESSAGE)) != null) {
-            writer.write(line)
-                    .addEventListener(new FutureEventListener<DLSN>() {
-                        @Override
-                        public void onFailure(Throwable cause) {
-                            System.out.println("Encountered error on writing data");
-                            cause.printStackTrace(System.err);
-                            Runtime.getRuntime().exit(0);
-                        }
-
-                        @Override
-                        public void onSuccess(DLSN value) {
-                            // done
-                        }
-                    });
-        }
-
-        client.close();
-    }
-
-}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/IdenticalTransformer.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/IdenticalTransformer.java
deleted file mode 100644
index 3514f8f..0000000
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/IdenticalTransformer.java
+++ /dev/null
@@ -1,25 +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 com.twitter.distributedlog.messaging;
-
-public class IdenticalTransformer<T> implements Transformer<T, T> {
-    @Override
-    public T transform(T value) {
-        return value;
-    }
-}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/IntPartitioner.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/IntPartitioner.java
deleted file mode 100644
index 02d09ae..0000000
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/IntPartitioner.java
+++ /dev/null
@@ -1,28 +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 com.twitter.distributedlog.messaging;
-
-/**
- * Partitioner where key is an integer
- */
-public class IntPartitioner implements Partitioner<Integer> {
-    @Override
-    public int partition(Integer key, int totalPartitions) {
-        return key % totalPartitions;
-    }
-}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/PartitionedMultiWriter.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/PartitionedMultiWriter.java
deleted file mode 100644
index 690b90b..0000000
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/PartitionedMultiWriter.java
+++ /dev/null
@@ -1,50 +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 com.twitter.distributedlog.messaging;
-
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.util.Future;
-
-import java.nio.ByteBuffer;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Partitioned Writer
- */
-public class PartitionedMultiWriter<KEY, VALUE> {
-
-    private final String[] streams;
-    private final Partitioner<KEY> partitioner;
-    private final DistributedLogClient client;
-
-    public PartitionedMultiWriter(String[] streams,
-                                  Partitioner<KEY> partitioner,
-                                  DistributedLogClient client) {
-        this.streams = streams;
-        this.partitioner = partitioner;
-        this.client = client;
-    }
-
-    public Future<DLSN> write(KEY key, VALUE value) {
-        int pid = partitioner.partition(key, streams.length);
-        return client.write(streams[pid], ByteBuffer.wrap(value.toString().getBytes(UTF_8)));
-    }
-
-}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/Partitioner.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/Partitioner.java
deleted file mode 100644
index 913d93a..0000000
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/Partitioner.java
+++ /dev/null
@@ -1,25 +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 com.twitter.distributedlog.messaging;
-
-/**
- * Partitioner
- */
-public interface Partitioner<KEY> {
-    int partition(KEY key, int totalPartitions);
-}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/RRMultiWriter.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/RRMultiWriter.java
deleted file mode 100644
index d0eac7a..0000000
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/RRMultiWriter.java
+++ /dev/null
@@ -1,94 +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 com.twitter.distributedlog.messaging;
-
-import com.google.common.collect.Sets;
-import com.twitter.distributedlog.DLSN;
-import com.twitter.distributedlog.service.DistributedLogClient;
-import com.twitter.finagle.NoBrokersAvailableException;
-import com.twitter.finagle.Service;
-import com.twitter.finagle.ServiceFactory;
-import com.twitter.finagle.loadbalancer.Balancers;
-import com.twitter.finagle.service.SingletonFactory;
-import com.twitter.finagle.stats.NullStatsReceiver;
-import com.twitter.util.Activity;
-import com.twitter.util.Future;
-import scala.collection.JavaConversions;
-
-import java.nio.ByteBuffer;
-import java.util.Set;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Multi stream writer that leverages finagle load balancer.
- */
-public class RRMultiWriter<KEY, VALUE> {
-
-    static class StreamWriter<VALUE> extends Service<VALUE, DLSN> {
-
-        private final String stream;
-        private final DistributedLogClient client;
-
-        StreamWriter(String stream,
-                     DistributedLogClient client) {
-            this.stream = stream;
-            this.client = client;
-        }
-
-        @Override
-        public Future<DLSN> apply(VALUE request) {
-            return client.write(stream, ByteBuffer.wrap(request.toString().getBytes(UTF_8)));
-        }
-    }
-
-
-    static <VALUE> Set<ServiceFactory<VALUE, DLSN>> initializeServices(
-            String[] streams, DistributedLogClient client) {
-        Set<ServiceFactory<VALUE, DLSN>> serviceFactories =
-                Sets.newHashSet();
-        for (String stream : streams) {
-            Service<VALUE, DLSN> service = new StreamWriter(stream, client);
-            serviceFactories.add(new SingletonFactory<VALUE, DLSN>(service));
-        }
-        return serviceFactories;
-    }
-
-    private final String[] streams;
-    private final DistributedLogClient client;
-    private final Service<VALUE, DLSN> service;
-
-    public RRMultiWriter(String[] streams,
-                         DistributedLogClient client) {
-        this.streams = streams;
-        this.client = client;
-        scala.collection.immutable.Set<ServiceFactory<VALUE, DLSN>> scalaSet =
-                JavaConversions.asScalaSet(initializeServices(streams, client)).toSet();
-        this.service = Balancers.heap(new scala.util.Random(System.currentTimeMillis()))
-                .newBalancer(
-                        Activity.value(scalaSet),
-                        NullStatsReceiver.get(),
-                        new NoBrokersAvailableException("No partitions available")
-                ).toService();
-    }
-
-    public Future<DLSN> write(VALUE data) {
-        return service.apply(data);
-    }
-
-}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/ReaderWithOffsets.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/ReaderWithOffsets.java
deleted file mode 100644
index 7a9d888..0000000
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/ReaderWithOffsets.java
+++ /dev/null
@@ -1,132 +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 com.twitter.distributedlog.messaging;
-
-import com.twitter.distributedlog.*;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.util.FutureUtils;
-import com.twitter.util.Duration;
-import com.twitter.util.FutureEventListener;
-import org.iq80.leveldb.DB;
-import org.iq80.leveldb.Options;
-
-import java.io.File;
-import java.net.URI;
-import java.util.concurrent.*;
-import java.util.concurrent.atomic.AtomicReference;
-
-import static com.google.common.base.Charsets.UTF_8;
-import static org.iq80.leveldb.impl.Iq80DBFactory.*;
-
-/**
- * Reader with offsets
- */
-public class ReaderWithOffsets {
-
-    private final static String HELP = "ReaderWithOffsets <uri> <string> <reader-id> <offset-store-file>";
-
-    public static void main(String[] args) throws Exception {
-        if (4 != args.length) {
-            System.out.println(HELP);
-            return;
-        }
-
-        String dlUriStr = args[0];
-        final String streamName = args[1];
-        final String readerId = args[2];
-        final String offsetStoreFile = args[3];
-
-        URI uri = URI.create(dlUriStr);
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(uri)
-                .build();
-
-        // open the dlm
-        System.out.println("Opening log stream " + streamName);
-        DistributedLogManager dlm = namespace.openLog(streamName);
-
-        // open the offset store
-        Options options = new Options();
-        options.createIfMissing(true);
-        final DB offsetDB = factory.open(new File(offsetStoreFile), options);
-        final AtomicReference<DLSN> lastDLSN = new AtomicReference<DLSN>(null);
-        // offset updater
-        final ScheduledExecutorService executorService =
-                Executors.newSingleThreadScheduledExecutor();
-        executorService.scheduleAtFixedRate(new Runnable() {
-            @Override
-            public void run() {
-                if (null != lastDLSN.get()) {
-                    offsetDB.put(readerId.getBytes(UTF_8), lastDLSN.get().serializeBytes());
-                    System.out.println("Updated reader " + readerId + " offset to " + lastDLSN.get());
-                }
-            }
-        }, 10, 10, TimeUnit.SECONDS);
-        try {
-            byte[] offset = offsetDB.get(readerId.getBytes(UTF_8));
-            DLSN dlsn;
-            if (null == offset) {
-                dlsn = DLSN.InitialDLSN;
-            } else {
-                dlsn = DLSN.deserializeBytes(offset);
-            }
-            readLoop(dlm, dlsn, lastDLSN);
-        } finally {
-            offsetDB.close();
-            dlm.close();
-            namespace.close();
-        }
-    }
-
-    private static void readLoop(final DistributedLogManager dlm,
-                                 final DLSN dlsn,
-                                 final AtomicReference<DLSN> lastDLSN)
-            throws Exception {
-
-        final CountDownLatch keepAliveLatch = new CountDownLatch(1);
-
-        System.out.println("Wait for records starting from " + dlsn);
-        final AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(dlsn));
-        final FutureEventListener<LogRecordWithDLSN> readListener = new FutureEventListener<LogRecordWithDLSN>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                System.err.println("Encountered error on reading records from stream " + dlm.getStreamName());
-                cause.printStackTrace(System.err);
-                keepAliveLatch.countDown();
-            }
-
-            @Override
-            public void onSuccess(LogRecordWithDLSN record) {
-                System.out.println("Received record " + record.getDlsn());
-                System.out.println("\"\"\"");
-                System.out.println(new String(record.getPayload(), UTF_8));
-                System.out.println("\"\"\"");
-                lastDLSN.set(record.getDlsn());
-                reader.readNext().addEventListener(this);
-            }
-        };
-        reader.readNext().addEventListener(readListener);
-
-        keepAliveLatch.await();
-        FutureUtils.result(reader.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
-    }
-
-}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/StreamTransformer.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/StreamTransformer.java
deleted file mode 100644
index 95ac3c9..0000000
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/StreamTransformer.java
+++ /dev/null
@@ -1,188 +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 com.twitter.distributedlog.messaging;
-
-import com.twitter.distributedlog.*;
-import com.twitter.distributedlog.exceptions.LogEmptyException;
-import com.twitter.distributedlog.exceptions.LogNotFoundException;
-import com.twitter.distributedlog.namespace.DistributedLogNamespace;
-import com.twitter.distributedlog.namespace.DistributedLogNamespaceBuilder;
-import com.twitter.distributedlog.thrift.messaging.TransformedRecord;
-import com.twitter.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 java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.net.URI;
-import java.nio.ByteBuffer;
-import java.util.concurrent.CountDownLatch;
-import java.util.concurrent.TimeUnit;
-
-import static com.google.common.base.Charsets.UTF_8;
-
-/**
- * Transform one stream to another stream. And apply transformation
- */
-public class StreamTransformer {
-
-    private final static String HELP = "StreamTransformer <uri> <src_stream> <target_stream>";
-    private final static TProtocolFactory protocolFactory =
-            new TBinaryProtocol.Factory();
-
-    public static void main(String[] args) throws Exception {
-        if (3 != args.length) {
-            System.out.println(HELP);
-            return;
-        }
-
-        String dlUriStr = args[0];
-        final String srcStreamName = args[1];
-        final String targetStreamName = args[2];
-
-        URI uri = URI.create(dlUriStr);
-        DistributedLogConfiguration conf = new DistributedLogConfiguration();
-        conf.setOutputBufferSize(16*1024); // 16KB
-        conf.setPeriodicFlushFrequencyMilliSeconds(5); // 5ms
-        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
-                .conf(conf)
-                .uri(uri)
-                .build();
-
-        // open the dlm
-        System.out.println("Opening log stream " + srcStreamName);
-        DistributedLogManager srcDlm = namespace.openLog(srcStreamName);
-        System.out.println("Opening log stream " + targetStreamName);
-        DistributedLogManager targetDlm = namespace.openLog(targetStreamName);
-
-        Transformer<byte[], byte[]> replicationTransformer =
-                new IdenticalTransformer<byte[]>();
-
-        LogRecordWithDLSN lastTargetRecord;
-        DLSN srcDlsn;
-        try {
-            lastTargetRecord = targetDlm.getLastLogRecord();
-            TransformedRecord lastTransformedRecord = new TransformedRecord();
-            try {
-                lastTransformedRecord.read(protocolFactory.getProtocol(
-                        new TIOStreamTransport(new ByteArrayInputStream(lastTargetRecord.getPayload()))));
-                srcDlsn = DLSN.deserializeBytes(lastTransformedRecord.getSrcDlsn());
-                System.out.println("Last transformed record is " + srcDlsn);
-            } catch (TException e) {
-                System.err.println("Error on reading last transformed record");
-                e.printStackTrace(System.err);
-                srcDlsn = DLSN.InitialDLSN;
-            }
-        } catch (LogNotFoundException lnfe) {
-            srcDlsn = DLSN.InitialDLSN;
-        } catch (LogEmptyException lee) {
-            srcDlsn = DLSN.InitialDLSN;
-        }
-
-        AsyncLogWriter targetWriter = FutureUtils.result(targetDlm.openAsyncLogWriter());
-        try {
-            readLoop(srcDlm, srcDlsn, targetWriter, replicationTransformer);
-        } finally {
-            FutureUtils.result(targetWriter.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
-            targetDlm.close();
-            srcDlm.close();
-            namespace.close();
-        }
-
-    }
-
-    private static void readLoop(final DistributedLogManager dlm,
-                                 final DLSN fromDLSN,
-                                 final AsyncLogWriter targetWriter,
-                                 final Transformer<byte[], byte[]> replicationTransformer)
-            throws Exception {
-
-        final CountDownLatch keepAliveLatch = new CountDownLatch(1);
-
-        System.out.println("Wait for records starting from " + fromDLSN);
-        final AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(fromDLSN));
-        final FutureEventListener<LogRecordWithDLSN> readListener = new FutureEventListener<LogRecordWithDLSN>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                System.err.println("Encountered error on reading records from stream " + dlm.getStreamName());
-                cause.printStackTrace(System.err);
-                keepAliveLatch.countDown();
-            }
-
-            @Override
-            public void onSuccess(LogRecordWithDLSN record) {
-                if (record.getDlsn().compareTo(fromDLSN) <= 0) {
-                    reader.readNext().addEventListener(this);
-                    return;
-                }
-                System.out.println("Received record " + record.getDlsn());
-                System.out.println("\"\"\"");
-                System.out.println(new String(record.getPayload(), UTF_8));
-                System.out.println("\"\"\"");
-                try {
-                    transform(targetWriter, record, replicationTransformer, keepAliveLatch);
-                } catch (Exception e) {
-                    System.err.println("Encountered error on transforming record " + record.getDlsn()
-                            + " from stream " + dlm.getStreamName());
-                    e.printStackTrace(System.err);
-                    keepAliveLatch.countDown();
-                }
-                reader.readNext().addEventListener(this);
-            }
-        };
-        reader.readNext().addEventListener(readListener);
-
-        keepAliveLatch.await();
-        FutureUtils.result(reader.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
-    }
-
-    private static void transform(final AsyncLogWriter writer,
-                                  LogRecordWithDLSN record,
-                                  Transformer<byte[], byte[]> replicationTransformer,
-                                  final CountDownLatch keepAliveLatch)
-            throws Exception {
-        DLSN srcDLSN = record.getDlsn();
-        byte[] payload = record.getPayload();
-        byte[] transformedPayload = replicationTransformer.transform(payload);
-        TransformedRecord transformedRecord =
-                new TransformedRecord(ByteBuffer.wrap(transformedPayload));
-        transformedRecord.setSrcDlsn(srcDLSN.serializeBytes());
-        ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
-        transformedRecord.write(protocolFactory.getProtocol(new TIOStreamTransport(baos)));
-        byte[] data = baos.toByteArray();
-        writer.write(new LogRecord(record.getSequenceId(), data))
-                .addEventListener(new FutureEventListener<DLSN>() {
-            @Override
-            public void onFailure(Throwable cause) {
-                System.err.println("Encountered error on writing records to stream " + writer.getStreamName());
-                cause.printStackTrace(System.err);
-                keepAliveLatch.countDown();
-            }
-
-            @Override
-            public void onSuccess(DLSN dlsn) {
-                System.out.println("Write transformed record " + dlsn);
-            }
-        });
-    }
-
-}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/Transformer.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/Transformer.java
deleted file mode 100644
index eb82ae3..0000000
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/java/com/twitter/distributedlog/messaging/Transformer.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 com.twitter.distributedlog.messaging;
-
-/**
- * Transform one format to the other format.
- */
-public interface Transformer<T, R> {
-
-    /**
-     * Transform value <i>T</i> to value <i>R</i>. If null is returned,
-     * the value <i>T</i> will be eliminated without transforming.
-     *
-     * @param value value to transform
-     * @return transformed value
-     */
-    R transform(T value);
-
-}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/ConsoleProxyPartitionedMultiWriter.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/ConsoleProxyPartitionedMultiWriter.java
new file mode 100644
index 0000000..700ba0c
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/ConsoleProxyPartitionedMultiWriter.java
@@ -0,0 +1,121 @@
+/**
+ * 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.messaging;
+
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.Future;
+import com.twitter.util.FutureEventListener;
+import jline.ConsoleReader;
+import jline.Terminal;
+import org.apache.commons.lang.StringUtils;
+
+import java.nio.ByteBuffer;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Writer write records from console
+ */
+public class ConsoleProxyPartitionedMultiWriter {
+
+    private final static String HELP = "ConsoleProxyPartitionedMultiWriter <finagle-name> <stream-1>[,<stream-2>,...,<stream-n>]";
+    private final static String PROMPT_MESSAGE = "[dlog] > ";
+
+    static class PartitionedWriter<KEY, VALUE> {
+
+        private final String[] streams;
+        private final Partitioner<KEY> partitioner;
+        private final DistributedLogClient client;
+
+        PartitionedWriter(String[] streams,
+                          Partitioner<KEY> partitioner,
+                          DistributedLogClient client) {
+            this.streams = streams;
+            this.partitioner = partitioner;
+            this.client = client;
+        }
+
+        Future<DLSN> write(KEY key, VALUE value) {
+            int pid = partitioner.partition(key, streams.length);
+            return client.write(streams[pid], ByteBuffer.wrap(value.toString().getBytes(UTF_8)));
+        }
+
+    }
+
+    public static void main(String[] args) throws Exception {
+        if (2 != args.length) {
+            System.out.println(HELP);
+            return;
+        }
+
+        String finagleNameStr = args[0];
+        final String streamList = args[1];
+
+        DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
+                .clientId(ClientId$.MODULE$.apply("console-proxy-writer"))
+                .name("console-proxy-writer")
+                .thriftmux(true)
+                .finagleNameStr(finagleNameStr)
+                .build();
+        String[] streamNameList = StringUtils.split(streamList, ',');
+        PartitionedWriter<Integer, String> partitionedWriter =
+                new PartitionedWriter<Integer, String>(
+                        streamNameList,
+                        new IntPartitioner(),
+                        client);
+
+        ConsoleReader reader = new ConsoleReader();
+        String line;
+        while ((line = reader.readLine(PROMPT_MESSAGE)) != null) {
+            String[] parts = StringUtils.split(line, ':');
+            if (parts.length != 2) {
+                System.out.println("Invalid input. Needs 'KEY:VALUE'");
+                continue;
+            }
+            int key;
+            try {
+                key = Integer.parseInt(parts[0]);
+            } catch (NumberFormatException nfe) {
+                System.out.println("Invalid input. Needs 'KEY:VALUE'");
+                continue;
+            }
+            String value = parts[1];
+
+            partitionedWriter.write(key, value)
+                    .addEventListener(new FutureEventListener<DLSN>() {
+                        @Override
+                        public void onFailure(Throwable cause) {
+                            System.out.println("Encountered error on writing data");
+                            cause.printStackTrace(System.err);
+                            Runtime.getRuntime().exit(0);
+                        }
+
+                        @Override
+                        public void onSuccess(DLSN value) {
+                            // done
+                        }
+                    });
+        }
+
+        client.close();
+    }
+
+}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/ConsoleProxyRRMultiWriter.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/ConsoleProxyRRMultiWriter.java
new file mode 100644
index 0000000..23e94d8
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/ConsoleProxyRRMultiWriter.java
@@ -0,0 +1,77 @@
+/**
+ * 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.messaging;
+
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.service.DistributedLogClient;
+import org.apache.distributedlog.service.DistributedLogClientBuilder;
+import com.twitter.finagle.thrift.ClientId$;
+import com.twitter.util.FutureEventListener;
+import jline.ConsoleReader;
+import jline.Terminal;
+import org.apache.commons.lang.StringUtils;
+
+/**
+ * Writer write records into multiple streams
+ */
+public class ConsoleProxyRRMultiWriter {
+
+    private final static String HELP = "ConsoleProxyRRMultiWriter <finagle-name> <stream-1>[,<stream-2>,...,<stream-n>]";
+    private final static String PROMPT_MESSAGE = "[dlog] > ";
+
+    public static void main(String[] args) throws Exception {
+        if (2 != args.length) {
+            System.out.println(HELP);
+            return;
+        }
+
+        String finagleNameStr = args[0];
+        final String streamList = args[1];
+
+        DistributedLogClient client = DistributedLogClientBuilder.newBuilder()
+                .clientId(ClientId$.MODULE$.apply("console-proxy-writer"))
+                .name("console-proxy-writer")
+                .thriftmux(true)
+                .finagleNameStr(finagleNameStr)
+                .build();
+        String[] streamNameList = StringUtils.split(streamList, ',');
+        RRMultiWriter<Integer, String> writer = new RRMultiWriter(streamNameList, client);
+
+        ConsoleReader reader = new ConsoleReader();
+        String line;
+        while ((line = reader.readLine(PROMPT_MESSAGE)) != null) {
+            writer.write(line)
+                    .addEventListener(new FutureEventListener<DLSN>() {
+                        @Override
+                        public void onFailure(Throwable cause) {
+                            System.out.println("Encountered error on writing data");
+                            cause.printStackTrace(System.err);
+                            Runtime.getRuntime().exit(0);
+                        }
+
+                        @Override
+                        public void onSuccess(DLSN value) {
+                            // done
+                        }
+                    });
+        }
+
+        client.close();
+    }
+
+}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/IdenticalTransformer.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/IdenticalTransformer.java
new file mode 100644
index 0000000..72ef93a
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/IdenticalTransformer.java
@@ -0,0 +1,25 @@
+/**
+ * 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.messaging;
+
+public class IdenticalTransformer<T> implements Transformer<T, T> {
+    @Override
+    public T transform(T value) {
+        return value;
+    }
+}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/IntPartitioner.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/IntPartitioner.java
new file mode 100644
index 0000000..f911442
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/IntPartitioner.java
@@ -0,0 +1,28 @@
+/**
+ * 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.messaging;
+
+/**
+ * Partitioner where key is an integer
+ */
+public class IntPartitioner implements Partitioner<Integer> {
+    @Override
+    public int partition(Integer key, int totalPartitions) {
+        return key % totalPartitions;
+    }
+}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/PartitionedMultiWriter.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/PartitionedMultiWriter.java
new file mode 100644
index 0000000..cf1fac7
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/PartitionedMultiWriter.java
@@ -0,0 +1,50 @@
+/**
+ * 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.messaging;
+
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.service.DistributedLogClient;
+import com.twitter.util.Future;
+
+import java.nio.ByteBuffer;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Partitioned Writer
+ */
+public class PartitionedMultiWriter<KEY, VALUE> {
+
+    private final String[] streams;
+    private final Partitioner<KEY> partitioner;
+    private final DistributedLogClient client;
+
+    public PartitionedMultiWriter(String[] streams,
+                                  Partitioner<KEY> partitioner,
+                                  DistributedLogClient client) {
+        this.streams = streams;
+        this.partitioner = partitioner;
+        this.client = client;
+    }
+
+    public Future<DLSN> write(KEY key, VALUE value) {
+        int pid = partitioner.partition(key, streams.length);
+        return client.write(streams[pid], ByteBuffer.wrap(value.toString().getBytes(UTF_8)));
+    }
+
+}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/Partitioner.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/Partitioner.java
new file mode 100644
index 0000000..e8193c4
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/Partitioner.java
@@ -0,0 +1,25 @@
+/**
+ * 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.messaging;
+
+/**
+ * Partitioner
+ */
+public interface Partitioner<KEY> {
+    int partition(KEY key, int totalPartitions);
+}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/RRMultiWriter.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/RRMultiWriter.java
new file mode 100644
index 0000000..594c9df
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/RRMultiWriter.java
@@ -0,0 +1,94 @@
+/**
+ * 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.messaging;
+
+import com.google.common.collect.Sets;
+import org.apache.distributedlog.DLSN;
+import org.apache.distributedlog.service.DistributedLogClient;
+import com.twitter.finagle.NoBrokersAvailableException;
+import com.twitter.finagle.Service;
+import com.twitter.finagle.ServiceFactory;
+import com.twitter.finagle.loadbalancer.Balancers;
+import com.twitter.finagle.service.SingletonFactory;
+import com.twitter.finagle.stats.NullStatsReceiver;
+import com.twitter.util.Activity;
+import com.twitter.util.Future;
+import scala.collection.JavaConversions;
+
+import java.nio.ByteBuffer;
+import java.util.Set;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Multi stream writer that leverages finagle load balancer.
+ */
+public class RRMultiWriter<KEY, VALUE> {
+
+    static class StreamWriter<VALUE> extends Service<VALUE, DLSN> {
+
+        private final String stream;
+        private final DistributedLogClient client;
+
+        StreamWriter(String stream,
+                     DistributedLogClient client) {
+            this.stream = stream;
+            this.client = client;
+        }
+
+        @Override
+        public Future<DLSN> apply(VALUE request) {
+            return client.write(stream, ByteBuffer.wrap(request.toString().getBytes(UTF_8)));
+        }
+    }
+
+
+    static <VALUE> Set<ServiceFactory<VALUE, DLSN>> initializeServices(
+            String[] streams, DistributedLogClient client) {
+        Set<ServiceFactory<VALUE, DLSN>> serviceFactories =
+                Sets.newHashSet();
+        for (String stream : streams) {
+            Service<VALUE, DLSN> service = new StreamWriter(stream, client);
+            serviceFactories.add(new SingletonFactory<VALUE, DLSN>(service));
+        }
+        return serviceFactories;
+    }
+
+    private final String[] streams;
+    private final DistributedLogClient client;
+    private final Service<VALUE, DLSN> service;
+
+    public RRMultiWriter(String[] streams,
+                         DistributedLogClient client) {
+        this.streams = streams;
+        this.client = client;
+        scala.collection.immutable.Set<ServiceFactory<VALUE, DLSN>> scalaSet =
+                JavaConversions.asScalaSet(initializeServices(streams, client)).toSet();
+        this.service = Balancers.heap(new scala.util.Random(System.currentTimeMillis()))
+                .newBalancer(
+                        Activity.value(scalaSet),
+                        NullStatsReceiver.get(),
+                        new NoBrokersAvailableException("No partitions available")
+                ).toService();
+    }
+
+    public Future<DLSN> write(VALUE data) {
+        return service.apply(data);
+    }
+
+}
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
new file mode 100644
index 0000000..ecf18fc
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/ReaderWithOffsets.java
@@ -0,0 +1,132 @@
+/**
+ * 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.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 java.io.File;
+import java.net.URI;
+import java.util.concurrent.*;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static com.google.common.base.Charsets.UTF_8;
+import static org.iq80.leveldb.impl.Iq80DBFactory.*;
+
+/**
+ * Reader with offsets
+ */
+public class ReaderWithOffsets {
+
+    private final static String HELP = "ReaderWithOffsets <uri> <string> <reader-id> <offset-store-file>";
+
+    public static void main(String[] args) throws Exception {
+        if (4 != args.length) {
+            System.out.println(HELP);
+            return;
+        }
+
+        String dlUriStr = args[0];
+        final String streamName = args[1];
+        final String readerId = args[2];
+        final String offsetStoreFile = args[3];
+
+        URI uri = URI.create(dlUriStr);
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build();
+
+        // open the dlm
+        System.out.println("Opening log stream " + streamName);
+        DistributedLogManager dlm = namespace.openLog(streamName);
+
+        // open the offset store
+        Options options = new Options();
+        options.createIfMissing(true);
+        final DB offsetDB = factory.open(new File(offsetStoreFile), options);
+        final AtomicReference<DLSN> lastDLSN = new AtomicReference<DLSN>(null);
+        // offset updater
+        final ScheduledExecutorService executorService =
+                Executors.newSingleThreadScheduledExecutor();
+        executorService.scheduleAtFixedRate(new Runnable() {
+            @Override
+            public void run() {
+                if (null != lastDLSN.get()) {
+                    offsetDB.put(readerId.getBytes(UTF_8), lastDLSN.get().serializeBytes());
+                    System.out.println("Updated reader " + readerId + " offset to " + lastDLSN.get());
+                }
+            }
+        }, 10, 10, TimeUnit.SECONDS);
+        try {
+            byte[] offset = offsetDB.get(readerId.getBytes(UTF_8));
+            DLSN dlsn;
+            if (null == offset) {
+                dlsn = DLSN.InitialDLSN;
+            } else {
+                dlsn = DLSN.deserializeBytes(offset);
+            }
+            readLoop(dlm, dlsn, lastDLSN);
+        } finally {
+            offsetDB.close();
+            dlm.close();
+            namespace.close();
+        }
+    }
+
+    private static void readLoop(final DistributedLogManager dlm,
+                                 final DLSN dlsn,
+                                 final AtomicReference<DLSN> lastDLSN)
+            throws Exception {
+
+        final CountDownLatch keepAliveLatch = new CountDownLatch(1);
+
+        System.out.println("Wait for records starting from " + dlsn);
+        final AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(dlsn));
+        final FutureEventListener<LogRecordWithDLSN> readListener = new FutureEventListener<LogRecordWithDLSN>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                System.err.println("Encountered error on reading records from stream " + dlm.getStreamName());
+                cause.printStackTrace(System.err);
+                keepAliveLatch.countDown();
+            }
+
+            @Override
+            public void onSuccess(LogRecordWithDLSN record) {
+                System.out.println("Received record " + record.getDlsn());
+                System.out.println("\"\"\"");
+                System.out.println(new String(record.getPayload(), UTF_8));
+                System.out.println("\"\"\"");
+                lastDLSN.set(record.getDlsn());
+                reader.readNext().addEventListener(this);
+            }
+        };
+        reader.readNext().addEventListener(readListener);
+
+        keepAliveLatch.await();
+        FutureUtils.result(reader.asyncClose(), Duration.apply(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
new file mode 100644
index 0000000..2cf202f
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/StreamTransformer.java
@@ -0,0 +1,188 @@
+/**
+ * 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.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 java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.net.URI;
+import java.nio.ByteBuffer;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import static com.google.common.base.Charsets.UTF_8;
+
+/**
+ * Transform one stream to another stream. And apply transformation
+ */
+public class StreamTransformer {
+
+    private final static String HELP = "StreamTransformer <uri> <src_stream> <target_stream>";
+    private final static TProtocolFactory protocolFactory =
+            new TBinaryProtocol.Factory();
+
+    public static void main(String[] args) throws Exception {
+        if (3 != args.length) {
+            System.out.println(HELP);
+            return;
+        }
+
+        String dlUriStr = args[0];
+        final String srcStreamName = args[1];
+        final String targetStreamName = args[2];
+
+        URI uri = URI.create(dlUriStr);
+        DistributedLogConfiguration conf = new DistributedLogConfiguration();
+        conf.setOutputBufferSize(16*1024); // 16KB
+        conf.setPeriodicFlushFrequencyMilliSeconds(5); // 5ms
+        DistributedLogNamespace namespace = DistributedLogNamespaceBuilder.newBuilder()
+                .conf(conf)
+                .uri(uri)
+                .build();
+
+        // open the dlm
+        System.out.println("Opening log stream " + srcStreamName);
+        DistributedLogManager srcDlm = namespace.openLog(srcStreamName);
+        System.out.println("Opening log stream " + targetStreamName);
+        DistributedLogManager targetDlm = namespace.openLog(targetStreamName);
+
+        Transformer<byte[], byte[]> replicationTransformer =
+                new IdenticalTransformer<byte[]>();
+
+        LogRecordWithDLSN lastTargetRecord;
+        DLSN srcDlsn;
+        try {
+            lastTargetRecord = targetDlm.getLastLogRecord();
+            TransformedRecord lastTransformedRecord = new TransformedRecord();
+            try {
+                lastTransformedRecord.read(protocolFactory.getProtocol(
+                        new TIOStreamTransport(new ByteArrayInputStream(lastTargetRecord.getPayload()))));
+                srcDlsn = DLSN.deserializeBytes(lastTransformedRecord.getSrcDlsn());
+                System.out.println("Last transformed record is " + srcDlsn);
+            } catch (TException e) {
+                System.err.println("Error on reading last transformed record");
+                e.printStackTrace(System.err);
+                srcDlsn = DLSN.InitialDLSN;
+            }
+        } catch (LogNotFoundException lnfe) {
+            srcDlsn = DLSN.InitialDLSN;
+        } catch (LogEmptyException lee) {
+            srcDlsn = DLSN.InitialDLSN;
+        }
+
+        AsyncLogWriter targetWriter = FutureUtils.result(targetDlm.openAsyncLogWriter());
+        try {
+            readLoop(srcDlm, srcDlsn, targetWriter, replicationTransformer);
+        } finally {
+            FutureUtils.result(targetWriter.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
+            targetDlm.close();
+            srcDlm.close();
+            namespace.close();
+        }
+
+    }
+
+    private static void readLoop(final DistributedLogManager dlm,
+                                 final DLSN fromDLSN,
+                                 final AsyncLogWriter targetWriter,
+                                 final Transformer<byte[], byte[]> replicationTransformer)
+            throws Exception {
+
+        final CountDownLatch keepAliveLatch = new CountDownLatch(1);
+
+        System.out.println("Wait for records starting from " + fromDLSN);
+        final AsyncLogReader reader = FutureUtils.result(dlm.openAsyncLogReader(fromDLSN));
+        final FutureEventListener<LogRecordWithDLSN> readListener = new FutureEventListener<LogRecordWithDLSN>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                System.err.println("Encountered error on reading records from stream " + dlm.getStreamName());
+                cause.printStackTrace(System.err);
+                keepAliveLatch.countDown();
+            }
+
+            @Override
+            public void onSuccess(LogRecordWithDLSN record) {
+                if (record.getDlsn().compareTo(fromDLSN) <= 0) {
+                    reader.readNext().addEventListener(this);
+                    return;
+                }
+                System.out.println("Received record " + record.getDlsn());
+                System.out.println("\"\"\"");
+                System.out.println(new String(record.getPayload(), UTF_8));
+                System.out.println("\"\"\"");
+                try {
+                    transform(targetWriter, record, replicationTransformer, keepAliveLatch);
+                } catch (Exception e) {
+                    System.err.println("Encountered error on transforming record " + record.getDlsn()
+                            + " from stream " + dlm.getStreamName());
+                    e.printStackTrace(System.err);
+                    keepAliveLatch.countDown();
+                }
+                reader.readNext().addEventListener(this);
+            }
+        };
+        reader.readNext().addEventListener(readListener);
+
+        keepAliveLatch.await();
+        FutureUtils.result(reader.asyncClose(), Duration.apply(5, TimeUnit.SECONDS));
+    }
+
+    private static void transform(final AsyncLogWriter writer,
+                                  LogRecordWithDLSN record,
+                                  Transformer<byte[], byte[]> replicationTransformer,
+                                  final CountDownLatch keepAliveLatch)
+            throws Exception {
+        DLSN srcDLSN = record.getDlsn();
+        byte[] payload = record.getPayload();
+        byte[] transformedPayload = replicationTransformer.transform(payload);
+        TransformedRecord transformedRecord =
+                new TransformedRecord(ByteBuffer.wrap(transformedPayload));
+        transformedRecord.setSrcDlsn(srcDLSN.serializeBytes());
+        ByteArrayOutputStream baos = new ByteArrayOutputStream(4096);
+        transformedRecord.write(protocolFactory.getProtocol(new TIOStreamTransport(baos)));
+        byte[] data = baos.toByteArray();
+        writer.write(new LogRecord(record.getSequenceId(), data))
+                .addEventListener(new FutureEventListener<DLSN>() {
+            @Override
+            public void onFailure(Throwable cause) {
+                System.err.println("Encountered error on writing records to stream " + writer.getStreamName());
+                cause.printStackTrace(System.err);
+                keepAliveLatch.countDown();
+            }
+
+            @Override
+            public void onSuccess(DLSN dlsn) {
+                System.out.println("Write transformed record " + dlsn);
+            }
+        });
+    }
+
+}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/Transformer.java b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/Transformer.java
new file mode 100644
index 0000000..df7cc99
--- /dev/null
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/java/org/apache/distributedlog/messaging/Transformer.java
@@ -0,0 +1,34 @@
+/**
+ * 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.messaging;
+
+/**
+ * Transform one format to the other format.
+ */
+public interface Transformer<T, R> {
+
+    /**
+     * Transform value <i>T</i> to value <i>R</i>. If null is returned,
+     * the value <i>T</i> will be eliminated without transforming.
+     *
+     * @param value value to transform
+     * @return transformed value
+     */
+    R transform(T value);
+
+}
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/resources/findbugsExclude.xml b/distributedlog-tutorials/distributedlog-messaging/src/main/resources/findbugsExclude.xml
index 9cb253a..4b86949 100644
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/resources/findbugsExclude.xml
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/resources/findbugsExclude.xml
@@ -22,12 +22,12 @@
   </Match>
   <Match>
     <!-- it is safe to store external bytes reference here. //-->
-    <Class name="com.twitter.distributedlog.messaging.PartitionedMultiWriter" />
+    <Class name="org.apache.distributedlog.messaging.PartitionedMultiWriter" />
     <Bug pattern="EI_EXPOSE_REP2" />
   </Match>
   <Match>
     <!-- it is safe to store external bytes reference here. //-->
-    <Class name="com.twitter.distributedlog.messaging.RRMultiWriter" />
+    <Class name="org.apache.distributedlog.messaging.RRMultiWriter" />
     <Bug pattern="EI_EXPOSE_REP2" />
   </Match>
 </FindBugsFilter>
diff --git a/distributedlog-tutorials/distributedlog-messaging/src/main/thrift/messaging.thrift b/distributedlog-tutorials/distributedlog-messaging/src/main/thrift/messaging.thrift
index 03c3792..a3634fe 100644
--- a/distributedlog-tutorials/distributedlog-messaging/src/main/thrift/messaging.thrift
+++ b/distributedlog-tutorials/distributedlog-messaging/src/main/thrift/messaging.thrift
@@ -17,7 +17,7 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
-namespace java com.twitter.distributedlog.thrift.messaging
+namespace java org.apache.distributedlog.thrift.messaging
 
 struct TransformedRecord {
     1: required binary payload
diff --git a/docs/_plugins/jekyll-rst/.gitignore b/docs/_plugins/jekyll-rst/.gitignore
index 7e99e36..0d20b64 100644
--- a/docs/_plugins/jekyll-rst/.gitignore
+++ b/docs/_plugins/jekyll-rst/.gitignore
@@ -1 +1 @@
-*.pyc
\ No newline at end of file
+*.pyc
diff --git a/docs/_plugins/jekyll-rst/LICENSE.txt b/docs/_plugins/jekyll-rst/LICENSE.txt
index 988ac9e..e70d9d9 100644
--- a/docs/_plugins/jekyll-rst/LICENSE.txt
+++ b/docs/_plugins/jekyll-rst/LICENSE.txt
@@ -17,4 +17,4 @@
 AUTHORS OR COPYRIGHT HOLDERS BE LIABLE FOR ANY CLAIM, DAMAGES OR OTHER
 LIABILITY, WHETHER IN AN ACTION OF CONTRACT, TORT OR OTHERWISE, ARISING FROM,
 OUT OF OR IN CONNECTION WITH THE SOFTWARE OR THE USE OR OTHER DEALINGS IN
-THE SOFTWARE.
\ No newline at end of file
+THE SOFTWARE.
diff --git a/docs/_plugins/jekyll-rst/README.rst b/docs/_plugins/jekyll-rst/README.rst
index 48c21f4..3c0b01a 100644
--- a/docs/_plugins/jekyll-rst/README.rst
+++ b/docs/_plugins/jekyll-rst/README.rst
@@ -94,4 +94,4 @@
 .. _Octopress: http://octopress.com/
 .. _RbST: http://rubygems.org/gems/RbST
 .. _bundler: http://gembundler.com/
-.. _Harry Marr's advice: http://hmarr.com/2010/jan/19/making-virtualenv-play-nice-with-git/
\ No newline at end of file
+.. _Harry Marr's advice: http://hmarr.com/2010/jan/19/making-virtualenv-play-nice-with-git/
diff --git a/docs/_plugins/jekyll-rst/converter.rb b/docs/_plugins/jekyll-rst/converter.rb
index 64b639c..31c6960 100644
--- a/docs/_plugins/jekyll-rst/converter.rb
+++ b/docs/_plugins/jekyll-rst/converter.rb
@@ -27,4 +27,4 @@
       converter.convert(input)
     end
   end
-end  
\ No newline at end of file
+end  
diff --git a/docs/_plugins/jekyll-rst/directives.py b/docs/_plugins/jekyll-rst/directives.py
index ac59d06..8feae57 100644
--- a/docs/_plugins/jekyll-rst/directives.py
+++ b/docs/_plugins/jekyll-rst/directives.py
@@ -94,4 +94,4 @@
         return [nodes.raw('', code, format='html')]
 
 directives.register_directive('code-block', Pygments)
-directives.register_directive('sourcecode', Pygments)
\ No newline at end of file
+directives.register_directive('sourcecode', Pygments)
diff --git a/docs/_plugins/jekyll-rst/transform.py b/docs/_plugins/jekyll-rst/transform.py
index f34723f..2904a04 100644
--- a/docs/_plugins/jekyll-rst/transform.py
+++ b/docs/_plugins/jekyll-rst/transform.py
@@ -37,4 +37,4 @@
     
     if opts.part in parts:
         return parts[opts.part]
-    return ''
\ No newline at end of file
+    return ''
diff --git a/docs/admin_guide/operations.rst b/docs/admin_guide/operations.rst
index cbd30fe..e56c596 100644
--- a/docs/admin_guide/operations.rst
+++ b/docs/admin_guide/operations.rst
@@ -93,7 +93,7 @@
 
 ::
 
-    featureProviderClass=com.twitter.distributedlog.feature.DynamicConfigurationFeatureProvider
+    featureProviderClass=org.apache.distributedlog.feature.DynamicConfigurationFeatureProvider
 
 
 
@@ -112,7 +112,7 @@
 
 
 You could configure `featureProviderClass` in distributedlog configuration file by setting it to
-`com.twitter.distributedlog.feature.DynamicConfigurationFeatureProvider` to enable file-based feature
+`org.apache.distributedlog.feature.DynamicConfigurationFeatureProvider` to enable file-based feature
 provider. The feature provider will load the features from two files, one is base config file configured
 by `fileFeatureProviderBaseConfigPath`, while the other one is overlay config file configured by
 `fileFeatureProviderOverlayConfigPath`. Current implementation doesn't differentiate these two files
@@ -122,7 +122,7 @@
 
 ::
 
-    featureProviderClass=com.twitter.distributedlog.feature.DynamicConfigurationFeatureProvider
+    featureProviderClass=org.apache.distributedlog.feature.DynamicConfigurationFeatureProvider
     fileFeatureProviderBaseConfigPath=/path/to/base/config
     fileFeatureProviderOverlayConfigPath=/path/to/overlay/config
     // how frequent we reload the config files
diff --git a/docs/deployment/cluster.rst b/docs/deployment/cluster.rst
index 250ba3c..b4caf3f 100644
--- a/docs/deployment/cluster.rst
+++ b/docs/deployment/cluster.rst
@@ -509,15 +509,15 @@
 
 ::
 
-    $ ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.MultiReader distributedlog://127.0.0.1:2181/messaging/distributedlog/mynamespace stream-0,stream-1,stream-2,stream-3,stream-4,stream-5,stream-6,stream-7,stream-8,stream-9,stream-10
+    $ ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.MultiReader distributedlog://127.0.0.1:2181/messaging/distributedlog/mynamespace stream-0,stream-1,stream-2,stream-3,stream-4,stream-5,stream-6,stream-7,stream-8,stream-9,stream-10
 
 
 Run record generator over some streams
 
 ::
 
-    $ ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.RecordGenerator 'zk!127.0.0.1:2181!/messaging/distributedlog/mynamespace/.write_proxy' stream-0 100
-    $ ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.RecordGenerator 'zk!127.0.0.1:2181!/messaging/distributedlog/mynamespace/.write_proxy' stream-1 100
+    $ ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.RecordGenerator 'zk!127.0.0.1:2181!/messaging/distributedlog/mynamespace/.write_proxy' stream-0 100
+    $ ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.RecordGenerator 'zk!127.0.0.1:2181!/messaging/distributedlog/mynamespace/.write_proxy' stream-1 100
 
 
 Check the terminal running `MultiReader`. You will see similar output as below:
diff --git a/docs/fonts/bootstrap/glyphicons-halflings-regular.svg b/docs/fonts/bootstrap/glyphicons-halflings-regular.svg
index 94fb549..ff1de8e 100755
--- a/docs/fonts/bootstrap/glyphicons-halflings-regular.svg
+++ b/docs/fonts/bootstrap/glyphicons-halflings-regular.svg
@@ -285,4 +285,4 @@
 <glyph unicode="&#x1f511;" d="M250 1200h600q21 0 35.5 -14.5t14.5 -35.5v-400q0 -21 -14.5 -35.5t-35.5 -14.5h-150v-500l-255 -178q-19 -9 -32 -1t-13 29v650h-150q-21 0 -35.5 14.5t-14.5 35.5v400q0 21 14.5 35.5t35.5 14.5zM400 1100v-100h300v100h-300z" />
 <glyph unicode="&#x1f6aa;" d="M250 1200h750q39 0 69.5 -40.5t30.5 -84.5v-933l-700 -117v950l600 125h-700v-1000h-100v1025q0 23 15.5 49t34.5 26zM500 525v-100l100 20v100z" />
 </font>
-</defs></svg> 
\ No newline at end of file
+</defs></svg> 
diff --git a/docs/js/bootstrap.min.js b/docs/js/bootstrap.min.js
index e79c065..e364a13 100755
--- a/docs/js/bootstrap.min.js
+++ b/docs/js/bootstrap.min.js
@@ -4,4 +4,4 @@
  * Licensed under the MIT license
  */
 if("undefined"==typeof jQuery)throw new Error("Bootstrap's JavaScript requires jQuery");+function(a){"use strict";var b=a.fn.jquery.split(" ")[0].split(".");if(b[0]<2&&b[1]<9||1==b[0]&&9==b[1]&&b[2]<1||b[0]>2)throw new Error("Bootstrap's JavaScript requires jQuery version 1.9.1 or higher, but lower than version 3")}(jQuery),+function(a){"use strict";function b(){var a=document.createElement("bootstrap"),b={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"};for(var c in b)if(void 0!==a.style[c])return{end:b[c]};return!1}a.fn.emulateTransitionEnd=function(b){var c=!1,d=this;a(this).one("bsTransitionEnd",function(){c=!0});var e=function(){c||a(d).trigger(a.support.transition.end)};return setTimeout(e,b),this},a(function(){a.support.transition=b(),a.support.transition&&(a.event.special.bsTransitionEnd={bindType:a.support.transition.end,delegateType:a.support.transition.end,handle:function(b){return a(b.target).is(this)?b.handleObj.handler.apply(this,arguments):void 0}})})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var c=a(this),e=c.data("bs.alert");e||c.data("bs.alert",e=new d(this)),"string"==typeof b&&e[b].call(c)})}var c='[data-dismiss="alert"]',d=function(b){a(b).on("click",c,this.close)};d.VERSION="3.3.6",d.TRANSITION_DURATION=150,d.prototype.close=function(b){function c(){g.detach().trigger("closed.bs.alert").remove()}var e=a(this),f=e.attr("data-target");f||(f=e.attr("href"),f=f&&f.replace(/.*(?=#[^\s]*$)/,""));var g=a(f);b&&b.preventDefault(),g.length||(g=e.closest(".alert")),g.trigger(b=a.Event("close.bs.alert")),b.isDefaultPrevented()||(g.removeClass("in"),a.support.transition&&g.hasClass("fade")?g.one("bsTransitionEnd",c).emulateTransitionEnd(d.TRANSITION_DURATION):c())};var e=a.fn.alert;a.fn.alert=b,a.fn.alert.Constructor=d,a.fn.alert.noConflict=function(){return a.fn.alert=e,this},a(document).on("click.bs.alert.data-api",c,d.prototype.close)}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.button"),f="object"==typeof b&&b;e||d.data("bs.button",e=new c(this,f)),"toggle"==b?e.toggle():b&&e.setState(b)})}var c=function(b,d){this.$element=a(b),this.options=a.extend({},c.DEFAULTS,d),this.isLoading=!1};c.VERSION="3.3.6",c.DEFAULTS={loadingText:"loading..."},c.prototype.setState=function(b){var c="disabled",d=this.$element,e=d.is("input")?"val":"html",f=d.data();b+="Text",null==f.resetText&&d.data("resetText",d[e]()),setTimeout(a.proxy(function(){d[e](null==f[b]?this.options[b]:f[b]),"loadingText"==b?(this.isLoading=!0,d.addClass(c).attr(c,c)):this.isLoading&&(this.isLoading=!1,d.removeClass(c).removeAttr(c))},this),0)},c.prototype.toggle=function(){var a=!0,b=this.$element.closest('[data-toggle="buttons"]');if(b.length){var c=this.$element.find("input");"radio"==c.prop("type")?(c.prop("checked")&&(a=!1),b.find(".active").removeClass("active"),this.$element.addClass("active")):"checkbox"==c.prop("type")&&(c.prop("checked")!==this.$element.hasClass("active")&&(a=!1),this.$element.toggleClass("active")),c.prop("checked",this.$element.hasClass("active")),a&&c.trigger("change")}else this.$element.attr("aria-pressed",!this.$element.hasClass("active")),this.$element.toggleClass("active")};var d=a.fn.button;a.fn.button=b,a.fn.button.Constructor=c,a.fn.button.noConflict=function(){return a.fn.button=d,this},a(document).on("click.bs.button.data-api",'[data-toggle^="button"]',function(c){var d=a(c.target);d.hasClass("btn")||(d=d.closest(".btn")),b.call(d,"toggle"),a(c.target).is('input[type="radio"]')||a(c.target).is('input[type="checkbox"]')||c.preventDefault()}).on("focus.bs.button.data-api blur.bs.button.data-api",'[data-toggle^="button"]',function(b){a(b.target).closest(".btn").toggleClass("focus",/^focus(in)?$/.test(b.type))})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.carousel"),f=a.extend({},c.DEFAULTS,d.data(),"object"==typeof b&&b),g="string"==typeof b?b:f.slide;e||d.data("bs.carousel",e=new c(this,f)),"number"==typeof b?e.to(b):g?e[g]():f.interval&&e.pause().cycle()})}var c=function(b,c){this.$element=a(b),this.$indicators=this.$element.find(".carousel-indicators"),this.options=c,this.paused=null,this.sliding=null,this.interval=null,this.$active=null,this.$items=null,this.options.keyboard&&this.$element.on("keydown.bs.carousel",a.proxy(this.keydown,this)),"hover"==this.options.pause&&!("ontouchstart"in document.documentElement)&&this.$element.on("mouseenter.bs.carousel",a.proxy(this.pause,this)).on("mouseleave.bs.carousel",a.proxy(this.cycle,this))};c.VERSION="3.3.6",c.TRANSITION_DURATION=600,c.DEFAULTS={interval:5e3,pause:"hover",wrap:!0,keyboard:!0},c.prototype.keydown=function(a){if(!/input|textarea/i.test(a.target.tagName)){switch(a.which){case 37:this.prev();break;case 39:this.next();break;default:return}a.preventDefault()}},c.prototype.cycle=function(b){return b||(this.paused=!1),this.interval&&clearInterval(this.interval),this.options.interval&&!this.paused&&(this.interval=setInterval(a.proxy(this.next,this),this.options.interval)),this},c.prototype.getItemIndex=function(a){return this.$items=a.parent().children(".item"),this.$items.index(a||this.$active)},c.prototype.getItemForDirection=function(a,b){var c=this.getItemIndex(b),d="prev"==a&&0===c||"next"==a&&c==this.$items.length-1;if(d&&!this.options.wrap)return b;var e="prev"==a?-1:1,f=(c+e)%this.$items.length;return this.$items.eq(f)},c.prototype.to=function(a){var b=this,c=this.getItemIndex(this.$active=this.$element.find(".item.active"));return a>this.$items.length-1||0>a?void 0:this.sliding?this.$element.one("slid.bs.carousel",function(){b.to(a)}):c==a?this.pause().cycle():this.slide(a>c?"next":"prev",this.$items.eq(a))},c.prototype.pause=function(b){return b||(this.paused=!0),this.$element.find(".next, .prev").length&&a.support.transition&&(this.$element.trigger(a.support.transition.end),this.cycle(!0)),this.interval=clearInterval(this.interval),this},c.prototype.next=function(){return this.sliding?void 0:this.slide("next")},c.prototype.prev=function(){return this.sliding?void 0:this.slide("prev")},c.prototype.slide=function(b,d){var e=this.$element.find(".item.active"),f=d||this.getItemForDirection(b,e),g=this.interval,h="next"==b?"left":"right",i=this;if(f.hasClass("active"))return this.sliding=!1;var j=f[0],k=a.Event("slide.bs.carousel",{relatedTarget:j,direction:h});if(this.$element.trigger(k),!k.isDefaultPrevented()){if(this.sliding=!0,g&&this.pause(),this.$indicators.length){this.$indicators.find(".active").removeClass("active");var l=a(this.$indicators.children()[this.getItemIndex(f)]);l&&l.addClass("active")}var m=a.Event("slid.bs.carousel",{relatedTarget:j,direction:h});return a.support.transition&&this.$element.hasClass("slide")?(f.addClass(b),f[0].offsetWidth,e.addClass(h),f.addClass(h),e.one("bsTransitionEnd",function(){f.removeClass([b,h].join(" ")).addClass("active"),e.removeClass(["active",h].join(" ")),i.sliding=!1,setTimeout(function(){i.$element.trigger(m)},0)}).emulateTransitionEnd(c.TRANSITION_DURATION)):(e.removeClass("active"),f.addClass("active"),this.sliding=!1,this.$element.trigger(m)),g&&this.cycle(),this}};var d=a.fn.carousel;a.fn.carousel=b,a.fn.carousel.Constructor=c,a.fn.carousel.noConflict=function(){return a.fn.carousel=d,this};var e=function(c){var d,e=a(this),f=a(e.attr("data-target")||(d=e.attr("href"))&&d.replace(/.*(?=#[^\s]+$)/,""));if(f.hasClass("carousel")){var g=a.extend({},f.data(),e.data()),h=e.attr("data-slide-to");h&&(g.interval=!1),b.call(f,g),h&&f.data("bs.carousel").to(h),c.preventDefault()}};a(document).on("click.bs.carousel.data-api","[data-slide]",e).on("click.bs.carousel.data-api","[data-slide-to]",e),a(window).on("load",function(){a('[data-ride="carousel"]').each(function(){var c=a(this);b.call(c,c.data())})})}(jQuery),+function(a){"use strict";function b(b){var c,d=b.attr("data-target")||(c=b.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,"");return a(d)}function c(b){return this.each(function(){var c=a(this),e=c.data("bs.collapse"),f=a.extend({},d.DEFAULTS,c.data(),"object"==typeof b&&b);!e&&f.toggle&&/show|hide/.test(b)&&(f.toggle=!1),e||c.data("bs.collapse",e=new d(this,f)),"string"==typeof b&&e[b]()})}var d=function(b,c){this.$element=a(b),this.options=a.extend({},d.DEFAULTS,c),this.$trigger=a('[data-toggle="collapse"][href="#'+b.id+'"],[data-toggle="collapse"][data-target="#'+b.id+'"]'),this.transitioning=null,this.options.parent?this.$parent=this.getParent():this.addAriaAndCollapsedClass(this.$element,this.$trigger),this.options.toggle&&this.toggle()};d.VERSION="3.3.6",d.TRANSITION_DURATION=350,d.DEFAULTS={toggle:!0},d.prototype.dimension=function(){var a=this.$element.hasClass("width");return a?"width":"height"},d.prototype.show=function(){if(!this.transitioning&&!this.$element.hasClass("in")){var b,e=this.$parent&&this.$parent.children(".panel").children(".in, .collapsing");if(!(e&&e.length&&(b=e.data("bs.collapse"),b&&b.transitioning))){var f=a.Event("show.bs.collapse");if(this.$element.trigger(f),!f.isDefaultPrevented()){e&&e.length&&(c.call(e,"hide"),b||e.data("bs.collapse",null));var g=this.dimension();this.$element.removeClass("collapse").addClass("collapsing")[g](0).attr("aria-expanded",!0),this.$trigger.removeClass("collapsed").attr("aria-expanded",!0),this.transitioning=1;var h=function(){this.$element.removeClass("collapsing").addClass("collapse in")[g](""),this.transitioning=0,this.$element.trigger("shown.bs.collapse")};if(!a.support.transition)return h.call(this);var i=a.camelCase(["scroll",g].join("-"));this.$element.one("bsTransitionEnd",a.proxy(h,this)).emulateTransitionEnd(d.TRANSITION_DURATION)[g](this.$element[0][i])}}}},d.prototype.hide=function(){if(!this.transitioning&&this.$element.hasClass("in")){var b=a.Event("hide.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.dimension();this.$element[c](this.$element[c]())[0].offsetHeight,this.$element.addClass("collapsing").removeClass("collapse in").attr("aria-expanded",!1),this.$trigger.addClass("collapsed").attr("aria-expanded",!1),this.transitioning=1;var e=function(){this.transitioning=0,this.$element.removeClass("collapsing").addClass("collapse").trigger("hidden.bs.collapse")};return a.support.transition?void this.$element[c](0).one("bsTransitionEnd",a.proxy(e,this)).emulateTransitionEnd(d.TRANSITION_DURATION):e.call(this)}}},d.prototype.toggle=function(){this[this.$element.hasClass("in")?"hide":"show"]()},d.prototype.getParent=function(){return a(this.options.parent).find('[data-toggle="collapse"][data-parent="'+this.options.parent+'"]').each(a.proxy(function(c,d){var e=a(d);this.addAriaAndCollapsedClass(b(e),e)},this)).end()},d.prototype.addAriaAndCollapsedClass=function(a,b){var c=a.hasClass("in");a.attr("aria-expanded",c),b.toggleClass("collapsed",!c).attr("aria-expanded",c)};var e=a.fn.collapse;a.fn.collapse=c,a.fn.collapse.Constructor=d,a.fn.collapse.noConflict=function(){return a.fn.collapse=e,this},a(document).on("click.bs.collapse.data-api",'[data-toggle="collapse"]',function(d){var e=a(this);e.attr("data-target")||d.preventDefault();var f=b(e),g=f.data("bs.collapse"),h=g?"toggle":e.data();c.call(f,h)})}(jQuery),+function(a){"use strict";function b(b){var c=b.attr("data-target");c||(c=b.attr("href"),c=c&&/#[A-Za-z]/.test(c)&&c.replace(/.*(?=#[^\s]*$)/,""));var d=c&&a(c);return d&&d.length?d:b.parent()}function c(c){c&&3===c.which||(a(e).remove(),a(f).each(function(){var d=a(this),e=b(d),f={relatedTarget:this};e.hasClass("open")&&(c&&"click"==c.type&&/input|textarea/i.test(c.target.tagName)&&a.contains(e[0],c.target)||(e.trigger(c=a.Event("hide.bs.dropdown",f)),c.isDefaultPrevented()||(d.attr("aria-expanded","false"),e.removeClass("open").trigger(a.Event("hidden.bs.dropdown",f)))))}))}function d(b){return this.each(function(){var c=a(this),d=c.data("bs.dropdown");d||c.data("bs.dropdown",d=new g(this)),"string"==typeof b&&d[b].call(c)})}var e=".dropdown-backdrop",f='[data-toggle="dropdown"]',g=function(b){a(b).on("click.bs.dropdown",this.toggle)};g.VERSION="3.3.6",g.prototype.toggle=function(d){var e=a(this);if(!e.is(".disabled, :disabled")){var f=b(e),g=f.hasClass("open");if(c(),!g){"ontouchstart"in document.documentElement&&!f.closest(".navbar-nav").length&&a(document.createElement("div")).addClass("dropdown-backdrop").insertAfter(a(this)).on("click",c);var h={relatedTarget:this};if(f.trigger(d=a.Event("show.bs.dropdown",h)),d.isDefaultPrevented())return;e.trigger("focus").attr("aria-expanded","true"),f.toggleClass("open").trigger(a.Event("shown.bs.dropdown",h))}return!1}},g.prototype.keydown=function(c){if(/(38|40|27|32)/.test(c.which)&&!/input|textarea/i.test(c.target.tagName)){var d=a(this);if(c.preventDefault(),c.stopPropagation(),!d.is(".disabled, :disabled")){var e=b(d),g=e.hasClass("open");if(!g&&27!=c.which||g&&27==c.which)return 27==c.which&&e.find(f).trigger("focus"),d.trigger("click");var h=" li:not(.disabled):visible a",i=e.find(".dropdown-menu"+h);if(i.length){var j=i.index(c.target);38==c.which&&j>0&&j--,40==c.which&&j<i.length-1&&j++,~j||(j=0),i.eq(j).trigger("focus")}}}};var h=a.fn.dropdown;a.fn.dropdown=d,a.fn.dropdown.Constructor=g,a.fn.dropdown.noConflict=function(){return a.fn.dropdown=h,this},a(document).on("click.bs.dropdown.data-api",c).on("click.bs.dropdown.data-api",".dropdown form",function(a){a.stopPropagation()}).on("click.bs.dropdown.data-api",f,g.prototype.toggle).on("keydown.bs.dropdown.data-api",f,g.prototype.keydown).on("keydown.bs.dropdown.data-api",".dropdown-menu",g.prototype.keydown)}(jQuery),+function(a){"use strict";function b(b,d){return this.each(function(){var e=a(this),f=e.data("bs.modal"),g=a.extend({},c.DEFAULTS,e.data(),"object"==typeof b&&b);f||e.data("bs.modal",f=new c(this,g)),"string"==typeof b?f[b](d):g.show&&f.show(d)})}var c=function(b,c){this.options=c,this.$body=a(document.body),this.$element=a(b),this.$dialog=this.$element.find(".modal-dialog"),this.$backdrop=null,this.isShown=null,this.originalBodyPad=null,this.scrollbarWidth=0,this.ignoreBackdropClick=!1,this.options.remote&&this.$element.find(".modal-content").load(this.options.remote,a.proxy(function(){this.$element.trigger("loaded.bs.modal")},this))};c.VERSION="3.3.6",c.TRANSITION_DURATION=300,c.BACKDROP_TRANSITION_DURATION=150,c.DEFAULTS={backdrop:!0,keyboard:!0,show:!0},c.prototype.toggle=function(a){return this.isShown?this.hide():this.show(a)},c.prototype.show=function(b){var d=this,e=a.Event("show.bs.modal",{relatedTarget:b});this.$element.trigger(e),this.isShown||e.isDefaultPrevented()||(this.isShown=!0,this.checkScrollbar(),this.setScrollbar(),this.$body.addClass("modal-open"),this.escape(),this.resize(),this.$element.on("click.dismiss.bs.modal",'[data-dismiss="modal"]',a.proxy(this.hide,this)),this.$dialog.on("mousedown.dismiss.bs.modal",function(){d.$element.one("mouseup.dismiss.bs.modal",function(b){a(b.target).is(d.$element)&&(d.ignoreBackdropClick=!0)})}),this.backdrop(function(){var e=a.support.transition&&d.$element.hasClass("fade");d.$element.parent().length||d.$element.appendTo(d.$body),d.$element.show().scrollTop(0),d.adjustDialog(),e&&d.$element[0].offsetWidth,d.$element.addClass("in"),d.enforceFocus();var f=a.Event("shown.bs.modal",{relatedTarget:b});e?d.$dialog.one("bsTransitionEnd",function(){d.$element.trigger("focus").trigger(f)}).emulateTransitionEnd(c.TRANSITION_DURATION):d.$element.trigger("focus").trigger(f)}))},c.prototype.hide=function(b){b&&b.preventDefault(),b=a.Event("hide.bs.modal"),this.$element.trigger(b),this.isShown&&!b.isDefaultPrevented()&&(this.isShown=!1,this.escape(),this.resize(),a(document).off("focusin.bs.modal"),this.$element.removeClass("in").off("click.dismiss.bs.modal").off("mouseup.dismiss.bs.modal"),this.$dialog.off("mousedown.dismiss.bs.modal"),a.support.transition&&this.$element.hasClass("fade")?this.$element.one("bsTransitionEnd",a.proxy(this.hideModal,this)).emulateTransitionEnd(c.TRANSITION_DURATION):this.hideModal())},c.prototype.enforceFocus=function(){a(document).off("focusin.bs.modal").on("focusin.bs.modal",a.proxy(function(a){this.$element[0]===a.target||this.$element.has(a.target).length||this.$element.trigger("focus")},this))},c.prototype.escape=function(){this.isShown&&this.options.keyboard?this.$element.on("keydown.dismiss.bs.modal",a.proxy(function(a){27==a.which&&this.hide()},this)):this.isShown||this.$element.off("keydown.dismiss.bs.modal")},c.prototype.resize=function(){this.isShown?a(window).on("resize.bs.modal",a.proxy(this.handleUpdate,this)):a(window).off("resize.bs.modal")},c.prototype.hideModal=function(){var a=this;this.$element.hide(),this.backdrop(function(){a.$body.removeClass("modal-open"),a.resetAdjustments(),a.resetScrollbar(),a.$element.trigger("hidden.bs.modal")})},c.prototype.removeBackdrop=function(){this.$backdrop&&this.$backdrop.remove(),this.$backdrop=null},c.prototype.backdrop=function(b){var d=this,e=this.$element.hasClass("fade")?"fade":"";if(this.isShown&&this.options.backdrop){var f=a.support.transition&&e;if(this.$backdrop=a(document.createElement("div")).addClass("modal-backdrop "+e).appendTo(this.$body),this.$element.on("click.dismiss.bs.modal",a.proxy(function(a){return this.ignoreBackdropClick?void(this.ignoreBackdropClick=!1):void(a.target===a.currentTarget&&("static"==this.options.backdrop?this.$element[0].focus():this.hide()))},this)),f&&this.$backdrop[0].offsetWidth,this.$backdrop.addClass("in"),!b)return;f?this.$backdrop.one("bsTransitionEnd",b).emulateTransitionEnd(c.BACKDROP_TRANSITION_DURATION):b()}else if(!this.isShown&&this.$backdrop){this.$backdrop.removeClass("in");var g=function(){d.removeBackdrop(),b&&b()};a.support.transition&&this.$element.hasClass("fade")?this.$backdrop.one("bsTransitionEnd",g).emulateTransitionEnd(c.BACKDROP_TRANSITION_DURATION):g()}else b&&b()},c.prototype.handleUpdate=function(){this.adjustDialog()},c.prototype.adjustDialog=function(){var a=this.$element[0].scrollHeight>document.documentElement.clientHeight;this.$element.css({paddingLeft:!this.bodyIsOverflowing&&a?this.scrollbarWidth:"",paddingRight:this.bodyIsOverflowing&&!a?this.scrollbarWidth:""})},c.prototype.resetAdjustments=function(){this.$element.css({paddingLeft:"",paddingRight:""})},c.prototype.checkScrollbar=function(){var a=window.innerWidth;if(!a){var b=document.documentElement.getBoundingClientRect();a=b.right-Math.abs(b.left)}this.bodyIsOverflowing=document.body.clientWidth<a,this.scrollbarWidth=this.measureScrollbar()},c.prototype.setScrollbar=function(){var a=parseInt(this.$body.css("padding-right")||0,10);this.originalBodyPad=document.body.style.paddingRight||"",this.bodyIsOverflowing&&this.$body.css("padding-right",a+this.scrollbarWidth)},c.prototype.resetScrollbar=function(){this.$body.css("padding-right",this.originalBodyPad)},c.prototype.measureScrollbar=function(){var a=document.createElement("div");a.className="modal-scrollbar-measure",this.$body.append(a);var b=a.offsetWidth-a.clientWidth;return this.$body[0].removeChild(a),b};var d=a.fn.modal;a.fn.modal=b,a.fn.modal.Constructor=c,a.fn.modal.noConflict=function(){return a.fn.modal=d,this},a(document).on("click.bs.modal.data-api",'[data-toggle="modal"]',function(c){var d=a(this),e=d.attr("href"),f=a(d.attr("data-target")||e&&e.replace(/.*(?=#[^\s]+$)/,"")),g=f.data("bs.modal")?"toggle":a.extend({remote:!/#/.test(e)&&e},f.data(),d.data());d.is("a")&&c.preventDefault(),f.one("show.bs.modal",function(a){a.isDefaultPrevented()||f.one("hidden.bs.modal",function(){d.is(":visible")&&d.trigger("focus")})}),b.call(f,g,this)})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.tooltip"),f="object"==typeof b&&b;(e||!/destroy|hide/.test(b))&&(e||d.data("bs.tooltip",e=new c(this,f)),"string"==typeof b&&e[b]())})}var c=function(a,b){this.type=null,this.options=null,this.enabled=null,this.timeout=null,this.hoverState=null,this.$element=null,this.inState=null,this.init("tooltip",a,b)};c.VERSION="3.3.6",c.TRANSITION_DURATION=150,c.DEFAULTS={animation:!0,placement:"top",selector:!1,template:'<div class="tooltip" role="tooltip"><div class="tooltip-arrow"></div><div class="tooltip-inner"></div></div>',trigger:"hover focus",title:"",delay:0,html:!1,container:!1,viewport:{selector:"body",padding:0}},c.prototype.init=function(b,c,d){if(this.enabled=!0,this.type=b,this.$element=a(c),this.options=this.getOptions(d),this.$viewport=this.options.viewport&&a(a.isFunction(this.options.viewport)?this.options.viewport.call(this,this.$element):this.options.viewport.selector||this.options.viewport),this.inState={click:!1,hover:!1,focus:!1},this.$element[0]instanceof document.constructor&&!this.options.selector)throw new Error("`selector` option must be specified when initializing "+this.type+" on the window.document object!");for(var e=this.options.trigger.split(" "),f=e.length;f--;){var g=e[f];if("click"==g)this.$element.on("click."+this.type,this.options.selector,a.proxy(this.toggle,this));else if("manual"!=g){var h="hover"==g?"mouseenter":"focusin",i="hover"==g?"mouseleave":"focusout";this.$element.on(h+"."+this.type,this.options.selector,a.proxy(this.enter,this)),this.$element.on(i+"."+this.type,this.options.selector,a.proxy(this.leave,this))}}this.options.selector?this._options=a.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},c.prototype.getDefaults=function(){return c.DEFAULTS},c.prototype.getOptions=function(b){return b=a.extend({},this.getDefaults(),this.$element.data(),b),b.delay&&"number"==typeof b.delay&&(b.delay={show:b.delay,hide:b.delay}),b},c.prototype.getDelegateOptions=function(){var b={},c=this.getDefaults();return this._options&&a.each(this._options,function(a,d){c[a]!=d&&(b[a]=d)}),b},c.prototype.enter=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget).data("bs."+this.type);return c||(c=new this.constructor(b.currentTarget,this.getDelegateOptions()),a(b.currentTarget).data("bs."+this.type,c)),b instanceof a.Event&&(c.inState["focusin"==b.type?"focus":"hover"]=!0),c.tip().hasClass("in")||"in"==c.hoverState?void(c.hoverState="in"):(clearTimeout(c.timeout),c.hoverState="in",c.options.delay&&c.options.delay.show?void(c.timeout=setTimeout(function(){"in"==c.hoverState&&c.show()},c.options.delay.show)):c.show())},c.prototype.isInStateTrue=function(){for(var a in this.inState)if(this.inState[a])return!0;return!1},c.prototype.leave=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget).data("bs."+this.type);return c||(c=new this.constructor(b.currentTarget,this.getDelegateOptions()),a(b.currentTarget).data("bs."+this.type,c)),b instanceof a.Event&&(c.inState["focusout"==b.type?"focus":"hover"]=!1),c.isInStateTrue()?void 0:(clearTimeout(c.timeout),c.hoverState="out",c.options.delay&&c.options.delay.hide?void(c.timeout=setTimeout(function(){"out"==c.hoverState&&c.hide()},c.options.delay.hide)):c.hide())},c.prototype.show=function(){var b=a.Event("show.bs."+this.type);if(this.hasContent()&&this.enabled){this.$element.trigger(b);var d=a.contains(this.$element[0].ownerDocument.documentElement,this.$element[0]);if(b.isDefaultPrevented()||!d)return;var e=this,f=this.tip(),g=this.getUID(this.type);this.setContent(),f.attr("id",g),this.$element.attr("aria-describedby",g),this.options.animation&&f.addClass("fade");var h="function"==typeof this.options.placement?this.options.placement.call(this,f[0],this.$element[0]):this.options.placement,i=/\s?auto?\s?/i,j=i.test(h);j&&(h=h.replace(i,"")||"top"),f.detach().css({top:0,left:0,display:"block"}).addClass(h).data("bs."+this.type,this),this.options.container?f.appendTo(this.options.container):f.insertAfter(this.$element),this.$element.trigger("inserted.bs."+this.type);var k=this.getPosition(),l=f[0].offsetWidth,m=f[0].offsetHeight;if(j){var n=h,o=this.getPosition(this.$viewport);h="bottom"==h&&k.bottom+m>o.bottom?"top":"top"==h&&k.top-m<o.top?"bottom":"right"==h&&k.right+l>o.width?"left":"left"==h&&k.left-l<o.left?"right":h,f.removeClass(n).addClass(h)}var p=this.getCalculatedOffset(h,k,l,m);this.applyPlacement(p,h);var q=function(){var a=e.hoverState;e.$element.trigger("shown.bs."+e.type),e.hoverState=null,"out"==a&&e.leave(e)};a.support.transition&&this.$tip.hasClass("fade")?f.one("bsTransitionEnd",q).emulateTransitionEnd(c.TRANSITION_DURATION):q()}},c.prototype.applyPlacement=function(b,c){var d=this.tip(),e=d[0].offsetWidth,f=d[0].offsetHeight,g=parseInt(d.css("margin-top"),10),h=parseInt(d.css("margin-left"),10);isNaN(g)&&(g=0),isNaN(h)&&(h=0),b.top+=g,b.left+=h,a.offset.setOffset(d[0],a.extend({using:function(a){d.css({top:Math.round(a.top),left:Math.round(a.left)})}},b),0),d.addClass("in");var i=d[0].offsetWidth,j=d[0].offsetHeight;"top"==c&&j!=f&&(b.top=b.top+f-j);var k=this.getViewportAdjustedDelta(c,b,i,j);k.left?b.left+=k.left:b.top+=k.top;var l=/top|bottom/.test(c),m=l?2*k.left-e+i:2*k.top-f+j,n=l?"offsetWidth":"offsetHeight";d.offset(b),this.replaceArrow(m,d[0][n],l)},c.prototype.replaceArrow=function(a,b,c){this.arrow().css(c?"left":"top",50*(1-a/b)+"%").css(c?"top":"left","")},c.prototype.setContent=function(){var a=this.tip(),b=this.getTitle();a.find(".tooltip-inner")[this.options.html?"html":"text"](b),a.removeClass("fade in top bottom left right")},c.prototype.hide=function(b){function d(){"in"!=e.hoverState&&f.detach(),e.$element.removeAttr("aria-describedby").trigger("hidden.bs."+e.type),b&&b()}var e=this,f=a(this.$tip),g=a.Event("hide.bs."+this.type);return this.$element.trigger(g),g.isDefaultPrevented()?void 0:(f.removeClass("in"),a.support.transition&&f.hasClass("fade")?f.one("bsTransitionEnd",d).emulateTransitionEnd(c.TRANSITION_DURATION):d(),this.hoverState=null,this)},c.prototype.fixTitle=function(){var a=this.$element;(a.attr("title")||"string"!=typeof a.attr("data-original-title"))&&a.attr("data-original-title",a.attr("title")||"").attr("title","")},c.prototype.hasContent=function(){return this.getTitle()},c.prototype.getPosition=function(b){b=b||this.$element;var c=b[0],d="BODY"==c.tagName,e=c.getBoundingClientRect();null==e.width&&(e=a.extend({},e,{width:e.right-e.left,height:e.bottom-e.top}));var f=d?{top:0,left:0}:b.offset(),g={scroll:d?document.documentElement.scrollTop||document.body.scrollTop:b.scrollTop()},h=d?{width:a(window).width(),height:a(window).height()}:null;return a.extend({},e,g,h,f)},c.prototype.getCalculatedOffset=function(a,b,c,d){return"bottom"==a?{top:b.top+b.height,left:b.left+b.width/2-c/2}:"top"==a?{top:b.top-d,left:b.left+b.width/2-c/2}:"left"==a?{top:b.top+b.height/2-d/2,left:b.left-c}:{top:b.top+b.height/2-d/2,left:b.left+b.width}},c.prototype.getViewportAdjustedDelta=function(a,b,c,d){var e={top:0,left:0};if(!this.$viewport)return e;var f=this.options.viewport&&this.options.viewport.padding||0,g=this.getPosition(this.$viewport);if(/right|left/.test(a)){var h=b.top-f-g.scroll,i=b.top+f-g.scroll+d;h<g.top?e.top=g.top-h:i>g.top+g.height&&(e.top=g.top+g.height-i)}else{var j=b.left-f,k=b.left+f+c;j<g.left?e.left=g.left-j:k>g.right&&(e.left=g.left+g.width-k)}return e},c.prototype.getTitle=function(){var a,b=this.$element,c=this.options;return a=b.attr("data-original-title")||("function"==typeof c.title?c.title.call(b[0]):c.title)},c.prototype.getUID=function(a){do a+=~~(1e6*Math.random());while(document.getElementById(a));return a},c.prototype.tip=function(){if(!this.$tip&&(this.$tip=a(this.options.template),1!=this.$tip.length))throw new Error(this.type+" `template` option must consist of exactly 1 top-level element!");return this.$tip},c.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".tooltip-arrow")},c.prototype.enable=function(){this.enabled=!0},c.prototype.disable=function(){this.enabled=!1},c.prototype.toggleEnabled=function(){this.enabled=!this.enabled},c.prototype.toggle=function(b){var c=this;b&&(c=a(b.currentTarget).data("bs."+this.type),c||(c=new this.constructor(b.currentTarget,this.getDelegateOptions()),a(b.currentTarget).data("bs."+this.type,c))),b?(c.inState.click=!c.inState.click,c.isInStateTrue()?c.enter(c):c.leave(c)):c.tip().hasClass("in")?c.leave(c):c.enter(c)},c.prototype.destroy=function(){var a=this;clearTimeout(this.timeout),this.hide(function(){a.$element.off("."+a.type).removeData("bs."+a.type),a.$tip&&a.$tip.detach(),a.$tip=null,a.$arrow=null,a.$viewport=null})};var d=a.fn.tooltip;a.fn.tooltip=b,a.fn.tooltip.Constructor=c,a.fn.tooltip.noConflict=function(){return a.fn.tooltip=d,this}}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.popover"),f="object"==typeof b&&b;(e||!/destroy|hide/.test(b))&&(e||d.data("bs.popover",e=new c(this,f)),"string"==typeof b&&e[b]())})}var c=function(a,b){this.init("popover",a,b)};if(!a.fn.tooltip)throw new Error("Popover requires tooltip.js");c.VERSION="3.3.6",c.DEFAULTS=a.extend({},a.fn.tooltip.Constructor.DEFAULTS,{placement:"right",trigger:"click",content:"",template:'<div class="popover" role="tooltip"><div class="arrow"></div><h3 class="popover-title"></h3><div class="popover-content"></div></div>'}),c.prototype=a.extend({},a.fn.tooltip.Constructor.prototype),c.prototype.constructor=c,c.prototype.getDefaults=function(){return c.DEFAULTS},c.prototype.setContent=function(){var a=this.tip(),b=this.getTitle(),c=this.getContent();a.find(".popover-title")[this.options.html?"html":"text"](b),a.find(".popover-content").children().detach().end()[this.options.html?"string"==typeof c?"html":"append":"text"](c),a.removeClass("fade top bottom left right in"),a.find(".popover-title").html()||a.find(".popover-title").hide()},c.prototype.hasContent=function(){return this.getTitle()||this.getContent()},c.prototype.getContent=function(){var a=this.$element,b=this.options;return a.attr("data-content")||("function"==typeof b.content?b.content.call(a[0]):b.content)},c.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".arrow")};var d=a.fn.popover;a.fn.popover=b,a.fn.popover.Constructor=c,a.fn.popover.noConflict=function(){return a.fn.popover=d,this}}(jQuery),+function(a){"use strict";function b(c,d){this.$body=a(document.body),this.$scrollElement=a(a(c).is(document.body)?window:c),this.options=a.extend({},b.DEFAULTS,d),this.selector=(this.options.target||"")+" .nav li > a",this.offsets=[],this.targets=[],this.activeTarget=null,this.scrollHeight=0,this.$scrollElement.on("scroll.bs.scrollspy",a.proxy(this.process,this)),this.refresh(),this.process()}function c(c){return this.each(function(){var d=a(this),e=d.data("bs.scrollspy"),f="object"==typeof c&&c;e||d.data("bs.scrollspy",e=new b(this,f)),"string"==typeof c&&e[c]()})}b.VERSION="3.3.6",b.DEFAULTS={offset:10},b.prototype.getScrollHeight=function(){return this.$scrollElement[0].scrollHeight||Math.max(this.$body[0].scrollHeight,document.documentElement.scrollHeight)},b.prototype.refresh=function(){var b=this,c="offset",d=0;this.offsets=[],this.targets=[],this.scrollHeight=this.getScrollHeight(),a.isWindow(this.$scrollElement[0])||(c="position",d=this.$scrollElement.scrollTop()),this.$body.find(this.selector).map(function(){var b=a(this),e=b.data("target")||b.attr("href"),f=/^#./.test(e)&&a(e);return f&&f.length&&f.is(":visible")&&[[f[c]().top+d,e]]||null}).sort(function(a,b){return a[0]-b[0]}).each(function(){b.offsets.push(this[0]),b.targets.push(this[1])})},b.prototype.process=function(){var a,b=this.$scrollElement.scrollTop()+this.options.offset,c=this.getScrollHeight(),d=this.options.offset+c-this.$scrollElement.height(),e=this.offsets,f=this.targets,g=this.activeTarget;if(this.scrollHeight!=c&&this.refresh(),b>=d)return g!=(a=f[f.length-1])&&this.activate(a);if(g&&b<e[0])return this.activeTarget=null,this.clear();for(a=e.length;a--;)g!=f[a]&&b>=e[a]&&(void 0===e[a+1]||b<e[a+1])&&this.activate(f[a])},b.prototype.activate=function(b){this.activeTarget=b,this.clear();var c=this.selector+'[data-target="'+b+'"],'+this.selector+'[href="'+b+'"]',d=a(c).parents("li").addClass("active");
-d.parent(".dropdown-menu").length&&(d=d.closest("li.dropdown").addClass("active")),d.trigger("activate.bs.scrollspy")},b.prototype.clear=function(){a(this.selector).parentsUntil(this.options.target,".active").removeClass("active")};var d=a.fn.scrollspy;a.fn.scrollspy=c,a.fn.scrollspy.Constructor=b,a.fn.scrollspy.noConflict=function(){return a.fn.scrollspy=d,this},a(window).on("load.bs.scrollspy.data-api",function(){a('[data-spy="scroll"]').each(function(){var b=a(this);c.call(b,b.data())})})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.tab");e||d.data("bs.tab",e=new c(this)),"string"==typeof b&&e[b]()})}var c=function(b){this.element=a(b)};c.VERSION="3.3.6",c.TRANSITION_DURATION=150,c.prototype.show=function(){var b=this.element,c=b.closest("ul:not(.dropdown-menu)"),d=b.data("target");if(d||(d=b.attr("href"),d=d&&d.replace(/.*(?=#[^\s]*$)/,"")),!b.parent("li").hasClass("active")){var e=c.find(".active:last a"),f=a.Event("hide.bs.tab",{relatedTarget:b[0]}),g=a.Event("show.bs.tab",{relatedTarget:e[0]});if(e.trigger(f),b.trigger(g),!g.isDefaultPrevented()&&!f.isDefaultPrevented()){var h=a(d);this.activate(b.closest("li"),c),this.activate(h,h.parent(),function(){e.trigger({type:"hidden.bs.tab",relatedTarget:b[0]}),b.trigger({type:"shown.bs.tab",relatedTarget:e[0]})})}}},c.prototype.activate=function(b,d,e){function f(){g.removeClass("active").find("> .dropdown-menu > .active").removeClass("active").end().find('[data-toggle="tab"]').attr("aria-expanded",!1),b.addClass("active").find('[data-toggle="tab"]').attr("aria-expanded",!0),h?(b[0].offsetWidth,b.addClass("in")):b.removeClass("fade"),b.parent(".dropdown-menu").length&&b.closest("li.dropdown").addClass("active").end().find('[data-toggle="tab"]').attr("aria-expanded",!0),e&&e()}var g=d.find("> .active"),h=e&&a.support.transition&&(g.length&&g.hasClass("fade")||!!d.find("> .fade").length);g.length&&h?g.one("bsTransitionEnd",f).emulateTransitionEnd(c.TRANSITION_DURATION):f(),g.removeClass("in")};var d=a.fn.tab;a.fn.tab=b,a.fn.tab.Constructor=c,a.fn.tab.noConflict=function(){return a.fn.tab=d,this};var e=function(c){c.preventDefault(),b.call(a(this),"show")};a(document).on("click.bs.tab.data-api",'[data-toggle="tab"]',e).on("click.bs.tab.data-api",'[data-toggle="pill"]',e)}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.affix"),f="object"==typeof b&&b;e||d.data("bs.affix",e=new c(this,f)),"string"==typeof b&&e[b]()})}var c=function(b,d){this.options=a.extend({},c.DEFAULTS,d),this.$target=a(this.options.target).on("scroll.bs.affix.data-api",a.proxy(this.checkPosition,this)).on("click.bs.affix.data-api",a.proxy(this.checkPositionWithEventLoop,this)),this.$element=a(b),this.affixed=null,this.unpin=null,this.pinnedOffset=null,this.checkPosition()};c.VERSION="3.3.6",c.RESET="affix affix-top affix-bottom",c.DEFAULTS={offset:0,target:window},c.prototype.getState=function(a,b,c,d){var e=this.$target.scrollTop(),f=this.$element.offset(),g=this.$target.height();if(null!=c&&"top"==this.affixed)return c>e?"top":!1;if("bottom"==this.affixed)return null!=c?e+this.unpin<=f.top?!1:"bottom":a-d>=e+g?!1:"bottom";var h=null==this.affixed,i=h?e:f.top,j=h?g:b;return null!=c&&c>=e?"top":null!=d&&i+j>=a-d?"bottom":!1},c.prototype.getPinnedOffset=function(){if(this.pinnedOffset)return this.pinnedOffset;this.$element.removeClass(c.RESET).addClass("affix");var a=this.$target.scrollTop(),b=this.$element.offset();return this.pinnedOffset=b.top-a},c.prototype.checkPositionWithEventLoop=function(){setTimeout(a.proxy(this.checkPosition,this),1)},c.prototype.checkPosition=function(){if(this.$element.is(":visible")){var b=this.$element.height(),d=this.options.offset,e=d.top,f=d.bottom,g=Math.max(a(document).height(),a(document.body).height());"object"!=typeof d&&(f=e=d),"function"==typeof e&&(e=d.top(this.$element)),"function"==typeof f&&(f=d.bottom(this.$element));var h=this.getState(g,b,e,f);if(this.affixed!=h){null!=this.unpin&&this.$element.css("top","");var i="affix"+(h?"-"+h:""),j=a.Event(i+".bs.affix");if(this.$element.trigger(j),j.isDefaultPrevented())return;this.affixed=h,this.unpin="bottom"==h?this.getPinnedOffset():null,this.$element.removeClass(c.RESET).addClass(i).trigger(i.replace("affix","affixed")+".bs.affix")}"bottom"==h&&this.$element.offset({top:g-b-f})}};var d=a.fn.affix;a.fn.affix=b,a.fn.affix.Constructor=c,a.fn.affix.noConflict=function(){return a.fn.affix=d,this},a(window).on("load",function(){a('[data-spy="affix"]').each(function(){var c=a(this),d=c.data();d.offset=d.offset||{},null!=d.offsetBottom&&(d.offset.bottom=d.offsetBottom),null!=d.offsetTop&&(d.offset.top=d.offsetTop),b.call(c,d)})})}(jQuery);
\ No newline at end of file
+d.parent(".dropdown-menu").length&&(d=d.closest("li.dropdown").addClass("active")),d.trigger("activate.bs.scrollspy")},b.prototype.clear=function(){a(this.selector).parentsUntil(this.options.target,".active").removeClass("active")};var d=a.fn.scrollspy;a.fn.scrollspy=c,a.fn.scrollspy.Constructor=b,a.fn.scrollspy.noConflict=function(){return a.fn.scrollspy=d,this},a(window).on("load.bs.scrollspy.data-api",function(){a('[data-spy="scroll"]').each(function(){var b=a(this);c.call(b,b.data())})})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.tab");e||d.data("bs.tab",e=new c(this)),"string"==typeof b&&e[b]()})}var c=function(b){this.element=a(b)};c.VERSION="3.3.6",c.TRANSITION_DURATION=150,c.prototype.show=function(){var b=this.element,c=b.closest("ul:not(.dropdown-menu)"),d=b.data("target");if(d||(d=b.attr("href"),d=d&&d.replace(/.*(?=#[^\s]*$)/,"")),!b.parent("li").hasClass("active")){var e=c.find(".active:last a"),f=a.Event("hide.bs.tab",{relatedTarget:b[0]}),g=a.Event("show.bs.tab",{relatedTarget:e[0]});if(e.trigger(f),b.trigger(g),!g.isDefaultPrevented()&&!f.isDefaultPrevented()){var h=a(d);this.activate(b.closest("li"),c),this.activate(h,h.parent(),function(){e.trigger({type:"hidden.bs.tab",relatedTarget:b[0]}),b.trigger({type:"shown.bs.tab",relatedTarget:e[0]})})}}},c.prototype.activate=function(b,d,e){function f(){g.removeClass("active").find("> .dropdown-menu > .active").removeClass("active").end().find('[data-toggle="tab"]').attr("aria-expanded",!1),b.addClass("active").find('[data-toggle="tab"]').attr("aria-expanded",!0),h?(b[0].offsetWidth,b.addClass("in")):b.removeClass("fade"),b.parent(".dropdown-menu").length&&b.closest("li.dropdown").addClass("active").end().find('[data-toggle="tab"]').attr("aria-expanded",!0),e&&e()}var g=d.find("> .active"),h=e&&a.support.transition&&(g.length&&g.hasClass("fade")||!!d.find("> .fade").length);g.length&&h?g.one("bsTransitionEnd",f).emulateTransitionEnd(c.TRANSITION_DURATION):f(),g.removeClass("in")};var d=a.fn.tab;a.fn.tab=b,a.fn.tab.Constructor=c,a.fn.tab.noConflict=function(){return a.fn.tab=d,this};var e=function(c){c.preventDefault(),b.call(a(this),"show")};a(document).on("click.bs.tab.data-api",'[data-toggle="tab"]',e).on("click.bs.tab.data-api",'[data-toggle="pill"]',e)}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.affix"),f="object"==typeof b&&b;e||d.data("bs.affix",e=new c(this,f)),"string"==typeof b&&e[b]()})}var c=function(b,d){this.options=a.extend({},c.DEFAULTS,d),this.$target=a(this.options.target).on("scroll.bs.affix.data-api",a.proxy(this.checkPosition,this)).on("click.bs.affix.data-api",a.proxy(this.checkPositionWithEventLoop,this)),this.$element=a(b),this.affixed=null,this.unpin=null,this.pinnedOffset=null,this.checkPosition()};c.VERSION="3.3.6",c.RESET="affix affix-top affix-bottom",c.DEFAULTS={offset:0,target:window},c.prototype.getState=function(a,b,c,d){var e=this.$target.scrollTop(),f=this.$element.offset(),g=this.$target.height();if(null!=c&&"top"==this.affixed)return c>e?"top":!1;if("bottom"==this.affixed)return null!=c?e+this.unpin<=f.top?!1:"bottom":a-d>=e+g?!1:"bottom";var h=null==this.affixed,i=h?e:f.top,j=h?g:b;return null!=c&&c>=e?"top":null!=d&&i+j>=a-d?"bottom":!1},c.prototype.getPinnedOffset=function(){if(this.pinnedOffset)return this.pinnedOffset;this.$element.removeClass(c.RESET).addClass("affix");var a=this.$target.scrollTop(),b=this.$element.offset();return this.pinnedOffset=b.top-a},c.prototype.checkPositionWithEventLoop=function(){setTimeout(a.proxy(this.checkPosition,this),1)},c.prototype.checkPosition=function(){if(this.$element.is(":visible")){var b=this.$element.height(),d=this.options.offset,e=d.top,f=d.bottom,g=Math.max(a(document).height(),a(document.body).height());"object"!=typeof d&&(f=e=d),"function"==typeof e&&(e=d.top(this.$element)),"function"==typeof f&&(f=d.bottom(this.$element));var h=this.getState(g,b,e,f);if(this.affixed!=h){null!=this.unpin&&this.$element.css("top","");var i="affix"+(h?"-"+h:""),j=a.Event(i+".bs.affix");if(this.$element.trigger(j),j.isDefaultPrevented())return;this.affixed=h,this.unpin="bottom"==h?this.getPinnedOffset():null,this.$element.removeClass(c.RESET).addClass(i).trigger(i.replace("affix","affixed")+".bs.affix")}"bottom"==h&&this.$element.offset({top:g-b-f})}};var d=a.fn.affix;a.fn.affix=b,a.fn.affix.Constructor=c,a.fn.affix.noConflict=function(){return a.fn.affix=d,this},a(window).on("load",function(){a('[data-spy="affix"]').each(function(){var c=a(this),d=c.data();d.offset=d.offset||{},null!=d.offsetBottom&&(d.offset.bottom=d.offsetBottom),null!=d.offsetTop&&(d.offset.top=d.offsetTop),b.call(c,d)})})}(jQuery);
diff --git a/docs/start/quickstart.rst b/docs/start/quickstart.rst
index 72ca0eb..010d323 100644
--- a/docs/start/quickstart.rst
+++ b/docs/start/quickstart.rst
@@ -109,7 +109,7 @@
 
 ::
     
-    > ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/my_namespace messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+    > ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/my_namespace messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
 
 
 Step 7: Write some records
@@ -122,6 +122,6 @@
 
 ::
     
-    > ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.ConsoleProxyMultiWriter 'inet!127.0.0.1:8000' messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+    > ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.ConsoleProxyMultiWriter 'inet!127.0.0.1:8000' messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
 
 If you have each of the above commands running in a different terminal then you should now be able to type messages into the writer terminal and see them appear in the reader terminal.
diff --git a/docs/tutorials/basic-1.rst b/docs/tutorials/basic-1.rst
index 47a4bad..07826a4 100644
--- a/docs/tutorials/basic-1.rst
+++ b/docs/tutorials/basic-1.rst
@@ -171,8 +171,8 @@
 ::
 
         // Tailing Stream `basic-stream-1`
-        // runner run com.twitter.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-1
+        // runner run org.apache.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-1
 
 
 Write records
@@ -183,8 +183,8 @@
 ::
 
         // Write Records into Stream `basic-stream-1`
-        // runner run com.twitter.distributedlog.basic.ConsoleWriter ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.ConsoleWriter distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-1
+        // runner run org.apache.distributedlog.basic.ConsoleWriter ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.ConsoleWriter distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-1
 
 
 Check the results
@@ -218,9 +218,9 @@
 ::
 
         Opening log stream basic-stream-1
-        Exception in thread "main" com.twitter.distributedlog.exceptions.OwnershipAcquireFailedException: LockPath - /messaging/distributedlog/basic-stream-1/<default>/lock: Lock acquisition failed, the current owner is console-writer
-            at com.twitter.distributedlog.lock.ZKSessionLock$8.apply(ZKSessionLock.java:570)
-            at com.twitter.distributedlog.lock.ZKSessionLock$8.apply(ZKSessionLock.java:567)
+        Exception in thread "main" org.apache.distributedlog.exceptions.OwnershipAcquireFailedException: LockPath - /messaging/distributedlog/basic-stream-1/<default>/lock: Lock acquisition failed, the current owner is console-writer
+            at org.apache.distributedlog.lock.ZKSessionLock$8.apply(ZKSessionLock.java:570)
+            at org.apache.distributedlog.lock.ZKSessionLock$8.apply(ZKSessionLock.java:567)
             at com.twitter.util.Future$$anonfun$map$1$$anonfun$apply$8.apply(Future.scala:1041)
             at com.twitter.util.Try$.apply(Try.scala:13)
             at com.twitter.util.Future$.apply(Future.scala:132)
@@ -239,7 +239,7 @@
             at com.twitter.util.Promise.updateIfEmpty(Promise.scala:716)
             at com.twitter.util.Promise.update(Promise.scala:694)
             at com.twitter.util.Promise.setValue(Promise.scala:670)
-            at com.twitter.distributedlog.lock.ZKSessionLock$9.safeRun(ZKSessionLock.java:622)
+            at org.apache.distributedlog.lock.ZKSessionLock$9.safeRun(ZKSessionLock.java:622)
             at org.apache.bookkeeper.util.SafeRunnable.run(SafeRunnable.java:31)
             at java.util.concurrent.Executors$RunnableAdapter.call(Executors.java:471)
             at java.util.concurrent.FutureTask.run(FutureTask.java:262)
diff --git a/docs/tutorials/basic-2.rst b/docs/tutorials/basic-2.rst
index 10fb794..ef4c7b6 100644
--- a/docs/tutorials/basic-2.rst
+++ b/docs/tutorials/basic-2.rst
@@ -125,7 +125,7 @@
 ::
 
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
 
 
 Create the stream
@@ -148,8 +148,8 @@
 ::
 
         // Tailing Stream `basic-stream-2`
-        // runner run com.twitter.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-2
+        // runner run org.apache.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-2
 
 
 Write records
@@ -160,8 +160,8 @@
 ::
 
         // Write Records into Stream `basic-stream-2`
-        // runner run com.twitter.distributedlog.basic.ConsoleProxyWriter ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.ConsoleProxyWriter 'inet!127.0.0.1:8000' basic-stream-2
+        // runner run org.apache.distributedlog.basic.ConsoleProxyWriter ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.ConsoleProxyWriter 'inet!127.0.0.1:8000' basic-stream-2
 
 
 Check the results
diff --git a/docs/tutorials/basic-3.rst b/docs/tutorials/basic-3.rst
index 26f3336..55de7b0 100644
--- a/docs/tutorials/basic-3.rst
+++ b/docs/tutorials/basic-3.rst
@@ -172,7 +172,7 @@
 ::
 
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
 
 
 Create multiple streams
@@ -195,8 +195,8 @@
 ::
 
         // Tailing Stream `basic-stream-{3-7}`
-        // runner run com.twitter.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[,${stream}]
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-3,basic-stream-4,basic-stream-5,basic-stream-6,basic-stream-7
+        // runner run org.apache.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[,${stream}]
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-3,basic-stream-4,basic-stream-5,basic-stream-6,basic-stream-7
 
 
 Write the records
@@ -207,8 +207,8 @@
 ::
 
         // Write Records into Stream `basic-stream-{3-7}`
-        // runner run com.twitter.distributedlog.basic.ConsoleProxyMultiWriter ${distributedlog-uri} ${stream}[,${stream}]
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.ConsoleProxyMultiWriter 'inet!127.0.0.1:8000' basic-stream-3,basic-stream-4,basic-stream-5,basic-stream-6,basic-stream-7
+        // runner run org.apache.distributedlog.basic.ConsoleProxyMultiWriter ${distributedlog-uri} ${stream}[,${stream}]
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.ConsoleProxyMultiWriter 'inet!127.0.0.1:8000' basic-stream-3,basic-stream-4,basic-stream-5,basic-stream-6,basic-stream-7
 
 Check the results
 -----------------
diff --git a/docs/tutorials/basic-4.rst b/docs/tutorials/basic-4.rst
index fe905d3..bcbacd1 100644
--- a/docs/tutorials/basic-4.rst
+++ b/docs/tutorials/basic-4.rst
@@ -150,7 +150,7 @@
 ::
 
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
 
 
 Create the stream
@@ -173,8 +173,8 @@
 ::
 
         // Tailing Stream `basic-stream-8`
-        // runner run com.twitter.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-8
+        // runner run org.apache.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-8
 
 
 Write records
@@ -185,8 +185,8 @@
 ::
 
         // Write Records into Stream `basic-stream-8`
-        // runner run com.twitter.distributedlog.basic.AtomicWriter ${distributedlog-uri} ${stream} ${message}[, ${message}]
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.AtomicWriter 'inet!127.0.0.1:8000' basic-stream-8 "message-1" "message-2" "message-3" "message-4" "message-5"
+        // runner run org.apache.distributedlog.basic.AtomicWriter ${distributedlog-uri} ${stream} ${message}[, ${message}]
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.AtomicWriter 'inet!127.0.0.1:8000' basic-stream-8 "message-1" "message-2" "message-3" "message-4" "message-5"
 
 
 Check the results
diff --git a/docs/tutorials/basic-5.rst b/docs/tutorials/basic-5.rst
index 329bba6..94220e8 100644
--- a/docs/tutorials/basic-5.rst
+++ b/docs/tutorials/basic-5.rst
@@ -149,7 +149,7 @@
 ::
 
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
 
 
 Create the stream
@@ -172,8 +172,8 @@
 ::
 
         // Tailing Stream `basic-stream-9`
-        // runner run com.twitter.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-9
+        // runner run org.apache.distributedlog.basic.TailReader ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.TailReader distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-9
 
 
 Write records
@@ -184,8 +184,8 @@
 ::
 
         // Write Records into Stream `basic-stream-9`
-        // runner run com.twitter.distributedlog.basic.ConsoleProxyWriter ${distributedlog-uri} ${stream}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.ConsoleProxyWriter 'inet!127.0.0.1:8000' basic-stream-9
+        // runner run org.apache.distributedlog.basic.ConsoleProxyWriter ${distributedlog-uri} ${stream}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.ConsoleProxyWriter 'inet!127.0.0.1:8000' basic-stream-9
 
 
 Check the results
diff --git a/docs/tutorials/basic-6.rst b/docs/tutorials/basic-6.rst
index 38109e7..4993fe4 100644
--- a/docs/tutorials/basic-6.rst
+++ b/docs/tutorials/basic-6.rst
@@ -139,7 +139,7 @@
 ::
 
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
 
 
 Create the stream
@@ -162,8 +162,8 @@
 ::
 
         // Write Records into Stream `basic-stream-10` in 1 requests/second
-        // runner run com.twitter.distributedlog.basic.RecordGenerator ${distributedlog-uri} ${stream} ${rate}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.RecordGenerator 'inet!127.0.0.1:8000' basic-stream-10 1
+        // runner run org.apache.distributedlog.basic.RecordGenerator ${distributedlog-uri} ${stream} ${rate}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.RecordGenerator 'inet!127.0.0.1:8000' basic-stream-10 1
 
 
 Rewind the stream
@@ -174,8 +174,8 @@
 ::
 
         // Rewind `basic-stream-10`
-        // runner run com.twitter.distributedlog.basic.StreamRewinder ${distributedlog-uri} ${stream} ${seconds-to-rewind}
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.StreamRewinder distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-10  30
+        // runner run org.apache.distributedlog.basic.StreamRewinder ${distributedlog-uri} ${stream} ${seconds-to-rewind}
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.StreamRewinder distributedlog://127.0.0.1:7000/messaging/distributedlog basic-stream-10  30
 
 
 Check the results
diff --git a/docs/tutorials/messaging-1.rst b/docs/tutorials/messaging-1.rst
index ba306ee..5536811 100644
--- a/docs/tutorials/messaging-1.rst
+++ b/docs/tutorials/messaging-1.rst
@@ -73,7 +73,7 @@
 ::
 
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
 
 
 Create the stream
@@ -96,8 +96,8 @@
 ::
 
         // Tailing Stream `messaging-stream-{1,5}`
-        // runner run com.twitter.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[, ${stream}]
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+        // runner run org.apache.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[, ${stream}]
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
 
 
 Write records
@@ -108,8 +108,8 @@
 ::
 
         // Write Records into Stream `messaging-stream-{1,5}`
-        // runner run com.twitter.distributedlog.messaging.ConsoleProxyPartitionedMultiWriter ${distributedlog-uri} ${stream}[, ${stream}]
-        ./distributedlog-tutorials/distributedlog-messaging/bin/runner run com.twitter.distributedlog.messaging.ConsoleProxyPartitionedMultiWriter 'inet!127.0.0.1:8000' messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+        // runner run org.apache.distributedlog.messaging.ConsoleProxyPartitionedMultiWriter ${distributedlog-uri} ${stream}[, ${stream}]
+        ./distributedlog-tutorials/distributedlog-messaging/bin/runner run org.apache.distributedlog.messaging.ConsoleProxyPartitionedMultiWriter 'inet!127.0.0.1:8000' messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
 
 
 Check the results
diff --git a/docs/tutorials/messaging-2.rst b/docs/tutorials/messaging-2.rst
index a51ca32..dc1cf5d 100644
--- a/docs/tutorials/messaging-2.rst
+++ b/docs/tutorials/messaging-2.rst
@@ -112,7 +112,7 @@
 ::
 
         // DistributedLogServerApp -p ${service-port} --shard-id ${shard-id} -sp ${stats-port} -u {distributedlog-uri} -mx -c ${conf-file}
-        ./distributedlog-service/bin/dlog com.twitter.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
+        ./distributedlog-service/bin/dlog org.apache.distributedlog.service.DistributedLogServerApp -p 8000 --shard-id 1 -sp 8001 -u distributedlog://127.0.0.1:7000/messaging/distributedlog -mx -c ${distributedlog-repo}/distributedlog-service/conf/distributedlog_proxy.conf
 
 
 Create the stream
@@ -135,8 +135,8 @@
 ::
 
         // Tailing Stream `messaging-stream-{1,5}`
-        // runner run com.twitter.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[, ${stream}]
-        ./distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+        // runner run org.apache.distributedlog.basic.MultiReader ${distributedlog-uri} ${stream}[, ${stream}]
+        ./distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.MultiReader distributedlog://127.0.0.1:7000/messaging/distributedlog messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
 
 
 Write records
@@ -147,8 +147,8 @@
 ::
 
         // Write Records into Stream `messaging-stream-{1,5}`
-        // runner run com.twitter.distributedlog.messaging.ConsoleProxyRRMultiWriter ${distributedlog-uri} ${stream}[, ${stream}]
-        ./distributedlog-tutorials/distributedlog-messaging/bin/runner run com.twitter.distributedlog.messaging.ConsoleProxyRRMultiWriter 'inet!127.0.0.1:8000' messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
+        // runner run org.apache.distributedlog.messaging.ConsoleProxyRRMultiWriter ${distributedlog-uri} ${stream}[, ${stream}]
+        ./distributedlog-tutorials/distributedlog-messaging/bin/runner run org.apache.distributedlog.messaging.ConsoleProxyRRMultiWriter 'inet!127.0.0.1:8000' messaging-stream-1,messaging-stream-2,messaging-stream-3,messaging-stream-4,messaging-stream-5
 
 
 Check the results
diff --git a/docs/user_guide/implementation/storage.rst b/docs/user_guide/implementation/storage.rst
index 80bc7c9..1fa3de0 100644
--- a/docs/user_guide/implementation/storage.rst
+++ b/docs/user_guide/implementation/storage.rst
@@ -322,5 +322,5 @@
 
 ::
 
-    bkEnsemblePlacementDnsResolverClass=com.twitter.distributedlog.net.DNSResolverForRacks
+    bkEnsemblePlacementDnsResolverClass=org.apache.distributedlog.net.DNSResolverForRacks
 
diff --git a/pom.xml b/pom.xml
index 52414da..423a8db 100644
--- a/pom.xml
+++ b/pom.xml
@@ -112,7 +112,7 @@
           <groups>
             <group>
               <title>DistributedLog</title>
-              <packages>com.twitter.distributedlog.*</packages>
+              <packages>org.apache.distributedlog.*</packages>
             </group>
           </groups>
         </configuration>
@@ -194,6 +194,7 @@
 	    <exclude>OWNERS</exclude>
 	    <exclude>CONFIG.ini</exclude>
             <exclude>**/**.md</exclude>
+            <exclude>scripts/dev/reviewers</exclude>
           </excludes>
         </configuration>
       </plugin>
diff --git a/scripts/integration/smoketest.sh b/scripts/integration/smoketest.sh
index 5141756..991dc6b 100755
--- a/scripts/integration/smoketest.sh
+++ b/scripts/integration/smoketest.sh
@@ -77,11 +77,11 @@
 ${DLOG_ROOT}/distributedlog-service/bin/dlog-daemon.sh start writeproxy
 
 # tail the the streams
-nohup ${DLOG_ROOT}/distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.MultiReader distributedlog://127.0.0.1:${ZK_PORT}/messaging/${NAMESPACE} ${SMOKESTREAM_PREFIX}1,${SMOKESTREAM_PREFIX}2,${SMOKESTREAM_PREFIX}3,${SMOKESTREAM_PREFIX}4,${SMOKESTREAM_PREFIX}5 > ${LOG_DIR}/reader.out 2>&1&
+nohup ${DLOG_ROOT}/distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.MultiReader distributedlog://127.0.0.1:${ZK_PORT}/messaging/${NAMESPACE} ${SMOKESTREAM_PREFIX}1,${SMOKESTREAM_PREFIX}2,${SMOKESTREAM_PREFIX}3,${SMOKESTREAM_PREFIX}4,${SMOKESTREAM_PREFIX}5 > ${LOG_DIR}/reader.out 2>&1&
 echo $! > ${LOG_DIR}/reader.pid
 
 # generate the records
-nohup ${DLOG_ROOT}/distributedlog-tutorials/distributedlog-basic/bin/runner run com.twitter.distributedlog.basic.RecordGenerator "inet!127.0.0.1:${WP_SERVICE_PORT}" ${SMOKESTREAM_PREFIX}1 1 > ${LOG_DIR}/writer.out 2>&1&
+nohup ${DLOG_ROOT}/distributedlog-tutorials/distributedlog-basic/bin/runner run org.apache.distributedlog.basic.RecordGenerator "inet!127.0.0.1:${WP_SERVICE_PORT}" ${SMOKESTREAM_PREFIX}1 1 > ${LOG_DIR}/writer.out 2>&1&
 echo $! > ${LOG_DIR}/writer.pid
 
 # wait for 20 seconds
diff --git a/website/fonts/bootstrap/glyphicons-halflings-regular.svg b/website/fonts/bootstrap/glyphicons-halflings-regular.svg
index 94fb549..ff1de8e 100755
--- a/website/fonts/bootstrap/glyphicons-halflings-regular.svg
+++ b/website/fonts/bootstrap/glyphicons-halflings-regular.svg
@@ -285,4 +285,4 @@
 <glyph unicode="&#x1f511;" d="M250 1200h600q21 0 35.5 -14.5t14.5 -35.5v-400q0 -21 -14.5 -35.5t-35.5 -14.5h-150v-500l-255 -178q-19 -9 -32 -1t-13 29v650h-150q-21 0 -35.5 14.5t-14.5 35.5v400q0 21 14.5 35.5t35.5 14.5zM400 1100v-100h300v100h-300z" />
 <glyph unicode="&#x1f6aa;" d="M250 1200h750q39 0 69.5 -40.5t30.5 -84.5v-933l-700 -117v950l600 125h-700v-1000h-100v1025q0 23 15.5 49t34.5 26zM500 525v-100l100 20v100z" />
 </font>
-</defs></svg> 
\ No newline at end of file
+</defs></svg> 
diff --git a/website/js/bootstrap.min.js b/website/js/bootstrap.min.js
index e79c065..e364a13 100755
--- a/website/js/bootstrap.min.js
+++ b/website/js/bootstrap.min.js
@@ -4,4 +4,4 @@
  * Licensed under the MIT license
  */
 if("undefined"==typeof jQuery)throw new Error("Bootstrap's JavaScript requires jQuery");+function(a){"use strict";var b=a.fn.jquery.split(" ")[0].split(".");if(b[0]<2&&b[1]<9||1==b[0]&&9==b[1]&&b[2]<1||b[0]>2)throw new Error("Bootstrap's JavaScript requires jQuery version 1.9.1 or higher, but lower than version 3")}(jQuery),+function(a){"use strict";function b(){var a=document.createElement("bootstrap"),b={WebkitTransition:"webkitTransitionEnd",MozTransition:"transitionend",OTransition:"oTransitionEnd otransitionend",transition:"transitionend"};for(var c in b)if(void 0!==a.style[c])return{end:b[c]};return!1}a.fn.emulateTransitionEnd=function(b){var c=!1,d=this;a(this).one("bsTransitionEnd",function(){c=!0});var e=function(){c||a(d).trigger(a.support.transition.end)};return setTimeout(e,b),this},a(function(){a.support.transition=b(),a.support.transition&&(a.event.special.bsTransitionEnd={bindType:a.support.transition.end,delegateType:a.support.transition.end,handle:function(b){return a(b.target).is(this)?b.handleObj.handler.apply(this,arguments):void 0}})})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var c=a(this),e=c.data("bs.alert");e||c.data("bs.alert",e=new d(this)),"string"==typeof b&&e[b].call(c)})}var c='[data-dismiss="alert"]',d=function(b){a(b).on("click",c,this.close)};d.VERSION="3.3.6",d.TRANSITION_DURATION=150,d.prototype.close=function(b){function c(){g.detach().trigger("closed.bs.alert").remove()}var e=a(this),f=e.attr("data-target");f||(f=e.attr("href"),f=f&&f.replace(/.*(?=#[^\s]*$)/,""));var g=a(f);b&&b.preventDefault(),g.length||(g=e.closest(".alert")),g.trigger(b=a.Event("close.bs.alert")),b.isDefaultPrevented()||(g.removeClass("in"),a.support.transition&&g.hasClass("fade")?g.one("bsTransitionEnd",c).emulateTransitionEnd(d.TRANSITION_DURATION):c())};var e=a.fn.alert;a.fn.alert=b,a.fn.alert.Constructor=d,a.fn.alert.noConflict=function(){return a.fn.alert=e,this},a(document).on("click.bs.alert.data-api",c,d.prototype.close)}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.button"),f="object"==typeof b&&b;e||d.data("bs.button",e=new c(this,f)),"toggle"==b?e.toggle():b&&e.setState(b)})}var c=function(b,d){this.$element=a(b),this.options=a.extend({},c.DEFAULTS,d),this.isLoading=!1};c.VERSION="3.3.6",c.DEFAULTS={loadingText:"loading..."},c.prototype.setState=function(b){var c="disabled",d=this.$element,e=d.is("input")?"val":"html",f=d.data();b+="Text",null==f.resetText&&d.data("resetText",d[e]()),setTimeout(a.proxy(function(){d[e](null==f[b]?this.options[b]:f[b]),"loadingText"==b?(this.isLoading=!0,d.addClass(c).attr(c,c)):this.isLoading&&(this.isLoading=!1,d.removeClass(c).removeAttr(c))},this),0)},c.prototype.toggle=function(){var a=!0,b=this.$element.closest('[data-toggle="buttons"]');if(b.length){var c=this.$element.find("input");"radio"==c.prop("type")?(c.prop("checked")&&(a=!1),b.find(".active").removeClass("active"),this.$element.addClass("active")):"checkbox"==c.prop("type")&&(c.prop("checked")!==this.$element.hasClass("active")&&(a=!1),this.$element.toggleClass("active")),c.prop("checked",this.$element.hasClass("active")),a&&c.trigger("change")}else this.$element.attr("aria-pressed",!this.$element.hasClass("active")),this.$element.toggleClass("active")};var d=a.fn.button;a.fn.button=b,a.fn.button.Constructor=c,a.fn.button.noConflict=function(){return a.fn.button=d,this},a(document).on("click.bs.button.data-api",'[data-toggle^="button"]',function(c){var d=a(c.target);d.hasClass("btn")||(d=d.closest(".btn")),b.call(d,"toggle"),a(c.target).is('input[type="radio"]')||a(c.target).is('input[type="checkbox"]')||c.preventDefault()}).on("focus.bs.button.data-api blur.bs.button.data-api",'[data-toggle^="button"]',function(b){a(b.target).closest(".btn").toggleClass("focus",/^focus(in)?$/.test(b.type))})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.carousel"),f=a.extend({},c.DEFAULTS,d.data(),"object"==typeof b&&b),g="string"==typeof b?b:f.slide;e||d.data("bs.carousel",e=new c(this,f)),"number"==typeof b?e.to(b):g?e[g]():f.interval&&e.pause().cycle()})}var c=function(b,c){this.$element=a(b),this.$indicators=this.$element.find(".carousel-indicators"),this.options=c,this.paused=null,this.sliding=null,this.interval=null,this.$active=null,this.$items=null,this.options.keyboard&&this.$element.on("keydown.bs.carousel",a.proxy(this.keydown,this)),"hover"==this.options.pause&&!("ontouchstart"in document.documentElement)&&this.$element.on("mouseenter.bs.carousel",a.proxy(this.pause,this)).on("mouseleave.bs.carousel",a.proxy(this.cycle,this))};c.VERSION="3.3.6",c.TRANSITION_DURATION=600,c.DEFAULTS={interval:5e3,pause:"hover",wrap:!0,keyboard:!0},c.prototype.keydown=function(a){if(!/input|textarea/i.test(a.target.tagName)){switch(a.which){case 37:this.prev();break;case 39:this.next();break;default:return}a.preventDefault()}},c.prototype.cycle=function(b){return b||(this.paused=!1),this.interval&&clearInterval(this.interval),this.options.interval&&!this.paused&&(this.interval=setInterval(a.proxy(this.next,this),this.options.interval)),this},c.prototype.getItemIndex=function(a){return this.$items=a.parent().children(".item"),this.$items.index(a||this.$active)},c.prototype.getItemForDirection=function(a,b){var c=this.getItemIndex(b),d="prev"==a&&0===c||"next"==a&&c==this.$items.length-1;if(d&&!this.options.wrap)return b;var e="prev"==a?-1:1,f=(c+e)%this.$items.length;return this.$items.eq(f)},c.prototype.to=function(a){var b=this,c=this.getItemIndex(this.$active=this.$element.find(".item.active"));return a>this.$items.length-1||0>a?void 0:this.sliding?this.$element.one("slid.bs.carousel",function(){b.to(a)}):c==a?this.pause().cycle():this.slide(a>c?"next":"prev",this.$items.eq(a))},c.prototype.pause=function(b){return b||(this.paused=!0),this.$element.find(".next, .prev").length&&a.support.transition&&(this.$element.trigger(a.support.transition.end),this.cycle(!0)),this.interval=clearInterval(this.interval),this},c.prototype.next=function(){return this.sliding?void 0:this.slide("next")},c.prototype.prev=function(){return this.sliding?void 0:this.slide("prev")},c.prototype.slide=function(b,d){var e=this.$element.find(".item.active"),f=d||this.getItemForDirection(b,e),g=this.interval,h="next"==b?"left":"right",i=this;if(f.hasClass("active"))return this.sliding=!1;var j=f[0],k=a.Event("slide.bs.carousel",{relatedTarget:j,direction:h});if(this.$element.trigger(k),!k.isDefaultPrevented()){if(this.sliding=!0,g&&this.pause(),this.$indicators.length){this.$indicators.find(".active").removeClass("active");var l=a(this.$indicators.children()[this.getItemIndex(f)]);l&&l.addClass("active")}var m=a.Event("slid.bs.carousel",{relatedTarget:j,direction:h});return a.support.transition&&this.$element.hasClass("slide")?(f.addClass(b),f[0].offsetWidth,e.addClass(h),f.addClass(h),e.one("bsTransitionEnd",function(){f.removeClass([b,h].join(" ")).addClass("active"),e.removeClass(["active",h].join(" ")),i.sliding=!1,setTimeout(function(){i.$element.trigger(m)},0)}).emulateTransitionEnd(c.TRANSITION_DURATION)):(e.removeClass("active"),f.addClass("active"),this.sliding=!1,this.$element.trigger(m)),g&&this.cycle(),this}};var d=a.fn.carousel;a.fn.carousel=b,a.fn.carousel.Constructor=c,a.fn.carousel.noConflict=function(){return a.fn.carousel=d,this};var e=function(c){var d,e=a(this),f=a(e.attr("data-target")||(d=e.attr("href"))&&d.replace(/.*(?=#[^\s]+$)/,""));if(f.hasClass("carousel")){var g=a.extend({},f.data(),e.data()),h=e.attr("data-slide-to");h&&(g.interval=!1),b.call(f,g),h&&f.data("bs.carousel").to(h),c.preventDefault()}};a(document).on("click.bs.carousel.data-api","[data-slide]",e).on("click.bs.carousel.data-api","[data-slide-to]",e),a(window).on("load",function(){a('[data-ride="carousel"]').each(function(){var c=a(this);b.call(c,c.data())})})}(jQuery),+function(a){"use strict";function b(b){var c,d=b.attr("data-target")||(c=b.attr("href"))&&c.replace(/.*(?=#[^\s]+$)/,"");return a(d)}function c(b){return this.each(function(){var c=a(this),e=c.data("bs.collapse"),f=a.extend({},d.DEFAULTS,c.data(),"object"==typeof b&&b);!e&&f.toggle&&/show|hide/.test(b)&&(f.toggle=!1),e||c.data("bs.collapse",e=new d(this,f)),"string"==typeof b&&e[b]()})}var d=function(b,c){this.$element=a(b),this.options=a.extend({},d.DEFAULTS,c),this.$trigger=a('[data-toggle="collapse"][href="#'+b.id+'"],[data-toggle="collapse"][data-target="#'+b.id+'"]'),this.transitioning=null,this.options.parent?this.$parent=this.getParent():this.addAriaAndCollapsedClass(this.$element,this.$trigger),this.options.toggle&&this.toggle()};d.VERSION="3.3.6",d.TRANSITION_DURATION=350,d.DEFAULTS={toggle:!0},d.prototype.dimension=function(){var a=this.$element.hasClass("width");return a?"width":"height"},d.prototype.show=function(){if(!this.transitioning&&!this.$element.hasClass("in")){var b,e=this.$parent&&this.$parent.children(".panel").children(".in, .collapsing");if(!(e&&e.length&&(b=e.data("bs.collapse"),b&&b.transitioning))){var f=a.Event("show.bs.collapse");if(this.$element.trigger(f),!f.isDefaultPrevented()){e&&e.length&&(c.call(e,"hide"),b||e.data("bs.collapse",null));var g=this.dimension();this.$element.removeClass("collapse").addClass("collapsing")[g](0).attr("aria-expanded",!0),this.$trigger.removeClass("collapsed").attr("aria-expanded",!0),this.transitioning=1;var h=function(){this.$element.removeClass("collapsing").addClass("collapse in")[g](""),this.transitioning=0,this.$element.trigger("shown.bs.collapse")};if(!a.support.transition)return h.call(this);var i=a.camelCase(["scroll",g].join("-"));this.$element.one("bsTransitionEnd",a.proxy(h,this)).emulateTransitionEnd(d.TRANSITION_DURATION)[g](this.$element[0][i])}}}},d.prototype.hide=function(){if(!this.transitioning&&this.$element.hasClass("in")){var b=a.Event("hide.bs.collapse");if(this.$element.trigger(b),!b.isDefaultPrevented()){var c=this.dimension();this.$element[c](this.$element[c]())[0].offsetHeight,this.$element.addClass("collapsing").removeClass("collapse in").attr("aria-expanded",!1),this.$trigger.addClass("collapsed").attr("aria-expanded",!1),this.transitioning=1;var e=function(){this.transitioning=0,this.$element.removeClass("collapsing").addClass("collapse").trigger("hidden.bs.collapse")};return a.support.transition?void this.$element[c](0).one("bsTransitionEnd",a.proxy(e,this)).emulateTransitionEnd(d.TRANSITION_DURATION):e.call(this)}}},d.prototype.toggle=function(){this[this.$element.hasClass("in")?"hide":"show"]()},d.prototype.getParent=function(){return a(this.options.parent).find('[data-toggle="collapse"][data-parent="'+this.options.parent+'"]').each(a.proxy(function(c,d){var e=a(d);this.addAriaAndCollapsedClass(b(e),e)},this)).end()},d.prototype.addAriaAndCollapsedClass=function(a,b){var c=a.hasClass("in");a.attr("aria-expanded",c),b.toggleClass("collapsed",!c).attr("aria-expanded",c)};var e=a.fn.collapse;a.fn.collapse=c,a.fn.collapse.Constructor=d,a.fn.collapse.noConflict=function(){return a.fn.collapse=e,this},a(document).on("click.bs.collapse.data-api",'[data-toggle="collapse"]',function(d){var e=a(this);e.attr("data-target")||d.preventDefault();var f=b(e),g=f.data("bs.collapse"),h=g?"toggle":e.data();c.call(f,h)})}(jQuery),+function(a){"use strict";function b(b){var c=b.attr("data-target");c||(c=b.attr("href"),c=c&&/#[A-Za-z]/.test(c)&&c.replace(/.*(?=#[^\s]*$)/,""));var d=c&&a(c);return d&&d.length?d:b.parent()}function c(c){c&&3===c.which||(a(e).remove(),a(f).each(function(){var d=a(this),e=b(d),f={relatedTarget:this};e.hasClass("open")&&(c&&"click"==c.type&&/input|textarea/i.test(c.target.tagName)&&a.contains(e[0],c.target)||(e.trigger(c=a.Event("hide.bs.dropdown",f)),c.isDefaultPrevented()||(d.attr("aria-expanded","false"),e.removeClass("open").trigger(a.Event("hidden.bs.dropdown",f)))))}))}function d(b){return this.each(function(){var c=a(this),d=c.data("bs.dropdown");d||c.data("bs.dropdown",d=new g(this)),"string"==typeof b&&d[b].call(c)})}var e=".dropdown-backdrop",f='[data-toggle="dropdown"]',g=function(b){a(b).on("click.bs.dropdown",this.toggle)};g.VERSION="3.3.6",g.prototype.toggle=function(d){var e=a(this);if(!e.is(".disabled, :disabled")){var f=b(e),g=f.hasClass("open");if(c(),!g){"ontouchstart"in document.documentElement&&!f.closest(".navbar-nav").length&&a(document.createElement("div")).addClass("dropdown-backdrop").insertAfter(a(this)).on("click",c);var h={relatedTarget:this};if(f.trigger(d=a.Event("show.bs.dropdown",h)),d.isDefaultPrevented())return;e.trigger("focus").attr("aria-expanded","true"),f.toggleClass("open").trigger(a.Event("shown.bs.dropdown",h))}return!1}},g.prototype.keydown=function(c){if(/(38|40|27|32)/.test(c.which)&&!/input|textarea/i.test(c.target.tagName)){var d=a(this);if(c.preventDefault(),c.stopPropagation(),!d.is(".disabled, :disabled")){var e=b(d),g=e.hasClass("open");if(!g&&27!=c.which||g&&27==c.which)return 27==c.which&&e.find(f).trigger("focus"),d.trigger("click");var h=" li:not(.disabled):visible a",i=e.find(".dropdown-menu"+h);if(i.length){var j=i.index(c.target);38==c.which&&j>0&&j--,40==c.which&&j<i.length-1&&j++,~j||(j=0),i.eq(j).trigger("focus")}}}};var h=a.fn.dropdown;a.fn.dropdown=d,a.fn.dropdown.Constructor=g,a.fn.dropdown.noConflict=function(){return a.fn.dropdown=h,this},a(document).on("click.bs.dropdown.data-api",c).on("click.bs.dropdown.data-api",".dropdown form",function(a){a.stopPropagation()}).on("click.bs.dropdown.data-api",f,g.prototype.toggle).on("keydown.bs.dropdown.data-api",f,g.prototype.keydown).on("keydown.bs.dropdown.data-api",".dropdown-menu",g.prototype.keydown)}(jQuery),+function(a){"use strict";function b(b,d){return this.each(function(){var e=a(this),f=e.data("bs.modal"),g=a.extend({},c.DEFAULTS,e.data(),"object"==typeof b&&b);f||e.data("bs.modal",f=new c(this,g)),"string"==typeof b?f[b](d):g.show&&f.show(d)})}var c=function(b,c){this.options=c,this.$body=a(document.body),this.$element=a(b),this.$dialog=this.$element.find(".modal-dialog"),this.$backdrop=null,this.isShown=null,this.originalBodyPad=null,this.scrollbarWidth=0,this.ignoreBackdropClick=!1,this.options.remote&&this.$element.find(".modal-content").load(this.options.remote,a.proxy(function(){this.$element.trigger("loaded.bs.modal")},this))};c.VERSION="3.3.6",c.TRANSITION_DURATION=300,c.BACKDROP_TRANSITION_DURATION=150,c.DEFAULTS={backdrop:!0,keyboard:!0,show:!0},c.prototype.toggle=function(a){return this.isShown?this.hide():this.show(a)},c.prototype.show=function(b){var d=this,e=a.Event("show.bs.modal",{relatedTarget:b});this.$element.trigger(e),this.isShown||e.isDefaultPrevented()||(this.isShown=!0,this.checkScrollbar(),this.setScrollbar(),this.$body.addClass("modal-open"),this.escape(),this.resize(),this.$element.on("click.dismiss.bs.modal",'[data-dismiss="modal"]',a.proxy(this.hide,this)),this.$dialog.on("mousedown.dismiss.bs.modal",function(){d.$element.one("mouseup.dismiss.bs.modal",function(b){a(b.target).is(d.$element)&&(d.ignoreBackdropClick=!0)})}),this.backdrop(function(){var e=a.support.transition&&d.$element.hasClass("fade");d.$element.parent().length||d.$element.appendTo(d.$body),d.$element.show().scrollTop(0),d.adjustDialog(),e&&d.$element[0].offsetWidth,d.$element.addClass("in"),d.enforceFocus();var f=a.Event("shown.bs.modal",{relatedTarget:b});e?d.$dialog.one("bsTransitionEnd",function(){d.$element.trigger("focus").trigger(f)}).emulateTransitionEnd(c.TRANSITION_DURATION):d.$element.trigger("focus").trigger(f)}))},c.prototype.hide=function(b){b&&b.preventDefault(),b=a.Event("hide.bs.modal"),this.$element.trigger(b),this.isShown&&!b.isDefaultPrevented()&&(this.isShown=!1,this.escape(),this.resize(),a(document).off("focusin.bs.modal"),this.$element.removeClass("in").off("click.dismiss.bs.modal").off("mouseup.dismiss.bs.modal"),this.$dialog.off("mousedown.dismiss.bs.modal"),a.support.transition&&this.$element.hasClass("fade")?this.$element.one("bsTransitionEnd",a.proxy(this.hideModal,this)).emulateTransitionEnd(c.TRANSITION_DURATION):this.hideModal())},c.prototype.enforceFocus=function(){a(document).off("focusin.bs.modal").on("focusin.bs.modal",a.proxy(function(a){this.$element[0]===a.target||this.$element.has(a.target).length||this.$element.trigger("focus")},this))},c.prototype.escape=function(){this.isShown&&this.options.keyboard?this.$element.on("keydown.dismiss.bs.modal",a.proxy(function(a){27==a.which&&this.hide()},this)):this.isShown||this.$element.off("keydown.dismiss.bs.modal")},c.prototype.resize=function(){this.isShown?a(window).on("resize.bs.modal",a.proxy(this.handleUpdate,this)):a(window).off("resize.bs.modal")},c.prototype.hideModal=function(){var a=this;this.$element.hide(),this.backdrop(function(){a.$body.removeClass("modal-open"),a.resetAdjustments(),a.resetScrollbar(),a.$element.trigger("hidden.bs.modal")})},c.prototype.removeBackdrop=function(){this.$backdrop&&this.$backdrop.remove(),this.$backdrop=null},c.prototype.backdrop=function(b){var d=this,e=this.$element.hasClass("fade")?"fade":"";if(this.isShown&&this.options.backdrop){var f=a.support.transition&&e;if(this.$backdrop=a(document.createElement("div")).addClass("modal-backdrop "+e).appendTo(this.$body),this.$element.on("click.dismiss.bs.modal",a.proxy(function(a){return this.ignoreBackdropClick?void(this.ignoreBackdropClick=!1):void(a.target===a.currentTarget&&("static"==this.options.backdrop?this.$element[0].focus():this.hide()))},this)),f&&this.$backdrop[0].offsetWidth,this.$backdrop.addClass("in"),!b)return;f?this.$backdrop.one("bsTransitionEnd",b).emulateTransitionEnd(c.BACKDROP_TRANSITION_DURATION):b()}else if(!this.isShown&&this.$backdrop){this.$backdrop.removeClass("in");var g=function(){d.removeBackdrop(),b&&b()};a.support.transition&&this.$element.hasClass("fade")?this.$backdrop.one("bsTransitionEnd",g).emulateTransitionEnd(c.BACKDROP_TRANSITION_DURATION):g()}else b&&b()},c.prototype.handleUpdate=function(){this.adjustDialog()},c.prototype.adjustDialog=function(){var a=this.$element[0].scrollHeight>document.documentElement.clientHeight;this.$element.css({paddingLeft:!this.bodyIsOverflowing&&a?this.scrollbarWidth:"",paddingRight:this.bodyIsOverflowing&&!a?this.scrollbarWidth:""})},c.prototype.resetAdjustments=function(){this.$element.css({paddingLeft:"",paddingRight:""})},c.prototype.checkScrollbar=function(){var a=window.innerWidth;if(!a){var b=document.documentElement.getBoundingClientRect();a=b.right-Math.abs(b.left)}this.bodyIsOverflowing=document.body.clientWidth<a,this.scrollbarWidth=this.measureScrollbar()},c.prototype.setScrollbar=function(){var a=parseInt(this.$body.css("padding-right")||0,10);this.originalBodyPad=document.body.style.paddingRight||"",this.bodyIsOverflowing&&this.$body.css("padding-right",a+this.scrollbarWidth)},c.prototype.resetScrollbar=function(){this.$body.css("padding-right",this.originalBodyPad)},c.prototype.measureScrollbar=function(){var a=document.createElement("div");a.className="modal-scrollbar-measure",this.$body.append(a);var b=a.offsetWidth-a.clientWidth;return this.$body[0].removeChild(a),b};var d=a.fn.modal;a.fn.modal=b,a.fn.modal.Constructor=c,a.fn.modal.noConflict=function(){return a.fn.modal=d,this},a(document).on("click.bs.modal.data-api",'[data-toggle="modal"]',function(c){var d=a(this),e=d.attr("href"),f=a(d.attr("data-target")||e&&e.replace(/.*(?=#[^\s]+$)/,"")),g=f.data("bs.modal")?"toggle":a.extend({remote:!/#/.test(e)&&e},f.data(),d.data());d.is("a")&&c.preventDefault(),f.one("show.bs.modal",function(a){a.isDefaultPrevented()||f.one("hidden.bs.modal",function(){d.is(":visible")&&d.trigger("focus")})}),b.call(f,g,this)})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.tooltip"),f="object"==typeof b&&b;(e||!/destroy|hide/.test(b))&&(e||d.data("bs.tooltip",e=new c(this,f)),"string"==typeof b&&e[b]())})}var c=function(a,b){this.type=null,this.options=null,this.enabled=null,this.timeout=null,this.hoverState=null,this.$element=null,this.inState=null,this.init("tooltip",a,b)};c.VERSION="3.3.6",c.TRANSITION_DURATION=150,c.DEFAULTS={animation:!0,placement:"top",selector:!1,template:'<div class="tooltip" role="tooltip"><div class="tooltip-arrow"></div><div class="tooltip-inner"></div></div>',trigger:"hover focus",title:"",delay:0,html:!1,container:!1,viewport:{selector:"body",padding:0}},c.prototype.init=function(b,c,d){if(this.enabled=!0,this.type=b,this.$element=a(c),this.options=this.getOptions(d),this.$viewport=this.options.viewport&&a(a.isFunction(this.options.viewport)?this.options.viewport.call(this,this.$element):this.options.viewport.selector||this.options.viewport),this.inState={click:!1,hover:!1,focus:!1},this.$element[0]instanceof document.constructor&&!this.options.selector)throw new Error("`selector` option must be specified when initializing "+this.type+" on the window.document object!");for(var e=this.options.trigger.split(" "),f=e.length;f--;){var g=e[f];if("click"==g)this.$element.on("click."+this.type,this.options.selector,a.proxy(this.toggle,this));else if("manual"!=g){var h="hover"==g?"mouseenter":"focusin",i="hover"==g?"mouseleave":"focusout";this.$element.on(h+"."+this.type,this.options.selector,a.proxy(this.enter,this)),this.$element.on(i+"."+this.type,this.options.selector,a.proxy(this.leave,this))}}this.options.selector?this._options=a.extend({},this.options,{trigger:"manual",selector:""}):this.fixTitle()},c.prototype.getDefaults=function(){return c.DEFAULTS},c.prototype.getOptions=function(b){return b=a.extend({},this.getDefaults(),this.$element.data(),b),b.delay&&"number"==typeof b.delay&&(b.delay={show:b.delay,hide:b.delay}),b},c.prototype.getDelegateOptions=function(){var b={},c=this.getDefaults();return this._options&&a.each(this._options,function(a,d){c[a]!=d&&(b[a]=d)}),b},c.prototype.enter=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget).data("bs."+this.type);return c||(c=new this.constructor(b.currentTarget,this.getDelegateOptions()),a(b.currentTarget).data("bs."+this.type,c)),b instanceof a.Event&&(c.inState["focusin"==b.type?"focus":"hover"]=!0),c.tip().hasClass("in")||"in"==c.hoverState?void(c.hoverState="in"):(clearTimeout(c.timeout),c.hoverState="in",c.options.delay&&c.options.delay.show?void(c.timeout=setTimeout(function(){"in"==c.hoverState&&c.show()},c.options.delay.show)):c.show())},c.prototype.isInStateTrue=function(){for(var a in this.inState)if(this.inState[a])return!0;return!1},c.prototype.leave=function(b){var c=b instanceof this.constructor?b:a(b.currentTarget).data("bs."+this.type);return c||(c=new this.constructor(b.currentTarget,this.getDelegateOptions()),a(b.currentTarget).data("bs."+this.type,c)),b instanceof a.Event&&(c.inState["focusout"==b.type?"focus":"hover"]=!1),c.isInStateTrue()?void 0:(clearTimeout(c.timeout),c.hoverState="out",c.options.delay&&c.options.delay.hide?void(c.timeout=setTimeout(function(){"out"==c.hoverState&&c.hide()},c.options.delay.hide)):c.hide())},c.prototype.show=function(){var b=a.Event("show.bs."+this.type);if(this.hasContent()&&this.enabled){this.$element.trigger(b);var d=a.contains(this.$element[0].ownerDocument.documentElement,this.$element[0]);if(b.isDefaultPrevented()||!d)return;var e=this,f=this.tip(),g=this.getUID(this.type);this.setContent(),f.attr("id",g),this.$element.attr("aria-describedby",g),this.options.animation&&f.addClass("fade");var h="function"==typeof this.options.placement?this.options.placement.call(this,f[0],this.$element[0]):this.options.placement,i=/\s?auto?\s?/i,j=i.test(h);j&&(h=h.replace(i,"")||"top"),f.detach().css({top:0,left:0,display:"block"}).addClass(h).data("bs."+this.type,this),this.options.container?f.appendTo(this.options.container):f.insertAfter(this.$element),this.$element.trigger("inserted.bs."+this.type);var k=this.getPosition(),l=f[0].offsetWidth,m=f[0].offsetHeight;if(j){var n=h,o=this.getPosition(this.$viewport);h="bottom"==h&&k.bottom+m>o.bottom?"top":"top"==h&&k.top-m<o.top?"bottom":"right"==h&&k.right+l>o.width?"left":"left"==h&&k.left-l<o.left?"right":h,f.removeClass(n).addClass(h)}var p=this.getCalculatedOffset(h,k,l,m);this.applyPlacement(p,h);var q=function(){var a=e.hoverState;e.$element.trigger("shown.bs."+e.type),e.hoverState=null,"out"==a&&e.leave(e)};a.support.transition&&this.$tip.hasClass("fade")?f.one("bsTransitionEnd",q).emulateTransitionEnd(c.TRANSITION_DURATION):q()}},c.prototype.applyPlacement=function(b,c){var d=this.tip(),e=d[0].offsetWidth,f=d[0].offsetHeight,g=parseInt(d.css("margin-top"),10),h=parseInt(d.css("margin-left"),10);isNaN(g)&&(g=0),isNaN(h)&&(h=0),b.top+=g,b.left+=h,a.offset.setOffset(d[0],a.extend({using:function(a){d.css({top:Math.round(a.top),left:Math.round(a.left)})}},b),0),d.addClass("in");var i=d[0].offsetWidth,j=d[0].offsetHeight;"top"==c&&j!=f&&(b.top=b.top+f-j);var k=this.getViewportAdjustedDelta(c,b,i,j);k.left?b.left+=k.left:b.top+=k.top;var l=/top|bottom/.test(c),m=l?2*k.left-e+i:2*k.top-f+j,n=l?"offsetWidth":"offsetHeight";d.offset(b),this.replaceArrow(m,d[0][n],l)},c.prototype.replaceArrow=function(a,b,c){this.arrow().css(c?"left":"top",50*(1-a/b)+"%").css(c?"top":"left","")},c.prototype.setContent=function(){var a=this.tip(),b=this.getTitle();a.find(".tooltip-inner")[this.options.html?"html":"text"](b),a.removeClass("fade in top bottom left right")},c.prototype.hide=function(b){function d(){"in"!=e.hoverState&&f.detach(),e.$element.removeAttr("aria-describedby").trigger("hidden.bs."+e.type),b&&b()}var e=this,f=a(this.$tip),g=a.Event("hide.bs."+this.type);return this.$element.trigger(g),g.isDefaultPrevented()?void 0:(f.removeClass("in"),a.support.transition&&f.hasClass("fade")?f.one("bsTransitionEnd",d).emulateTransitionEnd(c.TRANSITION_DURATION):d(),this.hoverState=null,this)},c.prototype.fixTitle=function(){var a=this.$element;(a.attr("title")||"string"!=typeof a.attr("data-original-title"))&&a.attr("data-original-title",a.attr("title")||"").attr("title","")},c.prototype.hasContent=function(){return this.getTitle()},c.prototype.getPosition=function(b){b=b||this.$element;var c=b[0],d="BODY"==c.tagName,e=c.getBoundingClientRect();null==e.width&&(e=a.extend({},e,{width:e.right-e.left,height:e.bottom-e.top}));var f=d?{top:0,left:0}:b.offset(),g={scroll:d?document.documentElement.scrollTop||document.body.scrollTop:b.scrollTop()},h=d?{width:a(window).width(),height:a(window).height()}:null;return a.extend({},e,g,h,f)},c.prototype.getCalculatedOffset=function(a,b,c,d){return"bottom"==a?{top:b.top+b.height,left:b.left+b.width/2-c/2}:"top"==a?{top:b.top-d,left:b.left+b.width/2-c/2}:"left"==a?{top:b.top+b.height/2-d/2,left:b.left-c}:{top:b.top+b.height/2-d/2,left:b.left+b.width}},c.prototype.getViewportAdjustedDelta=function(a,b,c,d){var e={top:0,left:0};if(!this.$viewport)return e;var f=this.options.viewport&&this.options.viewport.padding||0,g=this.getPosition(this.$viewport);if(/right|left/.test(a)){var h=b.top-f-g.scroll,i=b.top+f-g.scroll+d;h<g.top?e.top=g.top-h:i>g.top+g.height&&(e.top=g.top+g.height-i)}else{var j=b.left-f,k=b.left+f+c;j<g.left?e.left=g.left-j:k>g.right&&(e.left=g.left+g.width-k)}return e},c.prototype.getTitle=function(){var a,b=this.$element,c=this.options;return a=b.attr("data-original-title")||("function"==typeof c.title?c.title.call(b[0]):c.title)},c.prototype.getUID=function(a){do a+=~~(1e6*Math.random());while(document.getElementById(a));return a},c.prototype.tip=function(){if(!this.$tip&&(this.$tip=a(this.options.template),1!=this.$tip.length))throw new Error(this.type+" `template` option must consist of exactly 1 top-level element!");return this.$tip},c.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".tooltip-arrow")},c.prototype.enable=function(){this.enabled=!0},c.prototype.disable=function(){this.enabled=!1},c.prototype.toggleEnabled=function(){this.enabled=!this.enabled},c.prototype.toggle=function(b){var c=this;b&&(c=a(b.currentTarget).data("bs."+this.type),c||(c=new this.constructor(b.currentTarget,this.getDelegateOptions()),a(b.currentTarget).data("bs."+this.type,c))),b?(c.inState.click=!c.inState.click,c.isInStateTrue()?c.enter(c):c.leave(c)):c.tip().hasClass("in")?c.leave(c):c.enter(c)},c.prototype.destroy=function(){var a=this;clearTimeout(this.timeout),this.hide(function(){a.$element.off("."+a.type).removeData("bs."+a.type),a.$tip&&a.$tip.detach(),a.$tip=null,a.$arrow=null,a.$viewport=null})};var d=a.fn.tooltip;a.fn.tooltip=b,a.fn.tooltip.Constructor=c,a.fn.tooltip.noConflict=function(){return a.fn.tooltip=d,this}}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.popover"),f="object"==typeof b&&b;(e||!/destroy|hide/.test(b))&&(e||d.data("bs.popover",e=new c(this,f)),"string"==typeof b&&e[b]())})}var c=function(a,b){this.init("popover",a,b)};if(!a.fn.tooltip)throw new Error("Popover requires tooltip.js");c.VERSION="3.3.6",c.DEFAULTS=a.extend({},a.fn.tooltip.Constructor.DEFAULTS,{placement:"right",trigger:"click",content:"",template:'<div class="popover" role="tooltip"><div class="arrow"></div><h3 class="popover-title"></h3><div class="popover-content"></div></div>'}),c.prototype=a.extend({},a.fn.tooltip.Constructor.prototype),c.prototype.constructor=c,c.prototype.getDefaults=function(){return c.DEFAULTS},c.prototype.setContent=function(){var a=this.tip(),b=this.getTitle(),c=this.getContent();a.find(".popover-title")[this.options.html?"html":"text"](b),a.find(".popover-content").children().detach().end()[this.options.html?"string"==typeof c?"html":"append":"text"](c),a.removeClass("fade top bottom left right in"),a.find(".popover-title").html()||a.find(".popover-title").hide()},c.prototype.hasContent=function(){return this.getTitle()||this.getContent()},c.prototype.getContent=function(){var a=this.$element,b=this.options;return a.attr("data-content")||("function"==typeof b.content?b.content.call(a[0]):b.content)},c.prototype.arrow=function(){return this.$arrow=this.$arrow||this.tip().find(".arrow")};var d=a.fn.popover;a.fn.popover=b,a.fn.popover.Constructor=c,a.fn.popover.noConflict=function(){return a.fn.popover=d,this}}(jQuery),+function(a){"use strict";function b(c,d){this.$body=a(document.body),this.$scrollElement=a(a(c).is(document.body)?window:c),this.options=a.extend({},b.DEFAULTS,d),this.selector=(this.options.target||"")+" .nav li > a",this.offsets=[],this.targets=[],this.activeTarget=null,this.scrollHeight=0,this.$scrollElement.on("scroll.bs.scrollspy",a.proxy(this.process,this)),this.refresh(),this.process()}function c(c){return this.each(function(){var d=a(this),e=d.data("bs.scrollspy"),f="object"==typeof c&&c;e||d.data("bs.scrollspy",e=new b(this,f)),"string"==typeof c&&e[c]()})}b.VERSION="3.3.6",b.DEFAULTS={offset:10},b.prototype.getScrollHeight=function(){return this.$scrollElement[0].scrollHeight||Math.max(this.$body[0].scrollHeight,document.documentElement.scrollHeight)},b.prototype.refresh=function(){var b=this,c="offset",d=0;this.offsets=[],this.targets=[],this.scrollHeight=this.getScrollHeight(),a.isWindow(this.$scrollElement[0])||(c="position",d=this.$scrollElement.scrollTop()),this.$body.find(this.selector).map(function(){var b=a(this),e=b.data("target")||b.attr("href"),f=/^#./.test(e)&&a(e);return f&&f.length&&f.is(":visible")&&[[f[c]().top+d,e]]||null}).sort(function(a,b){return a[0]-b[0]}).each(function(){b.offsets.push(this[0]),b.targets.push(this[1])})},b.prototype.process=function(){var a,b=this.$scrollElement.scrollTop()+this.options.offset,c=this.getScrollHeight(),d=this.options.offset+c-this.$scrollElement.height(),e=this.offsets,f=this.targets,g=this.activeTarget;if(this.scrollHeight!=c&&this.refresh(),b>=d)return g!=(a=f[f.length-1])&&this.activate(a);if(g&&b<e[0])return this.activeTarget=null,this.clear();for(a=e.length;a--;)g!=f[a]&&b>=e[a]&&(void 0===e[a+1]||b<e[a+1])&&this.activate(f[a])},b.prototype.activate=function(b){this.activeTarget=b,this.clear();var c=this.selector+'[data-target="'+b+'"],'+this.selector+'[href="'+b+'"]',d=a(c).parents("li").addClass("active");
-d.parent(".dropdown-menu").length&&(d=d.closest("li.dropdown").addClass("active")),d.trigger("activate.bs.scrollspy")},b.prototype.clear=function(){a(this.selector).parentsUntil(this.options.target,".active").removeClass("active")};var d=a.fn.scrollspy;a.fn.scrollspy=c,a.fn.scrollspy.Constructor=b,a.fn.scrollspy.noConflict=function(){return a.fn.scrollspy=d,this},a(window).on("load.bs.scrollspy.data-api",function(){a('[data-spy="scroll"]').each(function(){var b=a(this);c.call(b,b.data())})})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.tab");e||d.data("bs.tab",e=new c(this)),"string"==typeof b&&e[b]()})}var c=function(b){this.element=a(b)};c.VERSION="3.3.6",c.TRANSITION_DURATION=150,c.prototype.show=function(){var b=this.element,c=b.closest("ul:not(.dropdown-menu)"),d=b.data("target");if(d||(d=b.attr("href"),d=d&&d.replace(/.*(?=#[^\s]*$)/,"")),!b.parent("li").hasClass("active")){var e=c.find(".active:last a"),f=a.Event("hide.bs.tab",{relatedTarget:b[0]}),g=a.Event("show.bs.tab",{relatedTarget:e[0]});if(e.trigger(f),b.trigger(g),!g.isDefaultPrevented()&&!f.isDefaultPrevented()){var h=a(d);this.activate(b.closest("li"),c),this.activate(h,h.parent(),function(){e.trigger({type:"hidden.bs.tab",relatedTarget:b[0]}),b.trigger({type:"shown.bs.tab",relatedTarget:e[0]})})}}},c.prototype.activate=function(b,d,e){function f(){g.removeClass("active").find("> .dropdown-menu > .active").removeClass("active").end().find('[data-toggle="tab"]').attr("aria-expanded",!1),b.addClass("active").find('[data-toggle="tab"]').attr("aria-expanded",!0),h?(b[0].offsetWidth,b.addClass("in")):b.removeClass("fade"),b.parent(".dropdown-menu").length&&b.closest("li.dropdown").addClass("active").end().find('[data-toggle="tab"]').attr("aria-expanded",!0),e&&e()}var g=d.find("> .active"),h=e&&a.support.transition&&(g.length&&g.hasClass("fade")||!!d.find("> .fade").length);g.length&&h?g.one("bsTransitionEnd",f).emulateTransitionEnd(c.TRANSITION_DURATION):f(),g.removeClass("in")};var d=a.fn.tab;a.fn.tab=b,a.fn.tab.Constructor=c,a.fn.tab.noConflict=function(){return a.fn.tab=d,this};var e=function(c){c.preventDefault(),b.call(a(this),"show")};a(document).on("click.bs.tab.data-api",'[data-toggle="tab"]',e).on("click.bs.tab.data-api",'[data-toggle="pill"]',e)}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.affix"),f="object"==typeof b&&b;e||d.data("bs.affix",e=new c(this,f)),"string"==typeof b&&e[b]()})}var c=function(b,d){this.options=a.extend({},c.DEFAULTS,d),this.$target=a(this.options.target).on("scroll.bs.affix.data-api",a.proxy(this.checkPosition,this)).on("click.bs.affix.data-api",a.proxy(this.checkPositionWithEventLoop,this)),this.$element=a(b),this.affixed=null,this.unpin=null,this.pinnedOffset=null,this.checkPosition()};c.VERSION="3.3.6",c.RESET="affix affix-top affix-bottom",c.DEFAULTS={offset:0,target:window},c.prototype.getState=function(a,b,c,d){var e=this.$target.scrollTop(),f=this.$element.offset(),g=this.$target.height();if(null!=c&&"top"==this.affixed)return c>e?"top":!1;if("bottom"==this.affixed)return null!=c?e+this.unpin<=f.top?!1:"bottom":a-d>=e+g?!1:"bottom";var h=null==this.affixed,i=h?e:f.top,j=h?g:b;return null!=c&&c>=e?"top":null!=d&&i+j>=a-d?"bottom":!1},c.prototype.getPinnedOffset=function(){if(this.pinnedOffset)return this.pinnedOffset;this.$element.removeClass(c.RESET).addClass("affix");var a=this.$target.scrollTop(),b=this.$element.offset();return this.pinnedOffset=b.top-a},c.prototype.checkPositionWithEventLoop=function(){setTimeout(a.proxy(this.checkPosition,this),1)},c.prototype.checkPosition=function(){if(this.$element.is(":visible")){var b=this.$element.height(),d=this.options.offset,e=d.top,f=d.bottom,g=Math.max(a(document).height(),a(document.body).height());"object"!=typeof d&&(f=e=d),"function"==typeof e&&(e=d.top(this.$element)),"function"==typeof f&&(f=d.bottom(this.$element));var h=this.getState(g,b,e,f);if(this.affixed!=h){null!=this.unpin&&this.$element.css("top","");var i="affix"+(h?"-"+h:""),j=a.Event(i+".bs.affix");if(this.$element.trigger(j),j.isDefaultPrevented())return;this.affixed=h,this.unpin="bottom"==h?this.getPinnedOffset():null,this.$element.removeClass(c.RESET).addClass(i).trigger(i.replace("affix","affixed")+".bs.affix")}"bottom"==h&&this.$element.offset({top:g-b-f})}};var d=a.fn.affix;a.fn.affix=b,a.fn.affix.Constructor=c,a.fn.affix.noConflict=function(){return a.fn.affix=d,this},a(window).on("load",function(){a('[data-spy="affix"]').each(function(){var c=a(this),d=c.data();d.offset=d.offset||{},null!=d.offsetBottom&&(d.offset.bottom=d.offsetBottom),null!=d.offsetTop&&(d.offset.top=d.offsetTop),b.call(c,d)})})}(jQuery);
\ No newline at end of file
+d.parent(".dropdown-menu").length&&(d=d.closest("li.dropdown").addClass("active")),d.trigger("activate.bs.scrollspy")},b.prototype.clear=function(){a(this.selector).parentsUntil(this.options.target,".active").removeClass("active")};var d=a.fn.scrollspy;a.fn.scrollspy=c,a.fn.scrollspy.Constructor=b,a.fn.scrollspy.noConflict=function(){return a.fn.scrollspy=d,this},a(window).on("load.bs.scrollspy.data-api",function(){a('[data-spy="scroll"]').each(function(){var b=a(this);c.call(b,b.data())})})}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.tab");e||d.data("bs.tab",e=new c(this)),"string"==typeof b&&e[b]()})}var c=function(b){this.element=a(b)};c.VERSION="3.3.6",c.TRANSITION_DURATION=150,c.prototype.show=function(){var b=this.element,c=b.closest("ul:not(.dropdown-menu)"),d=b.data("target");if(d||(d=b.attr("href"),d=d&&d.replace(/.*(?=#[^\s]*$)/,"")),!b.parent("li").hasClass("active")){var e=c.find(".active:last a"),f=a.Event("hide.bs.tab",{relatedTarget:b[0]}),g=a.Event("show.bs.tab",{relatedTarget:e[0]});if(e.trigger(f),b.trigger(g),!g.isDefaultPrevented()&&!f.isDefaultPrevented()){var h=a(d);this.activate(b.closest("li"),c),this.activate(h,h.parent(),function(){e.trigger({type:"hidden.bs.tab",relatedTarget:b[0]}),b.trigger({type:"shown.bs.tab",relatedTarget:e[0]})})}}},c.prototype.activate=function(b,d,e){function f(){g.removeClass("active").find("> .dropdown-menu > .active").removeClass("active").end().find('[data-toggle="tab"]').attr("aria-expanded",!1),b.addClass("active").find('[data-toggle="tab"]').attr("aria-expanded",!0),h?(b[0].offsetWidth,b.addClass("in")):b.removeClass("fade"),b.parent(".dropdown-menu").length&&b.closest("li.dropdown").addClass("active").end().find('[data-toggle="tab"]').attr("aria-expanded",!0),e&&e()}var g=d.find("> .active"),h=e&&a.support.transition&&(g.length&&g.hasClass("fade")||!!d.find("> .fade").length);g.length&&h?g.one("bsTransitionEnd",f).emulateTransitionEnd(c.TRANSITION_DURATION):f(),g.removeClass("in")};var d=a.fn.tab;a.fn.tab=b,a.fn.tab.Constructor=c,a.fn.tab.noConflict=function(){return a.fn.tab=d,this};var e=function(c){c.preventDefault(),b.call(a(this),"show")};a(document).on("click.bs.tab.data-api",'[data-toggle="tab"]',e).on("click.bs.tab.data-api",'[data-toggle="pill"]',e)}(jQuery),+function(a){"use strict";function b(b){return this.each(function(){var d=a(this),e=d.data("bs.affix"),f="object"==typeof b&&b;e||d.data("bs.affix",e=new c(this,f)),"string"==typeof b&&e[b]()})}var c=function(b,d){this.options=a.extend({},c.DEFAULTS,d),this.$target=a(this.options.target).on("scroll.bs.affix.data-api",a.proxy(this.checkPosition,this)).on("click.bs.affix.data-api",a.proxy(this.checkPositionWithEventLoop,this)),this.$element=a(b),this.affixed=null,this.unpin=null,this.pinnedOffset=null,this.checkPosition()};c.VERSION="3.3.6",c.RESET="affix affix-top affix-bottom",c.DEFAULTS={offset:0,target:window},c.prototype.getState=function(a,b,c,d){var e=this.$target.scrollTop(),f=this.$element.offset(),g=this.$target.height();if(null!=c&&"top"==this.affixed)return c>e?"top":!1;if("bottom"==this.affixed)return null!=c?e+this.unpin<=f.top?!1:"bottom":a-d>=e+g?!1:"bottom";var h=null==this.affixed,i=h?e:f.top,j=h?g:b;return null!=c&&c>=e?"top":null!=d&&i+j>=a-d?"bottom":!1},c.prototype.getPinnedOffset=function(){if(this.pinnedOffset)return this.pinnedOffset;this.$element.removeClass(c.RESET).addClass("affix");var a=this.$target.scrollTop(),b=this.$element.offset();return this.pinnedOffset=b.top-a},c.prototype.checkPositionWithEventLoop=function(){setTimeout(a.proxy(this.checkPosition,this),1)},c.prototype.checkPosition=function(){if(this.$element.is(":visible")){var b=this.$element.height(),d=this.options.offset,e=d.top,f=d.bottom,g=Math.max(a(document).height(),a(document.body).height());"object"!=typeof d&&(f=e=d),"function"==typeof e&&(e=d.top(this.$element)),"function"==typeof f&&(f=d.bottom(this.$element));var h=this.getState(g,b,e,f);if(this.affixed!=h){null!=this.unpin&&this.$element.css("top","");var i="affix"+(h?"-"+h:""),j=a.Event(i+".bs.affix");if(this.$element.trigger(j),j.isDefaultPrevented())return;this.affixed=h,this.unpin="bottom"==h?this.getPinnedOffset():null,this.$element.removeClass(c.RESET).addClass(i).trigger(i.replace("affix","affixed")+".bs.affix")}"bottom"==h&&this.$element.offset({top:g-b-f})}};var d=a.fn.affix;a.fn.affix=b,a.fn.affix.Constructor=c,a.fn.affix.noConflict=function(){return a.fn.affix=d,this},a(window).on("load",function(){a('[data-spy="affix"]').each(function(){var c=a(this),d=c.data();d.offset=d.offset||{},null!=d.offsetBottom&&(d.offset.bottom=d.offsetBottom),null!=d.offsetTop&&(d.offset.top=d.offsetTop),b.call(c,d)})})}(jQuery);