You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@distributedlog.apache.org by si...@apache.org on 2017/01/05 00:51:06 UTC

[01/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Repository: incubator-distributedlog
Updated Branches:
  refs/heads/master 7fab246d9 -> b44820b50


http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/auditor/DLAuditor.java
----------------------------------------------------------------------
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();
+        }
+    }
+
+}


[05/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConstants.java
----------------------------------------------------------------------
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();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogManager.java
----------------------------------------------------------------------
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();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/Entry.java
----------------------------------------------------------------------
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;
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/EntryBuffer.java
----------------------------------------------------------------------
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;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/EntryPosition.java
----------------------------------------------------------------------
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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntry.java
----------------------------------------------------------------------
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());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryReader.java
----------------------------------------------------------------------
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";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/EnvelopedEntryWriter.java
----------------------------------------------------------------------
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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/LedgerReadPosition.java
----------------------------------------------------------------------
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);
+        }
+    }
+
+}
+
+

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/LocalDLMEmulator.java
----------------------------------------------------------------------
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);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/LogReader.java
----------------------------------------------------------------------
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;
+}


[22/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryStore.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryWriter.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentRandomAccessEntryReader.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKUtils.java
----------------------------------------------------------------------
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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/BKDLConfig.java
----------------------------------------------------------------------
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));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZKLogStreamMetadataStore.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/metadata/ZkMetadataResolver.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionStateStore.java
----------------------------------------------------------------------
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();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/subscription/ZKSubscriptionsStore.java
----------------------------------------------------------------------
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();
-        }
-    }
-
-}



[35/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogHandler.java
----------------------------------------------------------------------
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));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogReadHandler.java
----------------------------------------------------------------------
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);
-        }
-    }
-}


[19/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLockFactory.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryReader.java
----------------------------------------------------------------------
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();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryStore.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentEntryWriter.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentFilter.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataCache.java
----------------------------------------------------------------------
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());
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentMetadataStore.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentRandomAccessEntryReader.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/LogSegmentWriter.java
----------------------------------------------------------------------
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();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/PerStreamLogSegmentCache.java
----------------------------------------------------------------------
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;
-        }
-    }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/RollingPolicy.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/SizeBasedRollingPolicy.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/TimeBasedRollingPolicy.java
----------------------------------------------------------------------
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;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/logsegment/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLConfig.java
----------------------------------------------------------------------
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;
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DLMetadata.java
----------------------------------------------------------------------
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);
-    }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/DryrunLogSegmentMetadataStoreUpdater.java
----------------------------------------------------------------------
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
-            }
-        };
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadata.java
----------------------------------------------------------------------
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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForReader.java
----------------------------------------------------------------------
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;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataForWriter.java
----------------------------------------------------------------------
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;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogMetadataStore.java
----------------------------------------------------------------------
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);
-}


[14/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/OrderedScheduler.java
----------------------------------------------------------------------
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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitLimiter.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/PermitManager.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/RetryPolicyUtils.java
----------------------------------------------------------------------
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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SafeQueueingFuturePool.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SchedulerUtils.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sequencer.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/SimplePermitLimiter.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Sizable.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/TimeSequencer.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Transaction.java
----------------------------------------------------------------------
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);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Utils.java
----------------------------------------------------------------------
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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/DefaultZKOp.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/LimitedPermitManager.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKOp.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKTransaction.java
----------------------------------------------------------------------
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);
-        }
-    }
-}



[02/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClient.java
----------------------------------------------------------------------
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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/ZooKeeperClientBuilder.java
----------------------------------------------------------------------
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
+        );
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/acl/AccessControlManager.java
----------------------------------------------------------------------
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();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/acl/DefaultAccessControlManager.java
----------------------------------------------------------------------
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() {
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/acl/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/admin/DistributedLogAdmin.java
----------------------------------------------------------------------
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";
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/admin/package-info.java
----------------------------------------------------------------------
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;


[50/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Benchmarker.java
----------------------------------------------------------------------
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);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/DLWriterWorker.java
----------------------------------------------------------------------
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);
-                    }
-                });
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/ReaderWorker.java
----------------------------------------------------------------------
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();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Utils.java
----------------------------------------------------------------------
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;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/Worker.java
----------------------------------------------------------------------
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 {
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/WriterWorker.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/package-info.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AbstractReaderBenchmark.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
----------------------------------------------------------------------
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);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerBatchReader.java
----------------------------------------------------------------------
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;
-        }
-
-    }
-}



[33/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogWriteHandler.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogReader.java
----------------------------------------------------------------------
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
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKSyncLogWriter.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKTransmitPacket.java
----------------------------------------------------------------------
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;
-    }
-
-}


[39/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/TestDistributedLogMultiStreamWriter.java
----------------------------------------------------------------------
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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/ownership/TestOwnershipCache.java
----------------------------------------------------------------------
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));
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockDistributedLogServices.java
----------------------------------------------------------------------
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);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockProxyClientBuilder.java
----------------------------------------------------------------------
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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/MockThriftClient.java
----------------------------------------------------------------------
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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/proxy/TestProxyClientManager.java
----------------------------------------------------------------------
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());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestConsistentHashRoutingService.java
----------------------------------------------------------------------
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);
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestInetNameResolution.java
----------------------------------------------------------------------
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());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRegionsRoutingService.java
----------------------------------------------------------------------
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
+        }
+    }
+
+}


[24/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/BKNamespaceDriver.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogMetadataStore.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentFilters.java
----------------------------------------------------------------------
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;
-        }
-    };
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKLogSegmentMetadataStore.java
----------------------------------------------------------------------
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);
-                }
-            }
-        });
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKMetadataAccessor.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/ZKNamespaceWatcher.java
----------------------------------------------------------------------
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();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControl.java
----------------------------------------------------------------------
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;
-    }
-
-}


[43/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogMultiStreamWriter.java
----------------------------------------------------------------------
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();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/MonitorServiceClient.java
----------------------------------------------------------------------
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();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/monitor/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/OwnershipCache.java
----------------------------------------------------------------------
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;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/ownership/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ClusterClient.java
----------------------------------------------------------------------
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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/HostProvider.java
----------------------------------------------------------------------
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();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClient.java
----------------------------------------------------------------------
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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyClientManager.java
----------------------------------------------------------------------
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();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/ProxyListener.java
----------------------------------------------------------------------
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);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/proxy/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/DefaultRegionResolver.java
----------------------------------------------------------------------
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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/RegionResolver.java
----------------------------------------------------------------------
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);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/resolver/package-info.java
----------------------------------------------------------------------
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;


[48/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Worker.java
----------------------------------------------------------------------
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 {
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/WriterWorker.java
----------------------------------------------------------------------
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);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AbstractReaderBenchmark.java
----------------------------------------------------------------------
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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/AsyncReaderBenchmark.java
----------------------------------------------------------------------
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);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerBatchReader.java
----------------------------------------------------------------------
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;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerReadBenchmark.java
----------------------------------------------------------------------
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);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/LedgerStreamReader.java
----------------------------------------------------------------------
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);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/ReadMode.java
----------------------------------------------------------------------
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
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/StreamBenchmark.java
----------------------------------------------------------------------
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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/SyncReaderBenchmark.java
----------------------------------------------------------------------
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);
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/stream/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/ShiftableRateLimiter.java
----------------------------------------------------------------------
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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/utils/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/resources/findbugsExclude.xml
----------------------------------------------------------------------
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>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/thrift/loadtest.thrift
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/pom.xml
----------------------------------------------------------------------
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>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ClientConfig.java
----------------------------------------------------------------------
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;
-    }
-}


[32/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClient.java
----------------------------------------------------------------------
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");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BookKeeperClientBuilder.java
----------------------------------------------------------------------
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);
-    }
-}


[16/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/DistributedLogTool.java
----------------------------------------------------------------------
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;
-      

<TRUNCATED>


[07/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClient.java
----------------------------------------------------------------------
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");
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BookKeeperClientBuilder.java
----------------------------------------------------------------------
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);
+    }
+}


[34/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKLogSegmentWriter.java
----------------------------------------------------------------------
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);
-    }
-
-}


[21/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncFailureInjector.java
----------------------------------------------------------------------
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);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/AsyncRandomFailureInjector.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/FailureInjector.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/RandomDelayFailureInjector.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/injector/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortable.java
----------------------------------------------------------------------
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;
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/io/Abortables.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncAbortable.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncCloseable.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/io/AsyncDeleteable.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/io/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ChainedRequestLimiter.java
----------------------------------------------------------------------
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));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/ComposableRequestLimiter.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/GuavaRateLimiter.java
----------------------------------------------------------------------
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;
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RateLimiter.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/RequestLimiter.java
----------------------------------------------------------------------
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;
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/limiter/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLock.java
----------------------------------------------------------------------
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;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/DistributedLockContext.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/EpochChangedException.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockAction.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockClosedException.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockListener.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockSessionExpiredException.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockStateChangedException.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockTimeoutException.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/LockWaiter.java
----------------------------------------------------------------------
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;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/NopDistributedLock.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLock.java
----------------------------------------------------------------------
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();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/SessionLockFactory.java
----------------------------------------------------------------------
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);
-
-}


[31/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConfiguration.java
----------------------------------------------------------------------
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
-  

<TRUNCATED>


[13/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKVersionedSetOp.java
----------------------------------------------------------------------
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);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/zk/ZKWatcherManager.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamReader.java
----------------------------------------------------------------------
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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/AppendOnlyStreamWriter.java
----------------------------------------------------------------------
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
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogReader.java
----------------------------------------------------------------------
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);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncLogWriter.java
----------------------------------------------------------------------
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();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/AsyncNotification.java
----------------------------------------------------------------------
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();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKAbstractLogWriter.java
----------------------------------------------------------------------
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;
+    }
+
+}


[36/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogManager.java
----------------------------------------------------------------------
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());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKDistributedLogNamespace.java
----------------------------------------------------------------------
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.");
-    }
-}


[42/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ConsistentHashRoutingService.java
----------------------------------------------------------------------
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);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/NameServerSet.java
----------------------------------------------------------------------
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
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RegionsRoutingService.java
----------------------------------------------------------------------
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);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingService.java
----------------------------------------------------------------------
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);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingServiceProvider.java
----------------------------------------------------------------------
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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/RoutingUtils.java
----------------------------------------------------------------------
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);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetRoutingService.java
----------------------------------------------------------------------
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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/ServerSetWatcher.java
----------------------------------------------------------------------
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;
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/SingleHostRoutingService.java
----------------------------------------------------------------------
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
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TestName.java
----------------------------------------------------------------------
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));
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/TwitterServerSetWatcher.java
----------------------------------------------------------------------
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);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/routing/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/DLZkServerSet.java
----------------------------------------------------------------------
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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/serverset/package-info.java
----------------------------------------------------------------------
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;


[25/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentListener.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/LogSegmentNamesListener.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/NamespaceListener.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/ReadAheadCallback.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/callback/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentBaseConfiguration.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConcurrentConstConfiguration.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationListener.java
----------------------------------------------------------------------
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);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/ConfigurationSubscription.java
----------------------------------------------------------------------
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));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicConfigurationFactory.java
----------------------------------------------------------------------
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()));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/DynamicDistributedLogConfiguration.java
----------------------------------------------------------------------
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));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/FileConfigurationBuilder.java
----------------------------------------------------------------------
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;
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/PropertiesConfigurationBuilder.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/config/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/exceptions/ZKException.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/AbstractFeatureProvider.java
----------------------------------------------------------------------
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
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/ConfigurationFeatureProvider.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/CoreFeatureKeys.java
----------------------------------------------------------------------
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,
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DefaultFeatureProvider.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/DynamicConfigurationFeatureProvider.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/feature/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/function/CloseAsyncCloseableFunction.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/function/DefaultValueMapFunction.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetLastTxIdFunction.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/function/GetVersionedValueFunction.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/function/VoidFunctions.java
----------------------------------------------------------------------
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;
-                }
-            };
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/function/package-info.java
----------------------------------------------------------------------
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;


[15/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/Tool.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/tools/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/Allocator.java
----------------------------------------------------------------------
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);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/CommandLineUtils.java
----------------------------------------------------------------------
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");
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/ConfUtils.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/DLUtils.java
----------------------------------------------------------------------
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");
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FailpointUtils.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/FutureUtils.java
----------------------------------------------------------------------
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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredFuturePool.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/util/MonitoredScheduledThreadPoolExecutor.java
----------------------------------------------------------------------
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);
-    }
-
-}


[45/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetRoutingService.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ServerSetWatcher.java
----------------------------------------------------------------------
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;
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/SingleHostRoutingService.java
----------------------------------------------------------------------
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
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TestName.java
----------------------------------------------------------------------
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));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/TwitterServerSetWatcher.java
----------------------------------------------------------------------
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);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/package-info.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/DLZkServerSet.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/serverset/package-info.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
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});
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/SpeculativeRequestExecutor.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/speculative/package-info.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStats.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/ClientStatsLogger.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStats.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OpStatsLogger.java
----------------------------------------------------------------------
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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/OwnershipStatsLogger.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/stats/package-info.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DLSocketAddress.java
----------------------------------------------------------------------
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();
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClient.java
----------------------------------------------------------------------
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();
-}


[51/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
DL-4: Repackage the source under apache namespace


Project: http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/commit/b44820b5
Tree: http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/tree/b44820b5
Diff: http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/diff/b44820b5

Branch: refs/heads/master
Commit: b44820b50eea9ef2447ec7e136e182ba39e2c269
Parents: 7fab246
Author: Sijie Guo <si...@twitter.com>
Authored: Wed Jan 4 16:44:15 2017 -0800
Committer: Sijie Guo <si...@twitter.com>
Committed: Wed Jan 4 16:46:19 2017 -0800

----------------------------------------------------------------------
 distributedlog-benchmark/conf/log4j.properties  |    8 +-
 .../distributedlog/benchmark/Benchmarker.java   |  468 ---
 .../benchmark/DLWriterWorker.java               |  245 --
 .../distributedlog/benchmark/ReaderWorker.java  |  468 ---
 .../twitter/distributedlog/benchmark/Utils.java |   57 -
 .../distributedlog/benchmark/Worker.java        |   26 -
 .../distributedlog/benchmark/WriterWorker.java  |  387 --
 .../distributedlog/benchmark/package-info.java  |   21 -
 .../stream/AbstractReaderBenchmark.java         |   70 -
 .../benchmark/stream/AsyncReaderBenchmark.java  |  158 -
 .../benchmark/stream/LedgerBatchReader.java     |   82 -
 .../benchmark/stream/LedgerReadBenchmark.java   |  151 -
 .../benchmark/stream/LedgerStreamReader.java    |  131 -
 .../benchmark/stream/ReadMode.java              |   28 -
 .../benchmark/stream/StreamBenchmark.java       |  138 -
 .../benchmark/stream/SyncReaderBenchmark.java   |  164 -
 .../benchmark/stream/package-info.java          |   21 -
 .../benchmark/utils/ShiftableRateLimiter.java   |   71 -
 .../benchmark/utils/package-info.java           |   21 -
 .../distributedlog/benchmark/Benchmarker.java   |  468 +++
 .../benchmark/DLWriterWorker.java               |  245 ++
 .../distributedlog/benchmark/ReaderWorker.java  |  468 +++
 .../apache/distributedlog/benchmark/Utils.java  |   57 +
 .../apache/distributedlog/benchmark/Worker.java |   26 +
 .../distributedlog/benchmark/WriterWorker.java  |  387 ++
 .../distributedlog/benchmark/package-info.java  |   21 +
 .../stream/AbstractReaderBenchmark.java         |   70 +
 .../benchmark/stream/AsyncReaderBenchmark.java  |  158 +
 .../benchmark/stream/LedgerBatchReader.java     |   82 +
 .../benchmark/stream/LedgerReadBenchmark.java   |  151 +
 .../benchmark/stream/LedgerStreamReader.java    |  131 +
 .../benchmark/stream/ReadMode.java              |   28 +
 .../benchmark/stream/StreamBenchmark.java       |  138 +
 .../benchmark/stream/SyncReaderBenchmark.java   |  164 +
 .../benchmark/stream/package-info.java          |   21 +
 .../benchmark/utils/ShiftableRateLimiter.java   |   71 +
 .../benchmark/utils/package-info.java           |   21 +
 .../src/main/resources/findbugsExclude.xml      |    2 +-
 .../src/main/thrift/loadtest.thrift             |    2 +-
 distributedlog-client/pom.xml                   |    2 +-
 .../distributedlog/client/ClientConfig.java     |  187 -
 .../client/DistributedLogClientImpl.java        | 1200 ------
 .../client/DistributedLogMultiStreamWriter.java |  486 ---
 .../client/monitor/MonitorServiceClient.java    |   68 -
 .../client/monitor/package-info.java            |   21 -
 .../client/ownership/OwnershipCache.java        |  235 --
 .../client/ownership/package-info.java          |   21 -
 .../distributedlog/client/package-info.java     |   21 -
 .../client/proxy/ClusterClient.java             |   51 -
 .../client/proxy/HostProvider.java              |   35 -
 .../client/proxy/ProxyClient.java               |  165 -
 .../client/proxy/ProxyClientManager.java        |  362 --
 .../client/proxy/ProxyListener.java             |   50 -
 .../client/proxy/package-info.java              |   21 -
 .../client/resolver/DefaultRegionResolver.java  |   85 -
 .../client/resolver/RegionResolver.java         |   43 -
 .../client/resolver/package-info.java           |   21 -
 .../routing/ConsistentHashRoutingService.java   |  500 ---
 .../client/routing/NameServerSet.java           |  263 --
 .../client/routing/RegionsRoutingService.java   |  192 -
 .../client/routing/RoutingService.java          |  206 -
 .../client/routing/RoutingServiceProvider.java  |   39 -
 .../client/routing/RoutingUtils.java            |   88 -
 .../client/routing/ServerSetRoutingService.java |  274 --
 .../client/routing/ServerSetWatcher.java        |   71 -
 .../routing/SingleHostRoutingService.java       |  128 -
 .../distributedlog/client/routing/TestName.java |   49 -
 .../client/routing/TwitterServerSetWatcher.java |   83 -
 .../client/routing/package-info.java            |   21 -
 .../client/serverset/DLZkServerSet.java         |   91 -
 .../client/serverset/package-info.java          |   21 -
 ...efaultSpeculativeRequestExecutionPolicy.java |  119 -
 .../SpeculativeRequestExecutionPolicy.java      |   34 -
 .../speculative/SpeculativeRequestExecutor.java |   33 -
 .../client/speculative/package-info.java        |   21 -
 .../client/stats/ClientStats.java               |  108 -
 .../client/stats/ClientStatsLogger.java         |   91 -
 .../distributedlog/client/stats/OpStats.java    |   82 -
 .../client/stats/OpStatsLogger.java             |   50 -
 .../client/stats/OwnershipStatsLogger.java      |  115 -
 .../client/stats/package-info.java              |   21 -
 .../distributedlog/service/DLSocketAddress.java |  161 -
 .../service/DistributedLogClient.java           |  108 -
 .../service/DistributedLogClientBuilder.java    |  608 ---
 .../distributedlog/service/package-info.java    |   21 -
 .../distributedlog/client/ClientConfig.java     |  187 +
 .../client/DistributedLogClientImpl.java        | 1200 ++++++
 .../client/DistributedLogMultiStreamWriter.java |  486 +++
 .../client/monitor/MonitorServiceClient.java    |   68 +
 .../client/monitor/package-info.java            |   21 +
 .../client/ownership/OwnershipCache.java        |  235 ++
 .../client/ownership/package-info.java          |   21 +
 .../distributedlog/client/package-info.java     |   21 +
 .../client/proxy/ClusterClient.java             |   51 +
 .../client/proxy/HostProvider.java              |   35 +
 .../client/proxy/ProxyClient.java               |  165 +
 .../client/proxy/ProxyClientManager.java        |  362 ++
 .../client/proxy/ProxyListener.java             |   50 +
 .../client/proxy/package-info.java              |   21 +
 .../client/resolver/DefaultRegionResolver.java  |   85 +
 .../client/resolver/RegionResolver.java         |   43 +
 .../client/resolver/package-info.java           |   21 +
 .../routing/ConsistentHashRoutingService.java   |  500 +++
 .../client/routing/NameServerSet.java           |  263 ++
 .../client/routing/RegionsRoutingService.java   |  192 +
 .../client/routing/RoutingService.java          |  206 +
 .../client/routing/RoutingServiceProvider.java  |   39 +
 .../client/routing/RoutingUtils.java            |   88 +
 .../client/routing/ServerSetRoutingService.java |  274 ++
 .../client/routing/ServerSetWatcher.java        |   71 +
 .../routing/SingleHostRoutingService.java       |  128 +
 .../distributedlog/client/routing/TestName.java |   49 +
 .../client/routing/TwitterServerSetWatcher.java |   83 +
 .../client/routing/package-info.java            |   21 +
 .../client/serverset/DLZkServerSet.java         |   91 +
 .../client/serverset/package-info.java          |   21 +
 ...efaultSpeculativeRequestExecutionPolicy.java |  119 +
 .../SpeculativeRequestExecutionPolicy.java      |   34 +
 .../speculative/SpeculativeRequestExecutor.java |   33 +
 .../client/speculative/package-info.java        |   21 +
 .../client/stats/ClientStats.java               |  108 +
 .../client/stats/ClientStatsLogger.java         |   91 +
 .../distributedlog/client/stats/OpStats.java    |   82 +
 .../client/stats/OpStatsLogger.java             |   50 +
 .../client/stats/OwnershipStatsLogger.java      |  115 +
 .../client/stats/package-info.java              |   21 +
 .../distributedlog/service/DLSocketAddress.java |  161 +
 .../service/DistributedLogClient.java           |  108 +
 .../service/DistributedLogClientBuilder.java    |  608 +++
 .../distributedlog/service/package-info.java    |   21 +
 .../src/main/resources/findbugsExclude.xml      |    2 +-
 .../TestDistributedLogMultiStreamWriter.java    |  383 --
 .../client/ownership/TestOwnershipCache.java    |  207 -
 .../proxy/MockDistributedLogServices.java       |  144 -
 .../client/proxy/MockProxyClientBuilder.java    |   49 -
 .../client/proxy/MockThriftClient.java          |   32 -
 .../client/proxy/TestProxyClientManager.java    |  368 --
 .../TestConsistentHashRoutingService.java       |  417 ---
 .../client/routing/TestInetNameResolution.java  |   73 -
 .../routing/TestRegionsRoutingService.java      |  133 -
 .../client/routing/TestRoutingService.java      |  146 -
 ...efaultSpeculativeRequestExecutionPolicy.java |  105 -
 .../TestDistributedLogClientBuilder.java        |   49 -
 .../TestDistributedLogMultiStreamWriter.java    |  383 ++
 .../client/ownership/TestOwnershipCache.java    |  207 +
 .../proxy/MockDistributedLogServices.java       |  144 +
 .../client/proxy/MockProxyClientBuilder.java    |   49 +
 .../client/proxy/MockThriftClient.java          |   32 +
 .../client/proxy/TestProxyClientManager.java    |  368 ++
 .../TestConsistentHashRoutingService.java       |  417 +++
 .../client/routing/TestInetNameResolution.java  |   73 +
 .../routing/TestRegionsRoutingService.java      |  133 +
 .../client/routing/TestRoutingService.java      |  146 +
 ...efaultSpeculativeRequestExecutionPolicy.java |  105 +
 .../TestDistributedLogClientBuilder.java        |   49 +
 distributedlog-core/conf/log4j.properties       |    8 +-
 .../conf/zookeeper.conf.dynamic.template        |    2 +-
 distributedlog-core/pom.xml                     |    2 +-
 .../distributedlog/AppendOnlyStreamReader.java  |  198 -
 .../distributedlog/AppendOnlyStreamWriter.java  |  107 -
 .../twitter/distributedlog/AsyncLogReader.java  |   69 -
 .../twitter/distributedlog/AsyncLogWriter.java  |   70 -
 .../distributedlog/AsyncNotification.java       |   32 -
 .../distributedlog/BKAbstractLogWriter.java     |  555 ---
 .../distributedlog/BKAsyncLogReader.java        |  751 ----
 .../distributedlog/BKAsyncLogWriter.java        |  559 ---
 .../distributedlog/BKDistributedLogManager.java | 1106 ------
 .../BKDistributedLogNamespace.java              |  320 --
 .../twitter/distributedlog/BKLogHandler.java    |  715 ----
 .../distributedlog/BKLogReadHandler.java        |  431 ---
 .../distributedlog/BKLogSegmentWriter.java      | 1348 -------
 .../distributedlog/BKLogWriteHandler.java       | 1325 -------
 .../twitter/distributedlog/BKSyncLogReader.java |  276 --
 .../twitter/distributedlog/BKSyncLogWriter.java |  113 -
 .../distributedlog/BKTransmitPacket.java        |   90 -
 .../distributedlog/BookKeeperClient.java        |  289 --
 .../distributedlog/BookKeeperClientBuilder.java |  209 --
 .../DistributedLogConfiguration.java            | 3528 ------------------
 .../distributedlog/DistributedLogConstants.java |   69 -
 .../distributedlog/DistributedLogManager.java   |  308 --
 .../java/com/twitter/distributedlog/Entry.java  |  403 --
 .../com/twitter/distributedlog/EntryBuffer.java |   70 -
 .../twitter/distributedlog/EntryPosition.java   |   63 -
 .../twitter/distributedlog/EnvelopedEntry.java  |  296 --
 .../distributedlog/EnvelopedEntryReader.java    |  102 -
 .../distributedlog/EnvelopedEntryWriter.java    |  192 -
 .../distributedlog/LedgerReadPosition.java      |  171 -
 .../distributedlog/LocalDLMEmulator.java        |  364 --
 .../com/twitter/distributedlog/LogReader.java   |  195 -
 .../distributedlog/LogSegmentMetadata.java      | 1125 ------
 .../com/twitter/distributedlog/LogWriter.java   |   78 -
 .../distributedlog/MaxLogSegmentSequenceNo.java |   72 -
 .../com/twitter/distributedlog/MaxTxId.java     |   72 -
 .../distributedlog/MetadataAccessor.java        |   43 -
 .../distributedlog/ReadAheadEntryReader.java    |  992 -----
 .../com/twitter/distributedlog/ReadUtils.java   |  782 ----
 .../twitter/distributedlog/WriteLimiter.java    |   62 -
 .../twitter/distributedlog/ZooKeeperClient.java |  402 --
 .../distributedlog/ZooKeeperClientBuilder.java  |  233 --
 .../acl/AccessControlManager.java               |   74 -
 .../acl/DefaultAccessControlManager.java        |   55 -
 .../distributedlog/acl/package-info.java        |   21 -
 .../admin/DistributedLogAdmin.java              |  921 -----
 .../distributedlog/admin/package-info.java      |   21 -
 .../distributedlog/auditor/DLAuditor.java       |  630 ----
 .../bk/DynamicQuorumConfigProvider.java         |   37 -
 .../bk/ImmutableQuorumConfigProvider.java       |   35 -
 .../distributedlog/bk/LedgerAllocator.java      |   32 -
 .../bk/LedgerAllocatorDelegator.java            |   83 -
 .../distributedlog/bk/LedgerAllocatorPool.java  |  458 ---
 .../distributedlog/bk/LedgerAllocatorUtils.java |   54 -
 .../twitter/distributedlog/bk/QuorumConfig.java |   92 -
 .../distributedlog/bk/QuorumConfigProvider.java |   32 -
 .../bk/SimpleLedgerAllocator.java               |  536 ---
 .../twitter/distributedlog/bk/package-info.java |   24 -
 .../callback/LogSegmentListener.java            |   42 -
 .../callback/LogSegmentNamesListener.java       |   42 -
 .../callback/NamespaceListener.java             |   34 -
 .../callback/ReadAheadCallback.java             |   25 -
 .../distributedlog/callback/package-info.java   |   21 -
 .../config/ConcurrentBaseConfiguration.java     |   76 -
 .../config/ConcurrentConstConfiguration.java    |   31 -
 .../config/ConfigurationListener.java           |   32 -
 .../config/ConfigurationSubscription.java       |  186 -
 .../config/DynamicConfigurationFactory.java     |   91 -
 .../DynamicDistributedLogConfiguration.java     |  356 --
 .../config/FileConfigurationBuilder.java        |   28 -
 .../config/PropertiesConfigurationBuilder.java  |   40 -
 .../distributedlog/config/package-info.java     |   21 -
 .../distributedlog/exceptions/ZKException.java  |   54 -
 .../feature/AbstractFeatureProvider.java        |  103 -
 .../feature/ConfigurationFeatureProvider.java   |   76 -
 .../distributedlog/feature/CoreFeatureKeys.java |   29 -
 .../feature/DefaultFeatureProvider.java         |   47 -
 .../DynamicConfigurationFeatureProvider.java    |  132 -
 .../distributedlog/feature/package-info.java    |   21 -
 .../function/CloseAsyncCloseableFunction.java   |   51 -
 .../function/DefaultValueMapFunction.java       |   41 -
 .../function/GetLastTxIdFunction.java           |   43 -
 .../function/GetVersionedValueFunction.java     |   39 -
 .../distributedlog/function/VoidFunctions.java  |   34 -
 .../distributedlog/function/package-info.java   |   21 -
 .../distributedlog/impl/BKNamespaceDriver.java  |  631 ----
 .../distributedlog/impl/ZKLogMetadataStore.java |  124 -
 .../impl/ZKLogSegmentFilters.java               |   89 -
 .../impl/ZKLogSegmentMetadataStore.java         |  503 ---
 .../distributedlog/impl/ZKMetadataAccessor.java |  264 --
 .../distributedlog/impl/ZKNamespaceWatcher.java |  133 -
 .../impl/acl/ZKAccessControl.java               |  232 --
 .../impl/acl/ZKAccessControlManager.java        |  374 --
 .../federated/FederatedZKLogMetadataStore.java  |  760 ----
 .../impl/logsegment/BKLogSegmentAllocator.java  |   85 -
 .../logsegment/BKLogSegmentEntryReader.java     |  837 -----
 .../impl/logsegment/BKLogSegmentEntryStore.java |  286 --
 .../logsegment/BKLogSegmentEntryWriter.java     |   61 -
 .../BKLogSegmentRandomAccessEntryReader.java    |  119 -
 .../distributedlog/impl/logsegment/BKUtils.java |   72 -
 .../impl/metadata/BKDLConfig.java               |  400 --
 .../impl/metadata/ZKLogStreamMetadataStore.java |  633 ----
 .../impl/metadata/ZkMetadataResolver.java       |   72 -
 .../distributedlog/impl/package-info.java       |   21 -
 .../subscription/ZKSubscriptionStateStore.java  |  121 -
 .../impl/subscription/ZKSubscriptionsStore.java |  164 -
 .../injector/AsyncFailureInjector.java          |  131 -
 .../injector/AsyncRandomFailureInjector.java    |  186 -
 .../injector/FailureInjector.java               |   37 -
 .../injector/RandomDelayFailureInjector.java    |   60 -
 .../distributedlog/injector/package-info.java   |   21 -
 .../twitter/distributedlog/io/Abortable.java    |   41 -
 .../twitter/distributedlog/io/Abortables.java   |  183 -
 .../distributedlog/io/AsyncAbortable.java       |   57 -
 .../distributedlog/io/AsyncCloseable.java       |   60 -
 .../distributedlog/io/AsyncDeleteable.java      |   34 -
 .../twitter/distributedlog/io/package-info.java |   21 -
 .../limiter/ChainedRequestLimiter.java          |   76 -
 .../limiter/ComposableRequestLimiter.java       |   73 -
 .../limiter/GuavaRateLimiter.java               |   58 -
 .../distributedlog/limiter/RateLimiter.java     |   49 -
 .../distributedlog/limiter/RequestLimiter.java  |   24 -
 .../distributedlog/limiter/package-info.java    |   21 -
 .../distributedlog/lock/DistributedLock.java    |   54 -
 .../lock/DistributedLockContext.java            |   43 -
 .../lock/EpochChangedException.java             |   33 -
 .../twitter/distributedlog/lock/LockAction.java |   36 -
 .../lock/LockClosedException.java               |   38 -
 .../distributedlog/lock/LockListener.java       |   28 -
 .../lock/LockSessionExpiredException.java       |   34 -
 .../lock/LockStateChangedException.java         |   36 -
 .../lock/LockTimeoutException.java              |   34 -
 .../twitter/distributedlog/lock/LockWaiter.java |   95 -
 .../distributedlog/lock/NopDistributedLock.java |   51 -
 .../distributedlog/lock/SessionLock.java        |  126 -
 .../distributedlog/lock/SessionLockFactory.java |   38 -
 .../distributedlog/lock/ZKDistributedLock.java  |  537 ---
 .../distributedlog/lock/ZKSessionLock.java      | 1363 -------
 .../lock/ZKSessionLockFactory.java              |  133 -
 .../distributedlog/lock/package-info.java       |   21 -
 .../logsegment/LogSegmentEntryReader.java       |  113 -
 .../logsegment/LogSegmentEntryStore.java        |   73 -
 .../logsegment/LogSegmentEntryWriter.java       |   72 -
 .../logsegment/LogSegmentFilter.java            |   42 -
 .../logsegment/LogSegmentMetadataCache.java     |   98 -
 .../logsegment/LogSegmentMetadataStore.java     |  163 -
 .../LogSegmentRandomAccessEntryReader.java      |   47 -
 .../logsegment/LogSegmentWriter.java            |   87 -
 .../logsegment/PerStreamLogSegmentCache.java    |  243 --
 .../logsegment/RollingPolicy.java               |   33 -
 .../logsegment/SizeBasedRollingPolicy.java      |   34 -
 .../logsegment/TimeBasedRollingPolicy.java      |   46 -
 .../distributedlog/logsegment/package-info.java |   21 -
 .../distributedlog/metadata/DLConfig.java       |   39 -
 .../distributedlog/metadata/DLMetadata.java     |  227 --
 .../DryrunLogSegmentMetadataStoreUpdater.java   |   51 -
 .../distributedlog/metadata/LogMetadata.java    |  175 -
 .../metadata/LogMetadataForReader.java          |  103 -
 .../metadata/LogMetadataForWriter.java          |   64 -
 .../metadata/LogMetadataStore.java              |   66 -
 .../LogSegmentMetadataStoreUpdater.java         |  184 -
 .../metadata/LogStreamMetadataStore.java        |  114 -
 .../metadata/MetadataResolver.java              |   51 -
 .../metadata/MetadataUpdater.java               |  120 -
 .../distributedlog/metadata/package-info.java   |   21 -
 .../namespace/DistributedLogNamespace.java      |  190 -
 .../DistributedLogNamespaceBuilder.java         |  278 --
 .../namespace/NamespaceDriver.java              |  138 -
 .../namespace/NamespaceDriverManager.java       |  180 -
 .../namespace/NamespaceWatcher.java             |   61 -
 .../distributedlog/namespace/package-info.java  |   21 -
 .../twitter/distributedlog/net/DNSResolver.java |  113 -
 .../distributedlog/net/DNSResolverForRacks.java |   66 -
 .../distributedlog/net/DNSResolverForRows.java  |   72 -
 .../twitter/distributedlog/net/NetUtils.java    |   74 -
 .../distributedlog/net/package-info.java        |   33 -
 .../twitter/distributedlog/package-info.java    |   21 -
 .../distributedlog/rate/MovingAverageRate.java  |   24 -
 .../rate/MovingAverageRateFactory.java          |   65 -
 .../rate/SampledMovingAverageRate.java          |   58 -
 .../distributedlog/readahead/package-info.java  |   21 -
 .../selector/FirstDLSNNotLessThanSelector.java  |   46 -
 .../selector/FirstRecordSelector.java           |   46 -
 .../selector/FirstTxIdNotLessThanSelector.java  |   51 -
 .../selector/LastRecordSelector.java            |   38 -
 .../selector/LogRecordSelector.java             |   40 -
 .../stats/BKExceptionStatsLogger.java           |  109 -
 .../stats/BroadCastStatsLogger.java             |  194 -
 .../distributedlog/stats/OpStatsListener.java   |   51 -
 .../subscription/SubscriptionStateStore.java    |   42 -
 .../subscription/SubscriptionsStore.java        |   69 -
 .../tools/DistributedLogTool.java               | 2873 --------------
 .../com/twitter/distributedlog/tools/Tool.java  |  243 --
 .../distributedlog/tools/package-info.java      |   21 -
 .../twitter/distributedlog/util/Allocator.java  |  102 -
 .../distributedlog/util/CommandLineUtils.java   |   56 -
 .../twitter/distributedlog/util/ConfUtils.java  |   65 -
 .../twitter/distributedlog/util/DLUtils.java    |  321 --
 .../distributedlog/util/FailpointUtils.java     |  134 -
 .../distributedlog/util/FutureUtils.java        |  534 ---
 .../util/MonitoredFuturePool.java               |  131 -
 .../MonitoredScheduledThreadPoolExecutor.java   |  257 --
 .../distributedlog/util/OrderedScheduler.java   |  490 ---
 .../distributedlog/util/PermitLimiter.java      |   57 -
 .../distributedlog/util/PermitManager.java      |   93 -
 .../distributedlog/util/RetryPolicyUtils.java   |   45 -
 .../util/SafeQueueingFuturePool.java            |  115 -
 .../distributedlog/util/SchedulerUtils.java     |   56 -
 .../twitter/distributedlog/util/Sequencer.java  |   31 -
 .../util/SimplePermitLimiter.java               |  117 -
 .../twitter/distributedlog/util/Sizable.java    |   31 -
 .../distributedlog/util/TimeSequencer.java      |   39 -
 .../distributedlog/util/Transaction.java        |   97 -
 .../com/twitter/distributedlog/util/Utils.java  |  607 ---
 .../distributedlog/util/package-info.java       |   21 -
 .../twitter/distributedlog/zk/DefaultZKOp.java  |   55 -
 .../distributedlog/zk/LimitedPermitManager.java |  195 -
 .../com/twitter/distributedlog/zk/ZKOp.java     |   63 -
 .../distributedlog/zk/ZKTransaction.java        |  103 -
 .../distributedlog/zk/ZKVersionedSetOp.java     |   71 -
 .../distributedlog/zk/ZKWatcherManager.java     |  239 --
 .../distributedlog/AppendOnlyStreamReader.java  |  198 +
 .../distributedlog/AppendOnlyStreamWriter.java  |  107 +
 .../apache/distributedlog/AsyncLogReader.java   |   69 +
 .../apache/distributedlog/AsyncLogWriter.java   |   70 +
 .../distributedlog/AsyncNotification.java       |   32 +
 .../distributedlog/BKAbstractLogWriter.java     |  555 +++
 .../apache/distributedlog/BKAsyncLogReader.java |  751 ++++
 .../apache/distributedlog/BKAsyncLogWriter.java |  559 +++
 .../distributedlog/BKDistributedLogManager.java | 1106 ++++++
 .../BKDistributedLogNamespace.java              |  320 ++
 .../org/apache/distributedlog/BKLogHandler.java |  715 ++++
 .../apache/distributedlog/BKLogReadHandler.java |  431 +++
 .../distributedlog/BKLogSegmentWriter.java      | 1348 +++++++
 .../distributedlog/BKLogWriteHandler.java       | 1325 +++++++
 .../apache/distributedlog/BKSyncLogReader.java  |  276 ++
 .../apache/distributedlog/BKSyncLogWriter.java  |  113 +
 .../apache/distributedlog/BKTransmitPacket.java |   90 +
 .../apache/distributedlog/BookKeeperClient.java |  289 ++
 .../distributedlog/BookKeeperClientBuilder.java |  209 ++
 .../DistributedLogConfiguration.java            | 3528 ++++++++++++++++++
 .../distributedlog/DistributedLogConstants.java |   69 +
 .../distributedlog/DistributedLogManager.java   |  308 ++
 .../java/org/apache/distributedlog/Entry.java   |  403 ++
 .../org/apache/distributedlog/EntryBuffer.java  |   70 +
 .../apache/distributedlog/EntryPosition.java    |   63 +
 .../apache/distributedlog/EnvelopedEntry.java   |  296 ++
 .../distributedlog/EnvelopedEntryReader.java    |  102 +
 .../distributedlog/EnvelopedEntryWriter.java    |  192 +
 .../distributedlog/LedgerReadPosition.java      |  171 +
 .../apache/distributedlog/LocalDLMEmulator.java |  364 ++
 .../org/apache/distributedlog/LogReader.java    |  195 +
 .../distributedlog/LogSegmentMetadata.java      | 1125 ++++++
 .../org/apache/distributedlog/LogWriter.java    |   78 +
 .../distributedlog/MaxLogSegmentSequenceNo.java |   72 +
 .../java/org/apache/distributedlog/MaxTxId.java |   72 +
 .../apache/distributedlog/MetadataAccessor.java |   43 +
 .../distributedlog/ReadAheadEntryReader.java    |  992 +++++
 .../org/apache/distributedlog/ReadUtils.java    |  782 ++++
 .../org/apache/distributedlog/WriteLimiter.java |   62 +
 .../apache/distributedlog/ZooKeeperClient.java  |  402 ++
 .../distributedlog/ZooKeeperClientBuilder.java  |  233 ++
 .../acl/AccessControlManager.java               |   74 +
 .../acl/DefaultAccessControlManager.java        |   55 +
 .../apache/distributedlog/acl/package-info.java |   21 +
 .../admin/DistributedLogAdmin.java              |  921 +++++
 .../distributedlog/admin/package-info.java      |   21 +
 .../distributedlog/auditor/DLAuditor.java       |  630 ++++
 .../bk/DynamicQuorumConfigProvider.java         |   37 +
 .../bk/ImmutableQuorumConfigProvider.java       |   35 +
 .../distributedlog/bk/LedgerAllocator.java      |   32 +
 .../bk/LedgerAllocatorDelegator.java            |   83 +
 .../distributedlog/bk/LedgerAllocatorPool.java  |  458 +++
 .../distributedlog/bk/LedgerAllocatorUtils.java |   54 +
 .../apache/distributedlog/bk/QuorumConfig.java  |   92 +
 .../distributedlog/bk/QuorumConfigProvider.java |   32 +
 .../bk/SimpleLedgerAllocator.java               |  536 +++
 .../apache/distributedlog/bk/package-info.java  |   24 +
 .../callback/LogSegmentListener.java            |   42 +
 .../callback/LogSegmentNamesListener.java       |   42 +
 .../callback/NamespaceListener.java             |   34 +
 .../callback/ReadAheadCallback.java             |   25 +
 .../distributedlog/callback/package-info.java   |   21 +
 .../config/ConcurrentBaseConfiguration.java     |   76 +
 .../config/ConcurrentConstConfiguration.java    |   31 +
 .../config/ConfigurationListener.java           |   32 +
 .../config/ConfigurationSubscription.java       |  186 +
 .../config/DynamicConfigurationFactory.java     |   91 +
 .../DynamicDistributedLogConfiguration.java     |  356 ++
 .../config/FileConfigurationBuilder.java        |   28 +
 .../config/PropertiesConfigurationBuilder.java  |   40 +
 .../distributedlog/config/package-info.java     |   21 +
 .../distributedlog/exceptions/ZKException.java  |   54 +
 .../feature/AbstractFeatureProvider.java        |  103 +
 .../feature/ConfigurationFeatureProvider.java   |   76 +
 .../distributedlog/feature/CoreFeatureKeys.java |   29 +
 .../feature/DefaultFeatureProvider.java         |   47 +
 .../DynamicConfigurationFeatureProvider.java    |  132 +
 .../distributedlog/feature/package-info.java    |   21 +
 .../function/CloseAsyncCloseableFunction.java   |   51 +
 .../function/DefaultValueMapFunction.java       |   41 +
 .../function/GetLastTxIdFunction.java           |   43 +
 .../function/GetVersionedValueFunction.java     |   39 +
 .../distributedlog/function/VoidFunctions.java  |   34 +
 .../distributedlog/function/package-info.java   |   21 +
 .../distributedlog/impl/BKNamespaceDriver.java  |  631 ++++
 .../distributedlog/impl/ZKLogMetadataStore.java |  124 +
 .../impl/ZKLogSegmentFilters.java               |   89 +
 .../impl/ZKLogSegmentMetadataStore.java         |  503 +++
 .../distributedlog/impl/ZKMetadataAccessor.java |  264 ++
 .../distributedlog/impl/ZKNamespaceWatcher.java |  133 +
 .../impl/acl/ZKAccessControl.java               |  232 ++
 .../impl/acl/ZKAccessControlManager.java        |  374 ++
 .../federated/FederatedZKLogMetadataStore.java  |  760 ++++
 .../impl/logsegment/BKLogSegmentAllocator.java  |   85 +
 .../logsegment/BKLogSegmentEntryReader.java     |  837 +++++
 .../impl/logsegment/BKLogSegmentEntryStore.java |  286 ++
 .../logsegment/BKLogSegmentEntryWriter.java     |   61 +
 .../BKLogSegmentRandomAccessEntryReader.java    |  119 +
 .../distributedlog/impl/logsegment/BKUtils.java |   72 +
 .../impl/metadata/BKDLConfig.java               |  400 ++
 .../impl/metadata/ZKLogStreamMetadataStore.java |  633 ++++
 .../impl/metadata/ZkMetadataResolver.java       |   72 +
 .../distributedlog/impl/package-info.java       |   21 +
 .../subscription/ZKSubscriptionStateStore.java  |  121 +
 .../impl/subscription/ZKSubscriptionsStore.java |  164 +
 .../injector/AsyncFailureInjector.java          |  131 +
 .../injector/AsyncRandomFailureInjector.java    |  186 +
 .../injector/FailureInjector.java               |   37 +
 .../injector/RandomDelayFailureInjector.java    |   60 +
 .../distributedlog/injector/package-info.java   |   21 +
 .../org/apache/distributedlog/io/Abortable.java |   41 +
 .../apache/distributedlog/io/Abortables.java    |  183 +
 .../distributedlog/io/AsyncAbortable.java       |   57 +
 .../distributedlog/io/AsyncCloseable.java       |   60 +
 .../distributedlog/io/AsyncDeleteable.java      |   34 +
 .../apache/distributedlog/io/package-info.java  |   21 +
 .../limiter/ChainedRequestLimiter.java          |   76 +
 .../limiter/ComposableRequestLimiter.java       |   73 +
 .../limiter/GuavaRateLimiter.java               |   58 +
 .../distributedlog/limiter/RateLimiter.java     |   49 +
 .../distributedlog/limiter/RequestLimiter.java  |   24 +
 .../distributedlog/limiter/package-info.java    |   21 +
 .../distributedlog/lock/DistributedLock.java    |   54 +
 .../lock/DistributedLockContext.java            |   43 +
 .../lock/EpochChangedException.java             |   33 +
 .../apache/distributedlog/lock/LockAction.java  |   36 +
 .../lock/LockClosedException.java               |   38 +
 .../distributedlog/lock/LockListener.java       |   28 +
 .../lock/LockSessionExpiredException.java       |   34 +
 .../lock/LockStateChangedException.java         |   36 +
 .../lock/LockTimeoutException.java              |   34 +
 .../apache/distributedlog/lock/LockWaiter.java  |   95 +
 .../distributedlog/lock/NopDistributedLock.java |   51 +
 .../apache/distributedlog/lock/SessionLock.java |  126 +
 .../distributedlog/lock/SessionLockFactory.java |   38 +
 .../distributedlog/lock/ZKDistributedLock.java  |  537 +++
 .../distributedlog/lock/ZKSessionLock.java      | 1363 +++++++
 .../lock/ZKSessionLockFactory.java              |  133 +
 .../distributedlog/lock/package-info.java       |   21 +
 .../logsegment/LogSegmentEntryReader.java       |  113 +
 .../logsegment/LogSegmentEntryStore.java        |   73 +
 .../logsegment/LogSegmentEntryWriter.java       |   72 +
 .../logsegment/LogSegmentFilter.java            |   42 +
 .../logsegment/LogSegmentMetadataCache.java     |   98 +
 .../logsegment/LogSegmentMetadataStore.java     |  163 +
 .../LogSegmentRandomAccessEntryReader.java      |   47 +
 .../logsegment/LogSegmentWriter.java            |   87 +
 .../logsegment/PerStreamLogSegmentCache.java    |  243 ++
 .../logsegment/RollingPolicy.java               |   33 +
 .../logsegment/SizeBasedRollingPolicy.java      |   34 +
 .../logsegment/TimeBasedRollingPolicy.java      |   46 +
 .../distributedlog/logsegment/package-info.java |   21 +
 .../distributedlog/metadata/DLConfig.java       |   39 +
 .../distributedlog/metadata/DLMetadata.java     |  227 ++
 .../DryrunLogSegmentMetadataStoreUpdater.java   |   51 +
 .../distributedlog/metadata/LogMetadata.java    |  175 +
 .../metadata/LogMetadataForReader.java          |  103 +
 .../metadata/LogMetadataForWriter.java          |   64 +
 .../metadata/LogMetadataStore.java              |   66 +
 .../LogSegmentMetadataStoreUpdater.java         |  184 +
 .../metadata/LogStreamMetadataStore.java        |  114 +
 .../metadata/MetadataResolver.java              |   51 +
 .../metadata/MetadataUpdater.java               |  120 +
 .../distributedlog/metadata/package-info.java   |   21 +
 .../namespace/DistributedLogNamespace.java      |  190 +
 .../DistributedLogNamespaceBuilder.java         |  278 ++
 .../namespace/NamespaceDriver.java              |  138 +
 .../namespace/NamespaceDriverManager.java       |  180 +
 .../namespace/NamespaceWatcher.java             |   61 +
 .../distributedlog/namespace/package-info.java  |   21 +
 .../apache/distributedlog/net/DNSResolver.java  |  113 +
 .../distributedlog/net/DNSResolverForRacks.java |   66 +
 .../distributedlog/net/DNSResolverForRows.java  |   72 +
 .../org/apache/distributedlog/net/NetUtils.java |   74 +
 .../apache/distributedlog/net/package-info.java |   33 +
 .../org/apache/distributedlog/package-info.java |   21 +
 .../distributedlog/rate/MovingAverageRate.java  |   24 +
 .../rate/MovingAverageRateFactory.java          |   65 +
 .../rate/SampledMovingAverageRate.java          |   58 +
 .../distributedlog/readahead/package-info.java  |   21 +
 .../selector/FirstDLSNNotLessThanSelector.java  |   46 +
 .../selector/FirstRecordSelector.java           |   46 +
 .../selector/FirstTxIdNotLessThanSelector.java  |   51 +
 .../selector/LastRecordSelector.java            |   38 +
 .../selector/LogRecordSelector.java             |   40 +
 .../stats/BKExceptionStatsLogger.java           |  109 +
 .../stats/BroadCastStatsLogger.java             |  194 +
 .../distributedlog/stats/OpStatsListener.java   |   51 +
 .../subscription/SubscriptionStateStore.java    |   42 +
 .../subscription/SubscriptionsStore.java        |   69 +
 .../tools/DistributedLogTool.java               | 2873 ++++++++++++++
 .../org/apache/distributedlog/tools/Tool.java   |  243 ++
 .../distributedlog/tools/package-info.java      |   21 +
 .../apache/distributedlog/util/Allocator.java   |  102 +
 .../distributedlog/util/CommandLineUtils.java   |   56 +
 .../apache/distributedlog/util/ConfUtils.java   |   65 +
 .../org/apache/distributedlog/util/DLUtils.java |  321 ++
 .../distributedlog/util/FailpointUtils.java     |  134 +
 .../apache/distributedlog/util/FutureUtils.java |  534 +++
 .../util/MonitoredFuturePool.java               |  131 +
 .../MonitoredScheduledThreadPoolExecutor.java   |  257 ++
 .../distributedlog/util/OrderedScheduler.java   |  490 +++
 .../distributedlog/util/PermitLimiter.java      |   57 +
 .../distributedlog/util/PermitManager.java      |   93 +
 .../distributedlog/util/RetryPolicyUtils.java   |   45 +
 .../util/SafeQueueingFuturePool.java            |  115 +
 .../distributedlog/util/SchedulerUtils.java     |   56 +
 .../apache/distributedlog/util/Sequencer.java   |   31 +
 .../util/SimplePermitLimiter.java               |  117 +
 .../org/apache/distributedlog/util/Sizable.java |   31 +
 .../distributedlog/util/TimeSequencer.java      |   39 +
 .../apache/distributedlog/util/Transaction.java |   97 +
 .../org/apache/distributedlog/util/Utils.java   |  607 +++
 .../distributedlog/util/package-info.java       |   21 +
 .../apache/distributedlog/zk/DefaultZKOp.java   |   55 +
 .../distributedlog/zk/LimitedPermitManager.java |  195 +
 .../java/org/apache/distributedlog/zk/ZKOp.java |   63 +
 .../apache/distributedlog/zk/ZKTransaction.java |  103 +
 .../distributedlog/zk/ZKVersionedSetOp.java     |   71 +
 .../distributedlog/zk/ZKWatcherManager.java     |  239 ++
 .../src/main/resources/findbugsExclude.xml      |    8 +-
 .../src/main/thrift/metadata.thrift             |    2 +-
 .../com/twitter/distributedlog/DLMTestUtil.java |  489 ---
 .../NonBlockingReadsTestUtil.java               |  145 -
 .../TestAppendOnlyStreamReader.java             |  207 -
 .../TestAppendOnlyStreamWriter.java             |  337 --
 .../distributedlog/TestAsyncBulkWrite.java      |  351 --
 .../distributedlog/TestAsyncReaderLock.java     |  607 ---
 .../distributedlog/TestAsyncReaderWriter.java   | 2206 -----------
 .../TestBKDistributedLogManager.java            | 1201 ------
 .../TestBKDistributedLogNamespace.java          |  440 ---
 .../distributedlog/TestBKLogReadHandler.java    |  398 --
 .../distributedlog/TestBKLogSegmentWriter.java  |  790 ----
 .../distributedlog/TestBKLogWriteHandler.java   |   90 -
 .../distributedlog/TestBKSyncLogReader.java     |  306 --
 .../twitter/distributedlog/TestDLMTestUtil.java |   62 -
 .../distributedlog/TestDistributedLogBase.java  |  244 --
 .../TestDistributedLogConfiguration.java        |  132 -
 .../com/twitter/distributedlog/TestEntry.java   |  345 --
 .../distributedlog/TestEntryPosition.java       |   59 -
 .../distributedlog/TestEnvelopedEntry.java      |   81 -
 .../distributedlog/TestInterleavedReaders.java  |  338 --
 .../distributedlog/TestLogSegmentCreation.java  |  106 -
 .../distributedlog/TestLogSegmentMetadata.java  |  159 -
 .../distributedlog/TestLogSegmentsZK.java       |  245 --
 .../distributedlog/TestNonBlockingReads.java    |  348 --
 .../TestNonBlockingReadsMultiReader.java        |  168 -
 .../TestReadAheadEntryReader.java               |  463 ---
 .../twitter/distributedlog/TestReadUtils.java   |  344 --
 .../com/twitter/distributedlog/TestReader.java  |  206 -
 .../distributedlog/TestRollLogSegments.java     |  429 ---
 .../twitter/distributedlog/TestSequenceID.java  |  254 --
 .../twitter/distributedlog/TestTruncate.java    |  341 --
 .../distributedlog/TestWriteLimiter.java        |  205 -
 .../distributedlog/TestZooKeeperClient.java     |  449 ---
 .../TestZooKeeperClientBuilder.java             |   57 -
 .../distributedlog/ZooKeeperClientUtils.java    |   95 -
 .../ZooKeeperClusterTestCase.java               |   53 -
 .../distributedlog/acl/TestZKAccessControl.java |  154 -
 .../acl/TestZKAccessControlManager.java         |  178 -
 .../twitter/distributedlog/admin/TestDLCK.java  |  164 -
 .../admin/TestDistributedLogAdmin.java          |  194 -
 .../distributedlog/bk/TestLedgerAllocator.java  |  389 --
 .../bk/TestLedgerAllocatorPool.java             |  311 --
 .../distributedlog/config/PropertiesWriter.java |   67 -
 .../config/TestConcurrentBaseConfiguration.java |   46 -
 .../config/TestConfigurationSubscription.java   |  171 -
 .../config/TestDynamicConfigurationFactory.java |  101 -
 .../TestDynamicDistributedLogConfiguration.java |  261 --
 .../TestConfigurationFeatureProvider.java       |   65 -
 ...TestDynamicConfigurationFeatureProvider.java |  183 -
 .../impl/TestZKLogMetadataStore.java            |  114 -
 .../impl/TestZKLogSegmentFilters.java           |   74 -
 .../impl/TestZKLogSegmentMetadataStore.java     |  831 -----
 .../impl/TestZKNamespaceWatcher.java            |  185 -
 .../TestFederatedZKLogMetadataStore.java        |  446 ---
 .../logsegment/TestBKLogSegmentEntryReader.java |  561 ---
 .../metadata/TestZKLogStreamMetadataStore.java  |  327 --
 .../TestZKLogStreamMetadataStoreUtils.java      |  208 --
 .../impl/metadata/TestZkMetadataResolver.java   |  203 -
 .../limiter/TestRequestLimiter.java             |   60 -
 .../lock/TestDistributedLock.java               |  835 -----
 .../distributedlog/lock/TestZKSessionLock.java  | 1224 ------
 .../TestPerStreamLogSegmentCache.java           |  186 -
 .../logsegment/TestRollingPolicy.java           |   63 -
 .../distributedlog/metadata/TestDLMetadata.java |  195 -
 .../metadata/TestLogMetadata.java               |   59 -
 .../TestLogSegmentMetadataStoreUpdater.java     |  284 --
 .../TestDistributedLogNamespaceBuilder.java     |  124 -
 .../distributedlog/net/TestDNSResolver.java     |   96 -
 .../distributedlog/net/TestNetUtils.java        |   82 -
 .../rate/TestMovingAverageRate.java             |   99 -
 .../selector/TestLogRecordSelectors.java        |  128 -
 .../tools/TestDistributedLogTool.java           |  231 --
 .../distributedlog/util/TestConfUtils.java      |   53 -
 .../distributedlog/util/TestDLUtils.java        |  273 --
 .../distributedlog/util/TestFutureUtils.java    |   71 -
 .../distributedlog/util/TestPermitManager.java  |   92 -
 .../util/TestSafeQueueingFuturePool.java        |  205 -
 .../distributedlog/util/TestTimeSequencer.java  |   42 -
 .../twitter/distributedlog/util/TestUtils.java  |  125 -
 .../distributedlog/zk/TestZKTransaction.java    |   97 -
 .../distributedlog/zk/TestZKVersionedSetOp.java |   86 -
 .../distributedlog/zk/TestZKWatcherManager.java |   80 -
 .../org/apache/distributedlog/DLMTestUtil.java  |  489 +++
 .../NonBlockingReadsTestUtil.java               |  145 +
 .../TestAppendOnlyStreamReader.java             |  207 +
 .../TestAppendOnlyStreamWriter.java             |  337 ++
 .../distributedlog/TestAsyncBulkWrite.java      |  351 ++
 .../distributedlog/TestAsyncReaderLock.java     |  607 +++
 .../distributedlog/TestAsyncReaderWriter.java   | 2206 +++++++++++
 .../TestBKDistributedLogManager.java            | 1201 ++++++
 .../TestBKDistributedLogNamespace.java          |  440 +++
 .../distributedlog/TestBKLogReadHandler.java    |  398 ++
 .../distributedlog/TestBKLogSegmentWriter.java  |  790 ++++
 .../distributedlog/TestBKLogWriteHandler.java   |   90 +
 .../distributedlog/TestBKSyncLogReader.java     |  306 ++
 .../apache/distributedlog/TestDLMTestUtil.java  |   62 +
 .../distributedlog/TestDistributedLogBase.java  |  244 ++
 .../TestDistributedLogConfiguration.java        |  132 +
 .../org/apache/distributedlog/TestEntry.java    |  345 ++
 .../distributedlog/TestEntryPosition.java       |   59 +
 .../distributedlog/TestEnvelopedEntry.java      |   81 +
 .../distributedlog/TestInterleavedReaders.java  |  338 ++
 .../distributedlog/TestLogSegmentCreation.java  |  106 +
 .../distributedlog/TestLogSegmentMetadata.java  |  159 +
 .../distributedlog/TestLogSegmentsZK.java       |  245 ++
 .../distributedlog/TestNonBlockingReads.java    |  348 ++
 .../TestNonBlockingReadsMultiReader.java        |  168 +
 .../TestReadAheadEntryReader.java               |  463 +++
 .../apache/distributedlog/TestReadUtils.java    |  344 ++
 .../org/apache/distributedlog/TestReader.java   |  206 +
 .../distributedlog/TestRollLogSegments.java     |  429 +++
 .../apache/distributedlog/TestSequenceID.java   |  254 ++
 .../org/apache/distributedlog/TestTruncate.java |  341 ++
 .../apache/distributedlog/TestWriteLimiter.java |  205 +
 .../distributedlog/TestZooKeeperClient.java     |  449 +++
 .../TestZooKeeperClientBuilder.java             |   57 +
 .../distributedlog/ZooKeeperClientUtils.java    |   95 +
 .../ZooKeeperClusterTestCase.java               |   53 +
 .../distributedlog/acl/TestZKAccessControl.java |  154 +
 .../acl/TestZKAccessControlManager.java         |  178 +
 .../apache/distributedlog/admin/TestDLCK.java   |  164 +
 .../admin/TestDistributedLogAdmin.java          |  194 +
 .../distributedlog/bk/TestLedgerAllocator.java  |  389 ++
 .../bk/TestLedgerAllocatorPool.java             |  311 ++
 .../distributedlog/config/PropertiesWriter.java |   67 +
 .../config/TestConcurrentBaseConfiguration.java |   46 +
 .../config/TestConfigurationSubscription.java   |  171 +
 .../config/TestDynamicConfigurationFactory.java |  101 +
 .../TestDynamicDistributedLogConfiguration.java |  261 ++
 .../TestConfigurationFeatureProvider.java       |   65 +
 ...TestDynamicConfigurationFeatureProvider.java |  183 +
 .../impl/TestZKLogMetadataStore.java            |  114 +
 .../impl/TestZKLogSegmentFilters.java           |   74 +
 .../impl/TestZKLogSegmentMetadataStore.java     |  831 +++++
 .../impl/TestZKNamespaceWatcher.java            |  185 +
 .../TestFederatedZKLogMetadataStore.java        |  446 +++
 .../logsegment/TestBKLogSegmentEntryReader.java |  561 +++
 .../metadata/TestZKLogStreamMetadataStore.java  |  327 ++
 .../TestZKLogStreamMetadataStoreUtils.java      |  208 ++
 .../impl/metadata/TestZkMetadataResolver.java   |  203 +
 .../limiter/TestRequestLimiter.java             |   60 +
 .../lock/TestDistributedLock.java               |  835 +++++
 .../distributedlog/lock/TestZKSessionLock.java  | 1224 ++++++
 .../TestPerStreamLogSegmentCache.java           |  186 +
 .../logsegment/TestRollingPolicy.java           |   63 +
 .../distributedlog/metadata/TestDLMetadata.java |  195 +
 .../metadata/TestLogMetadata.java               |   59 +
 .../TestLogSegmentMetadataStoreUpdater.java     |  284 ++
 .../TestDistributedLogNamespaceBuilder.java     |  124 +
 .../distributedlog/net/TestDNSResolver.java     |   96 +
 .../apache/distributedlog/net/TestNetUtils.java |   82 +
 .../rate/TestMovingAverageRate.java             |   99 +
 .../selector/TestLogRecordSelectors.java        |  128 +
 .../tools/TestDistributedLogTool.java           |  231 ++
 .../distributedlog/util/TestConfUtils.java      |   53 +
 .../apache/distributedlog/util/TestDLUtils.java |  273 ++
 .../distributedlog/util/TestFutureUtils.java    |   71 +
 .../distributedlog/util/TestPermitManager.java  |   92 +
 .../util/TestSafeQueueingFuturePool.java        |  205 +
 .../distributedlog/util/TestTimeSequencer.java  |   42 +
 .../apache/distributedlog/util/TestUtils.java   |  125 +
 .../distributedlog/zk/TestZKTransaction.java    |   97 +
 .../distributedlog/zk/TestZKVersionedSetOp.java |   86 +
 .../distributedlog/zk/TestZKWatcherManager.java |   80 +
 .../java/com/twitter/distributedlog/DLSN.java   |  266 --
 .../EnvelopedRecordSetReader.java               |  118 -
 .../EnvelopedRecordSetWriter.java               |  197 -
 .../com/twitter/distributedlog/LogRecord.java   |  606 ---
 .../twitter/distributedlog/LogRecordSet.java    |  155 -
 .../distributedlog/LogRecordSetBuffer.java      |   65 -
 .../distributedlog/LogRecordWithDLSN.java       |  107 -
 .../twitter/distributedlog/RecordStream.java    |   42 -
 .../annotations/DistributedLogAnnotations.java  |   35 -
 .../annotations/package-info.java               |   21 -
 .../exceptions/AlreadyClosedException.java      |   34 -
 .../AlreadyTruncatedTransactionException.java   |   32 -
 .../exceptions/BKTransmitException.java         |   42 -
 .../exceptions/ChecksumFailedException.java     |   32 -
 .../exceptions/DLClientClosedException.java     |   36 -
 .../distributedlog/exceptions/DLException.java  |   81 -
 .../exceptions/DLIllegalStateException.java     |   36 -
 .../exceptions/DLInterruptedException.java      |   36 -
 .../exceptions/EndOfLogSegmentException.java    |   32 -
 .../exceptions/EndOfStreamException.java        |   32 -
 .../exceptions/FlushException.java              |   51 -
 .../exceptions/IdleReaderException.java         |   34 -
 .../exceptions/InternalServerException.java     |   40 -
 .../InvalidEnvelopedEntryException.java         |   32 -
 .../exceptions/InvalidStreamNameException.java  |   36 -
 .../exceptions/LockCancelledException.java      |   31 -
 .../exceptions/LockingException.java            |   44 -
 .../exceptions/LogEmptyException.java           |   32 -
 .../exceptions/LogExistsException.java          |   31 -
 .../exceptions/LogNotFoundException.java        |   32 -
 .../exceptions/LogReadException.java            |   33 -
 .../exceptions/LogRecordTooLongException.java   |   34 -
 .../LogSegmentIsTruncatedException.java         |   33 -
 .../exceptions/LogSegmentNotFoundException.java |   32 -
 .../exceptions/MetadataException.java           |   36 -
 .../exceptions/NotYetImplementedException.java  |   32 -
 .../exceptions/OverCapacityException.java       |   39 -
 .../OwnershipAcquireFailedException.java        |   40 -
 .../exceptions/ReadCancelledException.java      |   33 -
 .../exceptions/RegionUnavailableException.java  |   34 -
 .../exceptions/RequestDeniedException.java      |   33 -
 .../exceptions/RetryableReadException.java      |   36 -
 .../exceptions/ServiceUnavailableException.java |   32 -
 .../exceptions/StreamNotReadyException.java     |   32 -
 .../exceptions/StreamUnavailableException.java  |   32 -
 .../exceptions/TooManyStreamsException.java     |   32 -
 .../TransactionIdOutOfOrderException.java       |   47 -
 .../exceptions/UnexpectedException.java         |   40 -
 .../UnsupportedMetadataVersionException.java    |   32 -
 .../exceptions/WriteCancelledException.java     |   45 -
 .../exceptions/WriteException.java              |   33 -
 .../distributedlog/exceptions/package-info.java |   21 -
 .../com/twitter/distributedlog/io/Buffer.java   |   33 -
 .../distributedlog/io/CompressionCodec.java     |   80 -
 .../distributedlog/io/CompressionUtils.java     |   58 -
 .../io/IdentityCompressionCodec.java            |   50 -
 .../distributedlog/io/LZ4CompressionCodec.java  |  101 -
 .../distributedlog/io/TransmitListener.java     |   56 -
 .../twitter/distributedlog/io/package-info.java |   21 -
 .../twitter/distributedlog/package-info.java    |   21 -
 .../distributedlog/util/BitMaskUtils.java       |   57 -
 .../distributedlog/util/ProtocolUtils.java      |   79 -
 .../distributedlog/util/package-info.java       |   21 -
 .../java/org/apache/distributedlog/DLSN.java    |  266 ++
 .../EnvelopedRecordSetReader.java               |  118 +
 .../EnvelopedRecordSetWriter.java               |  197 +
 .../org/apache/distributedlog/LogRecord.java    |  606 +++
 .../org/apache/distributedlog/LogRecordSet.java |  155 +
 .../distributedlog/LogRecordSetBuffer.java      |   65 +
 .../distributedlog/LogRecordWithDLSN.java       |  107 +
 .../org/apache/distributedlog/RecordStream.java |   42 +
 .../annotations/DistributedLogAnnotations.java  |   35 +
 .../annotations/package-info.java               |   21 +
 .../exceptions/AlreadyClosedException.java      |   34 +
 .../AlreadyTruncatedTransactionException.java   |   32 +
 .../exceptions/BKTransmitException.java         |   42 +
 .../exceptions/ChecksumFailedException.java     |   32 +
 .../exceptions/DLClientClosedException.java     |   36 +
 .../distributedlog/exceptions/DLException.java  |   81 +
 .../exceptions/DLIllegalStateException.java     |   36 +
 .../exceptions/DLInterruptedException.java      |   36 +
 .../exceptions/EndOfLogSegmentException.java    |   32 +
 .../exceptions/EndOfStreamException.java        |   32 +
 .../exceptions/FlushException.java              |   51 +
 .../exceptions/IdleReaderException.java         |   34 +
 .../exceptions/InternalServerException.java     |   40 +
 .../InvalidEnvelopedEntryException.java         |   32 +
 .../exceptions/InvalidStreamNameException.java  |   36 +
 .../exceptions/LockCancelledException.java      |   31 +
 .../exceptions/LockingException.java            |   44 +
 .../exceptions/LogEmptyException.java           |   32 +
 .../exceptions/LogExistsException.java          |   31 +
 .../exceptions/LogNotFoundException.java        |   32 +
 .../exceptions/LogReadException.java            |   33 +
 .../exceptions/LogRecordTooLongException.java   |   34 +
 .../LogSegmentIsTruncatedException.java         |   33 +
 .../exceptions/LogSegmentNotFoundException.java |   32 +
 .../exceptions/MetadataException.java           |   36 +
 .../exceptions/NotYetImplementedException.java  |   32 +
 .../exceptions/OverCapacityException.java       |   39 +
 .../OwnershipAcquireFailedException.java        |   40 +
 .../exceptions/ReadCancelledException.java      |   33 +
 .../exceptions/RegionUnavailableException.java  |   34 +
 .../exceptions/RequestDeniedException.java      |   33 +
 .../exceptions/RetryableReadException.java      |   36 +
 .../exceptions/ServiceUnavailableException.java |   32 +
 .../exceptions/StreamNotReadyException.java     |   32 +
 .../exceptions/StreamUnavailableException.java  |   32 +
 .../exceptions/TooManyStreamsException.java     |   32 +
 .../TransactionIdOutOfOrderException.java       |   47 +
 .../exceptions/UnexpectedException.java         |   40 +
 .../UnsupportedMetadataVersionException.java    |   32 +
 .../exceptions/WriteCancelledException.java     |   45 +
 .../exceptions/WriteException.java              |   33 +
 .../distributedlog/exceptions/package-info.java |   21 +
 .../org/apache/distributedlog/io/Buffer.java    |   33 +
 .../distributedlog/io/CompressionCodec.java     |   80 +
 .../distributedlog/io/CompressionUtils.java     |   58 +
 .../io/IdentityCompressionCodec.java            |   50 +
 .../distributedlog/io/LZ4CompressionCodec.java  |  101 +
 .../distributedlog/io/TransmitListener.java     |   56 +
 .../apache/distributedlog/io/package-info.java  |   21 +
 .../org/apache/distributedlog/package-info.java |   21 +
 .../distributedlog/util/BitMaskUtils.java       |   57 +
 .../distributedlog/util/ProtocolUtils.java      |   79 +
 .../distributedlog/util/package-info.java       |   21 +
 .../src/main/resources/findbugsExclude.xml      |    8 +-
 .../src/main/thrift/service.thrift              |    2 +-
 .../com/twitter/distributedlog/TestDLSN.java    |   64 -
 .../distributedlog/TestLogRecordSet.java        |  178 -
 .../TestTimedOutTestsListener.java              |  183 -
 .../distributedlog/TimedOutTestsListener.java   |  168 -
 .../org/apache/distributedlog/TestDLSN.java     |   64 +
 .../apache/distributedlog/TestLogRecordSet.java |  178 +
 .../TestTimedOutTestsListener.java              |  183 +
 .../distributedlog/TimedOutTestsListener.java   |  168 +
 distributedlog-service/bin/dlog-daemon.sh       |    2 +-
 distributedlog-service/bin/dlog-start.sh        |    2 +-
 distributedlog-service/pom.xml                  |    2 +-
 .../distributedlog/service/ClientUtils.java     |   33 -
 .../service/DistributedLogCluster.java          |  352 --
 .../service/DistributedLogServer.java           |  460 ---
 .../service/DistributedLogServerApp.java        |  187 -
 .../service/DistributedLogServiceImpl.java      |  794 ----
 .../service/FatalErrorHandler.java              |   30 -
 .../distributedlog/service/MonitorService.java  |  469 ---
 .../service/MonitorServiceApp.java              |  133 -
 .../distributedlog/service/ResponseUtils.java   |   86 -
 .../service/ServerFeatureKeys.java              |   29 -
 .../distributedlog/service/StatsFilter.java     |   60 -
 .../service/announcer/Announcer.java            |   41 -
 .../service/announcer/NOPAnnouncer.java         |   40 -
 .../service/announcer/ServerSetAnnouncer.java   |  111 -
 .../service/announcer/package-info.java         |   21 -
 .../service/balancer/Balancer.java              |   68 -
 .../service/balancer/BalancerTool.java          |  327 --
 .../service/balancer/BalancerUtils.java         |   74 -
 .../service/balancer/ClusterBalancer.java       |  378 --
 .../balancer/CountBasedStreamChooser.java       |  109 -
 .../service/balancer/LimitedStreamChooser.java  |   57 -
 .../service/balancer/SimpleBalancer.java        |  246 --
 .../service/balancer/StreamChooser.java         |   30 -
 .../service/balancer/StreamMover.java           |   34 -
 .../service/balancer/StreamMoverImpl.java       |   94 -
 .../service/balancer/package-info.java          |   21 -
 .../config/DefaultStreamConfigProvider.java     |   73 -
 .../config/NullStreamConfigProvider.java        |   40 -
 .../service/config/ServerConfiguration.java     |  443 ---
 .../config/ServiceStreamConfigProvider.java     |   88 -
 .../service/config/StreamConfigProvider.java    |   34 -
 .../service/config/package-info.java            |   21 -
 .../distributedlog/service/package-info.java    |   21 -
 .../service/placement/EqualLoadAppraiser.java   |   39 -
 .../placement/LeastLoadPlacementPolicy.java     |  200 -
 .../service/placement/LoadAppraiser.java        |   39 -
 .../service/placement/PlacementPolicy.java      |  148 -
 .../placement/PlacementStateManager.java        |   79 -
 .../service/placement/ServerLoad.java           |  158 -
 .../service/placement/StreamLoad.java           |  115 -
 .../placement/ZKPlacementStateManager.java      |  173 -
 .../service/placement/package-info.java         |   21 -
 .../service/stream/AbstractStreamOp.java        |  175 -
 .../service/stream/AbstractWriteOp.java         |   60 -
 .../service/stream/BulkWriteOp.java             |  253 --
 .../distributedlog/service/stream/DeleteOp.java |   76 -
 .../service/stream/HeartbeatOp.java             |  101 -
 .../service/stream/ReleaseOp.java               |   76 -
 .../distributedlog/service/stream/Stream.java   |   93 -
 .../service/stream/StreamFactory.java           |   38 -
 .../service/stream/StreamFactoryImpl.java       |   95 -
 .../service/stream/StreamImpl.java              |  925 -----
 .../service/stream/StreamManager.java           |  142 -
 .../service/stream/StreamManagerImpl.java       |  413 --
 .../distributedlog/service/stream/StreamOp.java |   75 -
 .../service/stream/StreamOpStats.java           |  104 -
 .../service/stream/TruncateOp.java              |   91 -
 .../distributedlog/service/stream/WriteOp.java  |  173 -
 .../service/stream/WriteOpWithPayload.java      |   27 -
 .../service/stream/admin/AdminOp.java           |   40 -
 .../service/stream/admin/CreateOp.java          |   57 -
 .../service/stream/admin/StreamAdminOp.java     |  100 -
 .../service/stream/admin/package-info.java      |   21 -
 .../stream/limiter/DynamicRequestLimiter.java   |   94 -
 .../stream/limiter/RequestLimiterBuilder.java   |  116 -
 .../stream/limiter/ServiceRequestLimiter.java   |  103 -
 .../stream/limiter/StreamAcquireLimiter.java    |   56 -
 .../stream/limiter/StreamRequestLimiter.java    |   87 -
 .../service/stream/limiter/package-info.java    |   21 -
 .../service/stream/package-info.java            |   21 -
 .../CacheableStreamPartitionConverter.java      |   58 -
 .../DelimiterStreamPartitionConverter.java      |   50 -
 .../IdentityStreamPartitionConverter.java       |   28 -
 .../service/streamset/Partition.java            |   99 -
 .../service/streamset/PartitionMap.java         |   58 -
 .../streamset/StreamPartitionConverter.java     |   35 -
 .../service/streamset/package-info.java         |   21 -
 .../distributedlog/service/tools/ProxyTool.java |  350 --
 .../service/tools/package-info.java             |   21 -
 .../service/utils/ServerUtils.java              |   49 -
 .../service/utils/package-info.java             |   21 -
 .../apache/bookkeeper/stats/package-info.java   |    2 +-
 .../distributedlog/service/ClientUtils.java     |   33 +
 .../service/DistributedLogCluster.java          |  352 ++
 .../service/DistributedLogServer.java           |  460 +++
 .../service/DistributedLogServerApp.java        |  187 +
 .../service/DistributedLogServiceImpl.java      |  794 ++++
 .../service/FatalErrorHandler.java              |   30 +
 .../distributedlog/service/MonitorService.java  |  469 +++
 .../service/MonitorServiceApp.java              |  133 +
 .../distributedlog/service/ResponseUtils.java   |   86 +
 .../service/ServerFeatureKeys.java              |   29 +
 .../distributedlog/service/StatsFilter.java     |   60 +
 .../service/announcer/Announcer.java            |   41 +
 .../service/announcer/NOPAnnouncer.java         |   40 +
 .../service/announcer/ServerSetAnnouncer.java   |  111 +
 .../service/announcer/package-info.java         |   21 +
 .../service/balancer/Balancer.java              |   68 +
 .../service/balancer/BalancerTool.java          |  327 ++
 .../service/balancer/BalancerUtils.java         |   74 +
 .../service/balancer/ClusterBalancer.java       |  378 ++
 .../balancer/CountBasedStreamChooser.java       |  109 +
 .../service/balancer/LimitedStreamChooser.java  |   57 +
 .../service/balancer/SimpleBalancer.java        |  246 ++
 .../service/balancer/StreamChooser.java         |   30 +
 .../service/balancer/StreamMover.java           |   34 +
 .../service/balancer/StreamMoverImpl.java       |   94 +
 .../service/balancer/package-info.java          |   21 +
 .../config/DefaultStreamConfigProvider.java     |   73 +
 .../config/NullStreamConfigProvider.java        |   40 +
 .../service/config/ServerConfiguration.java     |  443 +++
 .../config/ServiceStreamConfigProvider.java     |   88 +
 .../service/config/StreamConfigProvider.java    |   34 +
 .../service/config/package-info.java            |   21 +
 .../distributedlog/service/package-info.java    |   21 +
 .../service/placement/EqualLoadAppraiser.java   |   39 +
 .../placement/LeastLoadPlacementPolicy.java     |  200 +
 .../service/placement/LoadAppraiser.java        |   39 +
 .../service/placement/PlacementPolicy.java      |  148 +
 .../placement/PlacementStateManager.java        |   79 +
 .../service/placement/ServerLoad.java           |  158 +
 .../service/placement/StreamLoad.java           |  115 +
 .../placement/ZKPlacementStateManager.java      |  173 +
 .../service/placement/package-info.java         |   21 +
 .../service/stream/AbstractStreamOp.java        |  175 +
 .../service/stream/AbstractWriteOp.java         |   60 +
 .../service/stream/BulkWriteOp.java             |  253 ++
 .../distributedlog/service/stream/DeleteOp.java |   76 +
 .../service/stream/HeartbeatOp.java             |  101 +
 .../service/stream/ReleaseOp.java               |   76 +
 .../distributedlog/service/stream/Stream.java   |   93 +
 .../service/stream/StreamFactory.java           |   38 +
 .../service/stream/StreamFactoryImpl.java       |   95 +
 .../service/stream/StreamImpl.java              |  925 +++++
 .../service/stream/StreamManager.java           |  142 +
 .../service/stream/StreamManagerImpl.java       |  413 ++
 .../distributedlog/service/stream/StreamOp.java |   75 +
 .../service/stream/StreamOpStats.java           |  104 +
 .../service/stream/TruncateOp.java              |   91 +
 .../distributedlog/service/stream/WriteOp.java  |  173 +
 .../service/stream/WriteOpWithPayload.java      |   27 +
 .../service/stream/admin/AdminOp.java           |   40 +
 .../service/stream/admin/CreateOp.java          |   57 +
 .../service/stream/admin/StreamAdminOp.java     |  100 +
 .../service/stream/admin/package-info.java      |   21 +
 .../stream/limiter/DynamicRequestLimiter.java   |   94 +
 .../stream/limiter/RequestLimiterBuilder.java   |  116 +
 .../stream/limiter/ServiceRequestLimiter.java   |  103 +
 .../stream/limiter/StreamAcquireLimiter.java    |   56 +
 .../stream/limiter/StreamRequestLimiter.java    |   87 +
 .../service/stream/limiter/package-info.java    |   21 +
 .../service/stream/package-info.java            |   21 +
 .../CacheableStreamPartitionConverter.java      |   58 +
 .../DelimiterStreamPartitionConverter.java      |   50 +
 .../IdentityStreamPartitionConverter.java       |   28 +
 .../service/streamset/Partition.java            |   99 +
 .../service/streamset/PartitionMap.java         |   58 +
 .../streamset/StreamPartitionConverter.java     |   35 +
 .../service/streamset/package-info.java         |   21 +
 .../distributedlog/service/tools/ProxyTool.java |  350 ++
 .../service/tools/package-info.java             |   21 +
 .../service/utils/ServerUtils.java              |   49 +
 .../service/utils/package-info.java             |   21 +
 .../src/main/resources/findbugsExclude.xml      |    8 +-
 .../src/main/thrift/metadata.thrift             |    2 +-
 .../client/routing/LocalRoutingService.java     |  146 -
 .../service/DistributedLogServerTestCase.java   |  298 --
 .../service/TestDistributedLogServerBase.java   |  720 ----
 .../TestDistributedLogServerClientRouting.java  |   58 -
 .../TestDistributedLogServerServerRouting.java  |   28 -
 .../service/TestDistributedLogService.java      |  833 -----
 .../service/TestRegionUnavailable.java          |  140 -
 .../distributedlog/service/TestStatsFilter.java |   58 -
 .../service/balancer/TestBalancerUtils.java     |   65 -
 .../service/balancer/TestClusterBalancer.java   |  189 -
 .../balancer/TestCountBasedStreamChooser.java   |  204 -
 .../service/balancer/TestSimpleBalancer.java    |  180 -
 .../service/balancer/TestStreamMover.java       |   86 -
 .../service/config/TestServerConfiguration.java |   68 -
 .../config/TestStreamConfigProvider.java        |  140 -
 .../placement/TestLeastLoadPlacementPolicy.java |  176 -
 .../service/placement/TestServerLoad.java       |   50 -
 .../service/placement/TestStreamLoad.java       |   37 -
 .../placement/TestZKPlacementStateManager.java  |  136 -
 .../service/stream/TestStreamManager.java       |  135 -
 .../service/stream/TestStreamOp.java            |   95 -
 .../limiter/TestServiceRequestLimiter.java      |  301 --
 .../TestDelimiterStreamPartitionConverter.java  |   52 -
 .../TestIdentityStreamPartitionConverter.java   |   43 -
 .../service/streamset/TestPartitionMap.java     |   68 -
 .../service/utils/TestServerUtils.java          |   41 -
 .../client/routing/LocalRoutingService.java     |  146 +
 .../service/DistributedLogServerTestCase.java   |  298 ++
 .../service/TestDistributedLogServerBase.java   |  720 ++++
 .../TestDistributedLogServerClientRouting.java  |   58 +
 .../TestDistributedLogServerServerRouting.java  |   28 +
 .../service/TestDistributedLogService.java      |  833 +++++
 .../service/TestRegionUnavailable.java          |  140 +
 .../distributedlog/service/TestStatsFilter.java |   58 +
 .../service/balancer/TestBalancerUtils.java     |   65 +
 .../service/balancer/TestClusterBalancer.java   |  189 +
 .../balancer/TestCountBasedStreamChooser.java   |  204 +
 .../service/balancer/TestSimpleBalancer.java    |  180 +
 .../service/balancer/TestStreamMover.java       |   86 +
 .../service/config/TestServerConfiguration.java |   68 +
 .../config/TestStreamConfigProvider.java        |  140 +
 .../placement/TestLeastLoadPlacementPolicy.java |  176 +
 .../service/placement/TestServerLoad.java       |   50 +
 .../service/placement/TestStreamLoad.java       |   37 +
 .../placement/TestZKPlacementStateManager.java  |  136 +
 .../service/stream/TestStreamManager.java       |  135 +
 .../service/stream/TestStreamOp.java            |   95 +
 .../limiter/TestServiceRequestLimiter.java      |  301 ++
 .../TestDelimiterStreamPartitionConverter.java  |   52 +
 .../TestIdentityStreamPartitionConverter.java   |   43 +
 .../service/streamset/TestPartitionMap.java     |   68 +
 .../service/utils/TestServerUtils.java          |   41 +
 .../distributedlog-basic/basic-1.md             |   16 +-
 .../distributedlog-basic/basic-2.md             |   10 +-
 .../distributedlog-basic/basic-3.md             |   10 +-
 .../distributedlog-basic/basic-4.md             |   10 +-
 .../distributedlog-basic/basic-5.md             |   10 +-
 .../distributedlog-basic/basic-6.md             |   10 +-
 .../distributedlog-basic/conf/log4j.properties  |    8 +-
 .../distributedlog/basic/AtomicWriter.java      |  107 -
 .../basic/ConsoleProxyMultiWriter.java          |   91 -
 .../basic/ConsoleProxyWriter.java               |   77 -
 .../distributedlog/basic/ConsoleWriter.java     |  101 -
 .../distributedlog/basic/MultiReader.java       |  140 -
 .../distributedlog/basic/RecordGenerator.java   |   90 -
 .../distributedlog/basic/StreamRewinder.java    |  113 -
 .../distributedlog/basic/TailReader.java        |  112 -
 .../distributedlog/basic/AtomicWriter.java      |  107 +
 .../basic/ConsoleProxyMultiWriter.java          |   91 +
 .../basic/ConsoleProxyWriter.java               |   77 +
 .../distributedlog/basic/ConsoleWriter.java     |  101 +
 .../distributedlog/basic/MultiReader.java       |  140 +
 .../distributedlog/basic/RecordGenerator.java   |   90 +
 .../distributedlog/basic/StreamRewinder.java    |  113 +
 .../apache/distributedlog/basic/TailReader.java |  112 +
 .../distributedlog-kafka/conf/log4j.properties  |    8 +-
 .../kafka/DLFutureRecordMetadata.java           |  104 -
 .../kafka/KafkaDistributedLogProducer.java      |  158 -
 .../kafka/DLFutureRecordMetadata.java           |  104 +
 .../kafka/KafkaDistributedLogProducer.java      |  158 +
 .../distributedlog-mapreduce/README.md          |    2 +-
 .../mapreduce/DistributedLogInputFormat.java    |  129 -
 .../mapreduce/LogSegmentReader.java             |  148 -
 .../mapreduce/LogSegmentSplit.java              |   95 -
 .../mapreduce/DistributedLogInputFormat.java    |  129 +
 .../mapreduce/LogSegmentReader.java             |  148 +
 .../mapreduce/LogSegmentSplit.java              |   95 +
 .../conf/log4j.properties                       |    8 +-
 .../distributedlog-messaging/messaging-1.md     |   10 +-
 .../distributedlog-messaging/messaging-2.md     |   10 +-
 .../ConsoleProxyPartitionedMultiWriter.java     |  121 -
 .../messaging/ConsoleProxyRRMultiWriter.java    |   77 -
 .../messaging/IdenticalTransformer.java         |   25 -
 .../messaging/IntPartitioner.java               |   28 -
 .../messaging/PartitionedMultiWriter.java       |   50 -
 .../distributedlog/messaging/Partitioner.java   |   25 -
 .../distributedlog/messaging/RRMultiWriter.java |   94 -
 .../messaging/ReaderWithOffsets.java            |  132 -
 .../messaging/StreamTransformer.java            |  188 -
 .../distributedlog/messaging/Transformer.java   |   34 -
 .../ConsoleProxyPartitionedMultiWriter.java     |  121 +
 .../messaging/ConsoleProxyRRMultiWriter.java    |   77 +
 .../messaging/IdenticalTransformer.java         |   25 +
 .../messaging/IntPartitioner.java               |   28 +
 .../messaging/PartitionedMultiWriter.java       |   50 +
 .../distributedlog/messaging/Partitioner.java   |   25 +
 .../distributedlog/messaging/RRMultiWriter.java |   94 +
 .../messaging/ReaderWithOffsets.java            |  132 +
 .../messaging/StreamTransformer.java            |  188 +
 .../distributedlog/messaging/Transformer.java   |   34 +
 .../src/main/resources/findbugsExclude.xml      |    4 +-
 .../src/main/thrift/messaging.thrift            |    2 +-
 docs/_plugins/jekyll-rst/.gitignore             |    2 +-
 docs/_plugins/jekyll-rst/LICENSE.txt            |    2 +-
 docs/_plugins/jekyll-rst/README.rst             |    2 +-
 docs/_plugins/jekyll-rst/converter.rb           |    2 +-
 docs/_plugins/jekyll-rst/directives.py          |    2 +-
 docs/_plugins/jekyll-rst/transform.py           |    2 +-
 docs/admin_guide/operations.rst                 |    6 +-
 docs/deployment/cluster.rst                     |    6 +-
 .../bootstrap/glyphicons-halflings-regular.svg  |    2 +-
 docs/js/bootstrap.min.js                        |    2 +-
 docs/start/quickstart.rst                       |    4 +-
 docs/tutorials/basic-1.rst                      |   16 +-
 docs/tutorials/basic-2.rst                      |   10 +-
 docs/tutorials/basic-3.rst                      |   10 +-
 docs/tutorials/basic-4.rst                      |   10 +-
 docs/tutorials/basic-5.rst                      |   10 +-
 docs/tutorials/basic-6.rst                      |   10 +-
 docs/tutorials/messaging-1.rst                  |   10 +-
 docs/tutorials/messaging-2.rst                  |   10 +-
 docs/user_guide/implementation/storage.rst      |    2 +-
 pom.xml                                         |    3 +-
 scripts/integration/smoketest.sh                |    4 +-
 .../bootstrap/glyphicons-halflings-regular.svg  |    2 +-
 website/js/bootstrap.min.js                     |    2 +-
 1204 files changed, 100167 insertions(+), 100166 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/conf/log4j.properties
----------------------------------------------------------------------
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.zookeeper=INFO
 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


[12/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogReader.java
----------------------------------------------------------------------
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;
+    }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKAsyncLogWriter.java
----------------------------------------------------------------------
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());
+    }
+}


[11/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogManager.java
----------------------------------------------------------------------
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());
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKDistributedLogNamespace.java
----------------------------------------------------------------------
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.");
+    }
+}


[23/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/acl/ZKAccessControlManager.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/federated/FederatedZKLogMetadataStore.java
----------------------------------------------------------------------
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;
-            }
-        });
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentAllocator.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/impl/logsegment/BKLogSegmentEntryReader.java
----------------------------------------------------------------------
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;
-    }
-}


[47/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogClientImpl.java
----------------------------------------------------------------------
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 });
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/DistributedLogMultiStreamWriter.java
----------------------------------------------------------------------
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();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/MonitorServiceClient.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/monitor/package-info.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/OwnershipCache.java
----------------------------------------------------------------------
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;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/ownership/package-info.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ClusterClient.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/HostProvider.java
----------------------------------------------------------------------
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();
-
-}



[41/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/DefaultSpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
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});
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
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);
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/SpeculativeRequestExecutor.java
----------------------------------------------------------------------
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();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/speculative/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStats.java
----------------------------------------------------------------------
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);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/ClientStatsLogger.java
----------------------------------------------------------------------
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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStats.java
----------------------------------------------------------------------
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);
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OpStatsLogger.java
----------------------------------------------------------------------
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);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/OwnershipStatsLogger.java
----------------------------------------------------------------------
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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/stats/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/service/DLSocketAddress.java
----------------------------------------------------------------------
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();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClient.java
----------------------------------------------------------------------
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();
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/service/DistributedLogClientBuilder.java
----------------------------------------------------------------------
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;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/service/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/resources/findbugsExclude.xml
----------------------------------------------------------------------
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>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/TestDistributedLogMultiStreamWriter.java
----------------------------------------------------------------------
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();
-    }
-}


[28/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadAheadEntryReader.java
----------------------------------------------------------------------
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"));
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/ReadUtils.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/WriteLimiter.java
----------------------------------------------------------------------
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();
-    }
-}


[29/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/LogSegmentMetadata.java
----------------------------------------------------------------------
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;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/LogWriter.java
----------------------------------------------------------------------
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;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxLogSegmentSequenceNo.java
----------------------------------------------------------------------
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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/MaxTxId.java
----------------------------------------------------------------------
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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/MetadataAccessor.java
----------------------------------------------------------------------
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;
-
-}


[18/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogSegmentMetadataStoreUpdater.java
----------------------------------------------------------------------
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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/LogStreamMetadataStore.java
----------------------------------------------------------------------
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();
-
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataResolver.java
----------------------------------------------------------------------
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;
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/MetadataUpdater.java
----------------------------------------------------------------------
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);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/metadata/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespace.java
----------------------------------------------------------------------
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();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/DistributedLogNamespaceBuilder.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriver.java
----------------------------------------------------------------------
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);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceDriverManager.java
----------------------------------------------------------------------
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);
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/NamespaceWatcher.java
----------------------------------------------------------------------
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();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/namespace/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolver.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRacks.java
----------------------------------------------------------------------
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]);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/net/DNSResolverForRows.java
----------------------------------------------------------------------
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));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/net/NetUtils.java
----------------------------------------------------------------------
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);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/net/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRate.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/MovingAverageRateFactory.java
----------------------------------------------------------------------
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();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/rate/SampledMovingAverageRate.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/readahead/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstDLSNNotLessThanSelector.java
----------------------------------------------------------------------
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;
-    }
-}



[20/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKDistributedLock.java
----------------------------------------------------------------------
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;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/lock/ZKSessionLock.java
----------------------------------------------------------------------
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());
-            }
-        }
-
-    }
-}


[30/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogConstants.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/DistributedLogManager.java
----------------------------------------------------------------------
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();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/Entry.java
----------------------------------------------------------------------
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;
-
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryBuffer.java
----------------------------------------------------------------------
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;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/EntryPosition.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntry.java
----------------------------------------------------------------------
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());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryReader.java
----------------------------------------------------------------------
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";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/EnvelopedEntryWriter.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/LedgerReadPosition.java
----------------------------------------------------------------------
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);
-        }
-    }
-
-}
-
-

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/LocalDLMEmulator.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/LogReader.java
----------------------------------------------------------------------
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;
-}


[03/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/ReadAheadEntryReader.java
----------------------------------------------------------------------
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"));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/ReadUtils.java
----------------------------------------------------------------------
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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/WriteLimiter.java
----------------------------------------------------------------------
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();
+    }
+}


[17/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstRecordSelector.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/FirstTxIdNotLessThanSelector.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LastRecordSelector.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/selector/LogRecordSelector.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BKExceptionStatsLogger.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/BroadCastStatsLogger.java
----------------------------------------------------------------------
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));
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/stats/OpStatsListener.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionStateStore.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/subscription/SubscriptionsStore.java
----------------------------------------------------------------------
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);
-
-}


[37/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogReader.java
----------------------------------------------------------------------
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;
-    }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAsyncLogWriter.java
----------------------------------------------------------------------
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());
-    }
-}


[08/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogWriteHandler.java
----------------------------------------------------------------------
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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogReader.java
----------------------------------------------------------------------
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
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKSyncLogWriter.java
----------------------------------------------------------------------
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();
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKTransmitPacket.java
----------------------------------------------------------------------
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;
+    }
+
+}


[06/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/DistributedLogConfiguration.java
----------------------------------------------------------------------
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 #getMaxLogS

<TRUNCATED>


[49/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerReadBenchmark.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/LedgerStreamReader.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/ReadMode.java
----------------------------------------------------------------------
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
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/StreamBenchmark.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/SyncReaderBenchmark.java
----------------------------------------------------------------------
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);
-            }
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/stream/package-info.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/ShiftableRateLimiter.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/com/twitter/distributedlog/benchmark/utils/package-info.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Benchmarker.java
----------------------------------------------------------------------
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);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/DLWriterWorker.java
----------------------------------------------------------------------
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);
+                    }
+                });
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/ReaderWorker.java
----------------------------------------------------------------------
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();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-benchmark/src/main/java/org/apache/distributedlog/benchmark/Utils.java
----------------------------------------------------------------------
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;
+    }
+
+}



[10/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogHandler.java
----------------------------------------------------------------------
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));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogReadHandler.java
----------------------------------------------------------------------
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);
+        }
+    }
+}


[09/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/BKLogSegmentWriter.java
----------------------------------------------------------------------
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);
+    }
+
+}


[27/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClient.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/ZooKeeperClientBuilder.java
----------------------------------------------------------------------
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
-        );
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/AccessControlManager.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/DefaultAccessControlManager.java
----------------------------------------------------------------------
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() {
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/acl/package-info.java
----------------------------------------------------------------------
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;

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/DistributedLogAdmin.java
----------------------------------------------------------------------
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";
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/admin/package-info.java
----------------------------------------------------------------------
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;


[26/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/auditor/DLAuditor.java
----------------------------------------------------------------------
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();
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/DynamicQuorumConfigProvider.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/ImmutableQuorumConfigProvider.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocator.java
----------------------------------------------------------------------
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;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorDelegator.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorPool.java
----------------------------------------------------------------------
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));
-            }
-        });
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/LedgerAllocatorUtils.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfig.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/QuorumConfigProvider.java
----------------------------------------------------------------------
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();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/bk/SimpleLedgerAllocator.java
----------------------------------------------------------------------
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());
-            }
-        });
-    }
-
-}



[44/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/service/DistributedLogClientBuilder.java
----------------------------------------------------------------------
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;
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/service/package-info.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/ClientConfig.java
----------------------------------------------------------------------
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;
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/org/apache/distributedlog/client/DistributedLogClientImpl.java
----------------------------------------------------------------------
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 });
+        }
+    }
+
+}


[46/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClient.java
----------------------------------------------------------------------
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();
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyClientManager.java
----------------------------------------------------------------------
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();
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/ProxyListener.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/proxy/package-info.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/DefaultRegionResolver.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/RegionResolver.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/resolver/package-info.java
----------------------------------------------------------------------
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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/ConsistentHashRoutingService.java
----------------------------------------------------------------------
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);
-            }
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/NameServerSet.java
----------------------------------------------------------------------
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
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RegionsRoutingService.java
----------------------------------------------------------------------
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);
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingService.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingServiceProvider.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/main/java/com/twitter/distributedlog/client/routing/RoutingUtils.java
----------------------------------------------------------------------
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);
-    }
-
-}


[40/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/ownership/TestOwnershipCache.java
----------------------------------------------------------------------
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));
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockDistributedLogServices.java
----------------------------------------------------------------------
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);
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockProxyClientBuilder.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/MockThriftClient.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/proxy/TestProxyClientManager.java
----------------------------------------------------------------------
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());
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestConsistentHashRoutingService.java
----------------------------------------------------------------------
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);
-        }
-
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestInetNameResolution.java
----------------------------------------------------------------------
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());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRegionsRoutingService.java
----------------------------------------------------------------------
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
-        }
-    }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/routing/TestRoutingService.java
----------------------------------------------------------------------
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);
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
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());
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/com/twitter/distributedlog/service/TestDistributedLogClientBuilder.java
----------------------------------------------------------------------
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);
-    }
-}


[38/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/routing/TestRoutingService.java
----------------------------------------------------------------------
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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/client/speculative/TestDefaultSpeculativeRequestExecutionPolicy.java
----------------------------------------------------------------------
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());
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-client/src/test/java/org/apache/distributedlog/service/TestDistributedLogClientBuilder.java
----------------------------------------------------------------------
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);
+    }
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/conf/log4j.properties
----------------------------------------------------------------------
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.zookeeper=INFO
 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

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/conf/zookeeper.conf.dynamic.template
----------------------------------------------------------------------
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
@@ -1 +1 @@
-#/**# * Copyright 2007 The Apache Software Foundation# *# * Licensed to the Apache Software Foundation (ASF) under one# * or more contributor license agreements.  See the NOTICE file# * distributed with this work for additional information# * regarding copyright ownership.  The ASF licenses this file# * to you under the Apache License, Version 2.0 (the# * "License"); you may not use this file except in compliance# * with the License.  You may obtain a copy of the License at# *# *     http://www.apache.org/licenses/LICENSE-2.0# *# * Unless required by applicable law or agreed to in writing, software# * distributed under the License is distributed on an "AS IS" BASIS,# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.# * See the License for the specific language governing permissions and# * limitations under the License.# */server.1=127.0.0.1:2710:3710:participant;0.0.0.0:2181
\ No newline at end of file
+#/**# * Copyright 2007 The Apache Software Foundation# *# * Licensed to the Apache Software Foundation (ASF) under one# * or more contributor license agreements.  See the NOTICE file# * distributed with this work for additional information# * regarding copyright ownership.  The ASF licenses this file# * to you under the Apache License, Version 2.0 (the# * "License"); you may not use this file except in compliance# * with the License.  You may obtain a copy of the License at# *# *     http://www.apache.org/licenses/LICENSE-2.0# *# * Unless required by applicable law or agreed to in writing, software# * distributed under the License is distributed on an "AS IS" BASIS,# * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.# * See the License for the specific language governing permissions and# * limitations under the License.# */server.1=127.0.0.1:2710:3710:participant;0.0.0.0:2181

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/pom.xml
----------------------------------------------------------------------
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>

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamReader.java
----------------------------------------------------------------------
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;
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/AppendOnlyStreamWriter.java
----------------------------------------------------------------------
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
-        }
-    }
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogReader.java
----------------------------------------------------------------------
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);
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncLogWriter.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/AsyncNotification.java
----------------------------------------------------------------------
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();
-}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/com/twitter/distributedlog/BKAbstractLogWriter.java
----------------------------------------------------------------------
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;
-    }
-
-}


[04/51] [partial] incubator-distributedlog git commit: DL-4: Repackage the source under apache namespace

Posted by si...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/LogSegmentMetadata.java
----------------------------------------------------------------------
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;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/LogWriter.java
----------------------------------------------------------------------
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;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/MaxLogSegmentSequenceNo.java
----------------------------------------------------------------------
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);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/MaxTxId.java
----------------------------------------------------------------------
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);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-distributedlog/blob/b44820b5/distributedlog-core/src/main/java/org/apache/distributedlog/MetadataAccessor.java
----------------------------------------------------------------------
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;
+
+}